diff --git a/python/pyspark/shell.py b/python/pyspark/shell.py index adaa3b5a799383c34fb402aa1c640bda4bfa0871..ef46d3065e5e8df735638bc5e1f87f47da3702c7 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 diff --git a/python/pyspark/sql/session.py b/python/pyspark/sql/session.py index 0e04b88265fa175d874aebb62a507736ec5635ef..241947537fa29e510e36eb6a6f5b6e73c2e8f18b 100644 --- a/python/pyspark/sql/session.py +++ b/python/pyspark/sql/session.py @@ -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):