Skip to content

Commit

Permalink
[SPARK-15417][SQL][PYTHON] PySpark shell always uses in-memory catalog
Browse files Browse the repository at this point in the history
## 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 <andrew@databricks.com>

Closes #13203 from andrewor14/fix-pyspark-shell.
  • Loading branch information
Andrew Or committed May 20, 2016
1 parent 2573750 commit c32b1b1
Show file tree
Hide file tree
Showing 2 changed files with 11 additions and 3 deletions.
8 changes: 5 additions & 3 deletions python/pyspark/shell.py
Original file line number Diff line number Diff line change
Expand Up @@ -35,12 +35,11 @@
if os.environ.get("SPARK_EXECUTOR_URI"):
SparkContext.setSystemProperty("spark.executor.uri", os.environ["SPARK_EXECUTOR_URI"])

sc = SparkContext()
atexit.register(lambda: sc.stop())
SparkContext._ensure_initialized()

try:
# Try to access HiveConf, it will raise exception if Hive is not added
sc._jvm.org.apache.hadoop.hive.conf.HiveConf()
SparkContext._jvm.org.apache.hadoop.hive.conf.HiveConf()
spark = SparkSession.builder\
.enableHiveSupport()\
.getOrCreate()
Expand All @@ -49,6 +48,9 @@
except TypeError:
spark = SparkSession(sc)

sc = spark.sparkContext
atexit.register(lambda: sc.stop())

# for compatibility
sqlContext = spark._wrapped
sqlCtx = sqlContext
Expand Down
6 changes: 6 additions & 0 deletions python/pyspark/sql/session.py
Original file line number Diff line number Diff line change
Expand Up @@ -202,6 +202,12 @@ def newSession(self):
"""
return self.__class__(self._sc, self._jsparkSession.newSession())

@property
@since(2.0)
def sparkContext(self):
"""Returns the underlying :class:`SparkContext`."""
return self._sc

@property
@since(2.0)
def conf(self):
Expand Down

0 comments on commit c32b1b1

Please sign in to comment.