Skip to content
Snippets Groups Projects
Commit c32b1b16 authored by Andrew Or's avatar Andrew Or
Browse files

[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 <andrew@databricks.com>

Closes #13203 from andrewor14/fix-pyspark-shell.
parent 25737501
No related branches found
No related tags found
No related merge requests found
......@@ -35,12 +35,11 @@ from pyspark.storagelevel import StorageLevel
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()
......@@ -49,6 +48,9 @@ except py4j.protocol.Py4JError:
except TypeError:
spark = SparkSession(sc)
sc = spark.sparkContext
atexit.register(lambda: sc.stop())
# for compatibility
sqlContext = spark._wrapped
sqlCtx = sqlContext
......
......@@ -202,6 +202,12 @@ class SparkSession(object):
"""
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):
......
0% Loading or .
You are about to add 0 people to the discussion. Proceed with caution.
Finish editing this message first!
Please register or to comment