aboutsummaryrefslogtreecommitdiff
path: root/python/pyspark/sql
diff options
context:
space:
mode:
authorJeff Zhang <zjffdu@apache.org>2016-06-17 22:57:38 -0700
committerDavies Liu <davies.liu@gmail.com>2016-06-17 22:57:38 -0700
commit898cb652558ce31a9883448334a45ff0265e2241 (patch)
tree88ede6ded6d62988bf8f54469eff30e361f97afd /python/pyspark/sql
parent4c64e88d5ba4c36cbdbc903376492f0f43401e4e (diff)
downloadspark-898cb652558ce31a9883448334a45ff0265e2241.tar.gz
spark-898cb652558ce31a9883448334a45ff0265e2241.tar.bz2
spark-898cb652558ce31a9883448334a45ff0265e2241.zip
[SPARK-15803] [PYSPARK] Support with statement syntax for SparkSession
## What changes were proposed in this pull request? Support with statement syntax for SparkSession in pyspark ## How was this patch tested? Manually verify it. Although I can add unit test for it, it would affect other unit test because the SparkContext is stopped after the with statement. Author: Jeff Zhang <zjffdu@apache.org> Closes #13541 from zjffdu/SPARK-15803.
Diffstat (limited to 'python/pyspark/sql')
-rw-r--r--python/pyspark/sql/session.py16
1 files changed, 16 insertions, 0 deletions
diff --git a/python/pyspark/sql/session.py b/python/pyspark/sql/session.py
index 6edbd59856..0c8024e4a8 100644
--- a/python/pyspark/sql/session.py
+++ b/python/pyspark/sql/session.py
@@ -581,6 +581,22 @@ class SparkSession(object):
"""
self._sc.stop()
+ @since(2.0)
+ def __enter__(self):
+ """
+ Enable 'with SparkSession.builder.(...).getOrCreate() as session: app' syntax.
+ """
+ return self
+
+ @since(2.0)
+ def __exit__(self, exc_type, exc_val, exc_tb):
+ """
+ Enable 'with SparkSession.builder.(...).getOrCreate() as session: app' syntax.
+
+ Specifically stop the SparkSession on exit of the with block.
+ """
+ self.stop()
+
def _test():
import os