From 99a9473127ec389283ac4ec3b721d2e34434e647 Mon Sep 17 00:00:00 2001 From: Jeff Zhang Date: Wed, 12 Apr 2017 10:54:50 -0700 Subject: [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 Closes #16906 from zjffdu/SPARK-19570. --- python/pyspark/shell.py | 22 ++++++++++++++++------ 1 file changed, 16 insertions(+), 6 deletions(-) diff --git a/python/pyspark/shell.py b/python/pyspark/shell.py index c1917d2be6..b5fcf7092d 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 -- cgit v1.2.3