From c32b1b162e7e5ecc5c823f79ba9f23cbd1407dbf Mon Sep 17 00:00:00 2001 From: Andrew Or Date: Thu, 19 May 2016 23:44:10 -0700 Subject: [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 Closes #13203 from andrewor14/fix-pyspark-shell. --- python/pyspark/shell.py | 8 +++++--- 1 file changed, 5 insertions(+), 3 deletions(-) (limited to 'python/pyspark/shell.py') diff --git a/python/pyspark/shell.py b/python/pyspark/shell.py index adaa3b5a79..ef46d3065e 100644 --- a/python/pyspark/shell.py +++ b/python/pyspark/shell.py @@ -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 -- cgit v1.2.3