Skip to content

Commit c32b1b1

Browse files
author
Andrew Or
committed
[SPARK-15417][SQL][PYTHON] PySpark shell always uses in-memory catalog
## What changes were proposed in this pull request? There is no way to use the Hive catalog in `pyspark-shell`. This is because we used to create a `SparkContext` before calling `SparkSession.enableHiveSupport().getOrCreate()`, which just gets the existing `SparkContext` instead of creating a new one. As a result, `spark.sql.catalogImplementation` was never propagated. ## How was this patch tested? Manual. Author: Andrew Or <[email protected]> Closes #13203 from andrewor14/fix-pyspark-shell.
1 parent 2573750 commit c32b1b1

File tree

2 files changed

+11
-3
lines changed

2 files changed

+11
-3
lines changed

python/pyspark/shell.py

Lines changed: 5 additions & 3 deletions
Original file line numberDiff line numberDiff line change
@@ -35,12 +35,11 @@
3535
if os.environ.get("SPARK_EXECUTOR_URI"):
3636
SparkContext.setSystemProperty("spark.executor.uri", os.environ["SPARK_EXECUTOR_URI"])
3737

38-
sc = SparkContext()
39-
atexit.register(lambda: sc.stop())
38+
SparkContext._ensure_initialized()
4039

4140
try:
4241
# Try to access HiveConf, it will raise exception if Hive is not added
43-
sc._jvm.org.apache.hadoop.hive.conf.HiveConf()
42+
SparkContext._jvm.org.apache.hadoop.hive.conf.HiveConf()
4443
spark = SparkSession.builder\
4544
.enableHiveSupport()\
4645
.getOrCreate()
@@ -49,6 +48,9 @@
4948
except TypeError:
5049
spark = SparkSession(sc)
5150

51+
sc = spark.sparkContext
52+
atexit.register(lambda: sc.stop())
53+
5254
# for compatibility
5355
sqlContext = spark._wrapped
5456
sqlCtx = sqlContext

python/pyspark/sql/session.py

Lines changed: 6 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -202,6 +202,12 @@ def newSession(self):
202202
"""
203203
return self.__class__(self._sc, self._jsparkSession.newSession())
204204

205+
@property
206+
@since(2.0)
207+
def sparkContext(self):
208+
"""Returns the underlying :class:`SparkContext`."""
209+
return self._sc
210+
205211
@property
206212
@since(2.0)
207213
def conf(self):

0 commit comments

Comments
 (0)