aboutsummaryrefslogtreecommitdiff
path: root/python/pyspark/sql/dataframe.py
diff options
context:
space:
mode:
authorSean Zhong <seanzhong@databricks.com>2016-05-12 15:51:53 +0800
committerCheng Lian <lian@databricks.com>2016-05-12 15:51:53 +0800
commit33c6eb5218ce3c31cc9f632a67fd2c7057569683 (patch)
treeb8c84c24107bf1ece596450ef3a3eec26df1f21d /python/pyspark/sql/dataframe.py
parent5207a005cc86618907b8f467abc03eacef485ecd (diff)
downloadspark-33c6eb5218ce3c31cc9f632a67fd2c7057569683.tar.gz
spark-33c6eb5218ce3c31cc9f632a67fd2c7057569683.tar.bz2
spark-33c6eb5218ce3c31cc9f632a67fd2c7057569683.zip
[SPARK-15171][SQL] Deprecate registerTempTable and add dataset.createTempView
## What changes were proposed in this pull request? Deprecates registerTempTable and add dataset.createTempView, dataset.createOrReplaceTempView. ## How was this patch tested? Unit tests. Author: Sean Zhong <seanzhong@databricks.com> Closes #12945 from clockfly/spark-15171.
Diffstat (limited to 'python/pyspark/sql/dataframe.py')
-rw-r--r--python/pyspark/sql/dataframe.py51
1 files changed, 48 insertions, 3 deletions
diff --git a/python/pyspark/sql/dataframe.py b/python/pyspark/sql/dataframe.py
index 49b4818bcc..a0264ce1ac 100644
--- a/python/pyspark/sql/dataframe.py
+++ b/python/pyspark/sql/dataframe.py
@@ -119,11 +119,55 @@ class DataFrame(object):
that was used to create this :class:`DataFrame`.
>>> df.registerTempTable("people")
- >>> df2 = sqlContext.sql("select * from people")
+ >>> df2 = spark.sql("select * from people")
>>> sorted(df.collect()) == sorted(df2.collect())
True
+ >>> spark.catalog.dropTempView("people")
+
+ .. note:: Deprecated in 2.0, use createOrReplaceTempView instead.
+ """
+ self._jdf.createOrReplaceTempView(name)
+
+ @since(2.0)
+ def createTempView(self, name):
+ """Creates a 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`.
+ throws :class:`TempTableAlreadyExistsException`, if the view name already exists in the
+ catalog.
+
+ >>> df.createTempView("people")
+ >>> df2 = spark.sql("select * from people")
+ >>> sorted(df.collect()) == sorted(df2.collect())
+ True
+ >>> df.createTempView("people") # doctest: +IGNORE_EXCEPTION_DETAIL
+ Traceback (most recent call last):
+ ...
+ Py4JJavaError: ...
+ : org.apache.spark.sql.catalyst.analysis.TempTableAlreadyExistsException...
+ >>> spark.catalog.dropTempView("people")
+
+ """
+ self._jdf.createTempView(name)
+
+ @since(2.0)
+ def createOrReplaceTempView(self, name):
+ """Creates or replaces a 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`.
+
+ >>> df.createOrReplaceTempView("people")
+ >>> df2 = df.filter(df.age > 3)
+ >>> df2.createOrReplaceTempView("people")
+ >>> df3 = spark.sql("select * from people")
+ >>> sorted(df3.collect()) == sorted(df2.collect())
+ True
+ >>> spark.catalog.dropTempView("people")
+
"""
- self._jdf.registerTempTable(name)
+ self._jdf.createOrReplaceTempView(name)
@property
@since(1.4)
@@ -1479,12 +1523,13 @@ class DataFrameStatFunctions(object):
def _test():
import doctest
from pyspark.context import SparkContext
- from pyspark.sql import Row, SQLContext
+ from pyspark.sql import Row, SQLContext, SparkSession
import pyspark.sql.dataframe
globs = pyspark.sql.dataframe.__dict__.copy()
sc = SparkContext('local[4]', 'PythonTest')
globs['sc'] = sc
globs['sqlContext'] = SQLContext(sc)
+ globs['spark'] = SparkSession(sc)
globs['df'] = sc.parallelize([(2, 'Alice'), (5, 'Bob')])\
.toDF(StructType([StructField('age', IntegerType()),
StructField('name', StringType())]))