Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

[SPARK-19236][SQL][BACKPORT-2.2] Added createOrReplaceGlobalTempView method #18167

Closed
wants to merge 1 commit into from
Closed
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
17 changes: 17 additions & 0 deletions python/pyspark/sql/dataframe.py
Original file line number Diff line number Diff line change
Expand Up @@ -191,6 +191,23 @@ def createGlobalTempView(self, name):
"""
self._jdf.createGlobalTempView(name)

@since(2.2)
def createOrReplaceGlobalTempView(self, name):
"""Creates or replaces a global temporary view using the given name.

The lifetime of this temporary view is tied to this Spark application.

>>> df.createOrReplaceGlobalTempView("people")
>>> df2 = df.filter(df.age > 3)
>>> df2.createOrReplaceGlobalTempView("people")
>>> df3 = spark.sql("select * from global_temp.people")
>>> sorted(df3.collect()) == sorted(df2.collect())
True
>>> spark.catalog.dropGlobalTempView("people")

"""
self._jdf.createOrReplaceGlobalTempView(name)

@property
@since(1.4)
def write(self):
Expand Down
16 changes: 16 additions & 0 deletions sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala
Original file line number Diff line number Diff line change
Expand Up @@ -2657,6 +2657,22 @@ class Dataset[T] private[sql](
createTempViewCommand(viewName, replace = false, global = true)
}

/**
* Creates or replaces a global temporary view using the given name. The lifetime of this
* temporary view is tied to this Spark application.
*
* Global temporary view is cross-session. Its lifetime is the lifetime of the Spark application,
* i.e. it will be automatically dropped when the application terminates. It's tied to a system
* preserved database `_global_temp`, and we must use the qualified name to refer a global temp
* view, e.g. `SELECT * FROM _global_temp.view1`.
*
* @group basic
* @since 2.2.0
*/
def createOrReplaceGlobalTempView(viewName: String): Unit = withPlan {
createTempViewCommand(viewName, replace = true, global = true)
}

private def createTempViewCommand(
viewName: String,
replace: Boolean,
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -35,39 +35,47 @@ class GlobalTempViewSuite extends QueryTest with SharedSQLContext {
private var globalTempDB: String = _

test("basic semantic") {
sql("CREATE GLOBAL TEMP VIEW src AS SELECT 1, 'a'")
try {
sql("CREATE GLOBAL TEMP VIEW src AS SELECT 1, 'a'")

// If there is no database in table name, we should try local temp view first, if not found,
// try table/view in current database, which is "default" in this case. So we expect
// NoSuchTableException here.
intercept[NoSuchTableException](spark.table("src"))

// If there is no database in table name, we should try local temp view first, if not found,
// try table/view in current database, which is "default" in this case. So we expect
// NoSuchTableException here.
intercept[NoSuchTableException](spark.table("src"))
// Use qualified name to refer to the global temp view explicitly.
checkAnswer(spark.table(s"$globalTempDB.src"), Row(1, "a"))

// Use qualified name to refer to the global temp view explicitly.
checkAnswer(spark.table(s"$globalTempDB.src"), Row(1, "a"))
// Table name without database will never refer to a global temp view.
intercept[NoSuchTableException](sql("DROP VIEW src"))

// Table name without database will never refer to a global temp view.
intercept[NoSuchTableException](sql("DROP VIEW src"))
sql(s"DROP VIEW $globalTempDB.src")
// The global temp view should be dropped successfully.
intercept[NoSuchTableException](spark.table(s"$globalTempDB.src"))

sql(s"DROP VIEW $globalTempDB.src")
// The global temp view should be dropped successfully.
intercept[NoSuchTableException](spark.table(s"$globalTempDB.src"))
// We can also use Dataset API to create global temp view
Seq(1 -> "a").toDF("i", "j").createGlobalTempView("src")
checkAnswer(spark.table(s"$globalTempDB.src"), Row(1, "a"))

// We can also use Dataset API to create global temp view
Seq(1 -> "a").toDF("i", "j").createGlobalTempView("src")
checkAnswer(spark.table(s"$globalTempDB.src"), Row(1, "a"))
// Use qualified name to rename a global temp view.
sql(s"ALTER VIEW $globalTempDB.src RENAME TO src2")
intercept[NoSuchTableException](spark.table(s"$globalTempDB.src"))
checkAnswer(spark.table(s"$globalTempDB.src2"), Row(1, "a"))

// Use qualified name to rename a global temp view.
sql(s"ALTER VIEW $globalTempDB.src RENAME TO src2")
intercept[NoSuchTableException](spark.table(s"$globalTempDB.src"))
checkAnswer(spark.table(s"$globalTempDB.src2"), Row(1, "a"))
// Use qualified name to alter a global temp view.
sql(s"ALTER VIEW $globalTempDB.src2 AS SELECT 2, 'b'")
checkAnswer(spark.table(s"$globalTempDB.src2"), Row(2, "b"))

// Use qualified name to alter a global temp view.
sql(s"ALTER VIEW $globalTempDB.src2 AS SELECT 2, 'b'")
checkAnswer(spark.table(s"$globalTempDB.src2"), Row(2, "b"))
// We can also use Catalog API to drop global temp view
spark.catalog.dropGlobalTempView("src2")
intercept[NoSuchTableException](spark.table(s"$globalTempDB.src2"))

// We can also use Catalog API to drop global temp view
spark.catalog.dropGlobalTempView("src2")
intercept[NoSuchTableException](spark.table(s"$globalTempDB.src2"))
// We can also use Dataset API to replace global temp view
Seq(2 -> "b").toDF("i", "j").createOrReplaceGlobalTempView("src")
checkAnswer(spark.table(s"$globalTempDB.src"), Row(2, "b"))
} finally {
spark.catalog.dropGlobalTempView("src")
}
}

test("global temp view is shared among all sessions") {
Expand Down Expand Up @@ -106,7 +114,7 @@ class GlobalTempViewSuite extends QueryTest with SharedSQLContext {
test("CREATE TABLE LIKE should work for global temp view") {
try {
sql("CREATE GLOBAL TEMP VIEW src AS SELECT 1 AS a, '2' AS b")
sql(s"CREATE TABLE cloned LIKE ${globalTempDB}.src")
sql(s"CREATE TABLE cloned LIKE $globalTempDB.src")
val tableMeta = spark.sessionState.catalog.getTableMetadata(TableIdentifier("cloned"))
assert(tableMeta.schema == new StructType().add("a", "int", false).add("b", "string", false))
} finally {
Expand Down