aboutsummaryrefslogtreecommitdiff
path: root/python/pyspark
diff options
context:
space:
mode:
authorWenchen Fan <wenchen@databricks.com>2016-10-10 15:48:57 +0800
committerWenchen Fan <wenchen@databricks.com>2016-10-10 15:48:57 +0800
commit23ddff4b2b2744c3dc84d928e144c541ad5df376 (patch)
treef61b64ea46adbd1eb424a0bbb8e8e383d1ee4e3b /python/pyspark
parent16590030c15b32e83b584283697b6f783cffe043 (diff)
downloadspark-23ddff4b2b2744c3dc84d928e144c541ad5df376.tar.gz
spark-23ddff4b2b2744c3dc84d928e144c541ad5df376.tar.bz2
spark-23ddff4b2b2744c3dc84d928e144c541ad5df376.zip
[SPARK-17338][SQL] add global temp view
## What changes were proposed in this pull request? Global temporary view is a cross-session temporary view, which means it's shared among all sessions. 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`(configurable via SparkConf), and we must use the qualified name to refer a global temp view, e.g. SELECT * FROM global_temp.view1. changes for `SessionCatalog`: 1. add a new field `gloabalTempViews: GlobalTempViewManager`, to access the shared global temp views, and the global temp db name. 2. `createDatabase` will fail if users wanna create `global_temp`, which is system preserved. 3. `setCurrentDatabase` will fail if users wanna set `global_temp`, which is system preserved. 4. add `createGlobalTempView`, which is used in `CreateViewCommand` to create global temp views. 5. add `dropGlobalTempView`, which is used in `CatalogImpl` to drop global temp view. 6. add `alterTempViewDefinition`, which is used in `AlterViewAsCommand` to update the view definition for local/global temp views. 7. `renameTable`/`dropTable`/`isTemporaryTable`/`lookupRelation`/`getTempViewOrPermanentTableMetadata`/`refreshTable` will handle global temp views. changes for SQL commands: 1. `CreateViewCommand`/`AlterViewAsCommand` is updated to support global temp views 2. `ShowTablesCommand` outputs a new column `database`, which is used to distinguish global and local temp views. 3. other commands can also handle global temp views if they call `SessionCatalog` APIs which accepts global temp views, e.g. `DropTableCommand`, `AlterTableRenameCommand`, `ShowColumnsCommand`, etc. changes for other public API 1. add a new method `dropGlobalTempView` in `Catalog` 2. `Catalog.findTable` can find global temp view 3. add a new method `createGlobalTempView` in `Dataset` ## How was this patch tested? new tests in `SQLViewSuite` Author: Wenchen Fan <wenchen@databricks.com> Closes #14897 from cloud-fan/global-temp-view.
Diffstat (limited to 'python/pyspark')
-rw-r--r--python/pyspark/sql/catalog.py18
-rw-r--r--python/pyspark/sql/context.py2
-rw-r--r--python/pyspark/sql/dataframe.py25
3 files changed, 41 insertions, 4 deletions
diff --git a/python/pyspark/sql/catalog.py b/python/pyspark/sql/catalog.py
index 3c5030722f..df3bf4254d 100644
--- a/python/pyspark/sql/catalog.py
+++ b/python/pyspark/sql/catalog.py
@@ -167,7 +167,7 @@ class Catalog(object):
@since(2.0)
def dropTempView(self, viewName):
- """Drops the temporary view with the given view name in the catalog.
+ """Drops the local temporary view with the given view name in the catalog.
If the view has been cached before, then it will also be uncached.
>>> spark.createDataFrame([(1, 1)]).createTempView("my_table")
@@ -181,6 +181,22 @@ class Catalog(object):
"""
self._jcatalog.dropTempView(viewName)
+ @since(2.1)
+ def dropGlobalTempView(self, viewName):
+ """Drops the global temporary view with the given view name in the catalog.
+ If the view has been cached before, then it will also be uncached.
+
+ >>> spark.createDataFrame([(1, 1)]).createGlobalTempView("my_table")
+ >>> spark.table("global_temp.my_table").collect()
+ [Row(_1=1, _2=1)]
+ >>> spark.catalog.dropGlobalTempView("my_table")
+ >>> spark.table("global_temp.my_table") # doctest: +IGNORE_EXCEPTION_DETAIL
+ Traceback (most recent call last):
+ ...
+ AnalysisException: ...
+ """
+ self._jcatalog.dropGlobalTempView(viewName)
+
@ignore_unicode_prefix
@since(2.0)
def registerFunction(self, name, f, returnType=StringType()):
diff --git a/python/pyspark/sql/context.py b/python/pyspark/sql/context.py
index 7482be8bda..8264dcf8a9 100644
--- a/python/pyspark/sql/context.py
+++ b/python/pyspark/sql/context.py
@@ -386,7 +386,7 @@ class SQLContext(object):
>>> sqlContext.registerDataFrameAsTable(df, "table1")
>>> df2 = sqlContext.tables()
>>> df2.filter("tableName = 'table1'").first()
- Row(tableName=u'table1', isTemporary=True)
+ Row(database=u'', tableName=u'table1', isTemporary=True)
"""
if dbName is None:
return DataFrame(self._ssql_ctx.tables(), self)
diff --git a/python/pyspark/sql/dataframe.py b/python/pyspark/sql/dataframe.py
index 0ac481a8a8..14e80ea461 100644
--- a/python/pyspark/sql/dataframe.py
+++ b/python/pyspark/sql/dataframe.py
@@ -131,7 +131,7 @@ class DataFrame(object):
@since(2.0)
def createTempView(self, name):
- """Creates a temporary view with this DataFrame.
+ """Creates a local temporary view with this DataFrame.
The lifetime of this temporary table is tied to the :class:`SparkSession`
that was used to create this :class:`DataFrame`.
@@ -153,7 +153,7 @@ class DataFrame(object):
@since(2.0)
def createOrReplaceTempView(self, name):
- """Creates or replaces a temporary view with this DataFrame.
+ """Creates or replaces a local temporary view with this DataFrame.
The lifetime of this temporary table is tied to the :class:`SparkSession`
that was used to create this :class:`DataFrame`.
@@ -169,6 +169,27 @@ class DataFrame(object):
"""
self._jdf.createOrReplaceTempView(name)
+ @since(2.1)
+ def createGlobalTempView(self, name):
+ """Creates a global temporary view with this DataFrame.
+
+ The lifetime of this temporary view is tied to this Spark application.
+ throws :class:`TempTableAlreadyExistsException`, if the view name already exists in the
+ catalog.
+
+ >>> df.createGlobalTempView("people")
+ >>> df2 = spark.sql("select * from global_temp.people")
+ >>> sorted(df.collect()) == sorted(df2.collect())
+ True
+ >>> df.createGlobalTempView("people") # doctest: +IGNORE_EXCEPTION_DETAIL
+ Traceback (most recent call last):
+ ...
+ AnalysisException: u"Temporary table 'people' already exists;"
+ >>> spark.catalog.dropGlobalTempView("people")
+
+ """
+ self._jdf.createGlobalTempView(name)
+
@property
@since(1.4)
def write(self):