You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@spark.apache.org by ho...@apache.org on 2017/04/12 17:54:56 UTC

spark git commit: [SPARK-19570][PYSPARK] Allow to disable hive in pyspark shell

Repository: spark
Updated Branches:
  refs/heads/master 540855382 -> 99a947312


[SPARK-19570][PYSPARK] Allow to disable hive in pyspark shell

## What changes were proposed in this pull request?

SPARK-15236 do this for scala shell, this ticket is for pyspark shell. This is not only for pyspark itself, but can also benefit downstream project like livy which use shell.py for its interactive session. For now, livy has no control of whether enable hive or not.

## How was this patch tested?

I didn't find a way to add test for it. Just manually test it.
Run `bin/pyspark --master local --conf spark.sql.catalogImplementation=in-memory` and verify hive is not enabled.

Author: Jeff Zhang <zj...@apache.org>

Closes #16906 from zjffdu/SPARK-19570.


Project: http://git-wip-us.apache.org/repos/asf/spark/repo
Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/99a94731
Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/99a94731
Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/99a94731

Branch: refs/heads/master
Commit: 99a9473127ec389283ac4ec3b721d2e34434e647
Parents: 5408553
Author: Jeff Zhang <zj...@apache.org>
Authored: Wed Apr 12 10:54:50 2017 -0700
Committer: Holden Karau <ho...@us.ibm.com>
Committed: Wed Apr 12 10:54:50 2017 -0700

----------------------------------------------------------------------
 python/pyspark/shell.py | 22 ++++++++++++++++------
 1 file changed, 16 insertions(+), 6 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/spark/blob/99a94731/python/pyspark/shell.py
----------------------------------------------------------------------
diff --git a/python/pyspark/shell.py b/python/pyspark/shell.py
index c1917d2..b5fcf70 100644
--- a/python/pyspark/shell.py
+++ b/python/pyspark/shell.py
@@ -24,13 +24,13 @@ This file is designed to be launched as a PYTHONSTARTUP script.
 import atexit
 import os
 import platform
+import warnings
 
 import py4j
 
-import pyspark
+from pyspark import SparkConf
 from pyspark.context import SparkContext
 from pyspark.sql import SparkSession, SQLContext
-from pyspark.storagelevel import StorageLevel
 
 if os.environ.get("SPARK_EXECUTOR_URI"):
     SparkContext.setSystemProperty("spark.executor.uri", os.environ["SPARK_EXECUTOR_URI"])
@@ -39,13 +39,23 @@ SparkContext._ensure_initialized()
 
 try:
     # Try to access HiveConf, it will raise exception if Hive is not added
-    SparkContext._jvm.org.apache.hadoop.hive.conf.HiveConf()
-    spark = SparkSession.builder\
-        .enableHiveSupport()\
-        .getOrCreate()
+    conf = SparkConf()
+    if conf.get('spark.sql.catalogImplementation', 'hive').lower() == 'hive':
+        SparkContext._jvm.org.apache.hadoop.hive.conf.HiveConf()
+        spark = SparkSession.builder\
+            .enableHiveSupport()\
+            .getOrCreate()
+    else:
+        spark = SparkSession.builder.getOrCreate()
 except py4j.protocol.Py4JError:
+    if conf.get('spark.sql.catalogImplementation', '').lower() == 'hive':
+        warnings.warn("Fall back to non-hive support because failing to access HiveConf, "
+                      "please make sure you build spark with hive")
     spark = SparkSession.builder.getOrCreate()
 except TypeError:
+    if conf.get('spark.sql.catalogImplementation', '').lower() == 'hive':
+        warnings.warn("Fall back to non-hive support because failing to access HiveConf, "
+                      "please make sure you build spark with hive")
     spark = SparkSession.builder.getOrCreate()
 
 sc = spark.sparkContext


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@spark.apache.org
For additional commands, e-mail: commits-help@spark.apache.org