diff options
author | Jeff Zhang <zjffdu@apache.org> | 2017-02-28 22:21:29 -0800 |
---|---|---|
committer | Felix Cheung <felixcheung@apache.org> | 2017-02-28 22:21:29 -0800 |
commit | 7315880568fd07d4dfb9f76d538f220e9d320c6f (patch) | |
tree | de794f163b04f0363f29ce761464dd773fe57dfb /sql/core/src/main/scala/org | |
parent | 0fe8020f3aaf61c9992b6bcc5dba7ae8f751bab7 (diff) | |
download | spark-7315880568fd07d4dfb9f76d538f220e9d320c6f.tar.gz spark-7315880568fd07d4dfb9f76d538f220e9d320c6f.tar.bz2 spark-7315880568fd07d4dfb9f76d538f220e9d320c6f.zip |
[SPARK-19572][SPARKR] Allow to disable hive in sparkR shell
## What changes were proposed in this pull request?
SPARK-15236 do this for scala shell, this ticket is for sparkR shell. This is not only for sparkR itself, but can also benefit downstream project like livy which use shell.R for its interactive session. For now, livy has no control of whether enable hive or not.
## How was this patch tested?
Tested it manually, run `bin/sparkR --master local --conf spark.sql.catalogImplementation=in-memory` and verify hive is not enabled.
Author: Jeff Zhang <zjffdu@apache.org>
Closes #16907 from zjffdu/SPARK-19572.
Diffstat (limited to 'sql/core/src/main/scala/org')
-rw-r--r-- | sql/core/src/main/scala/org/apache/spark/sql/api/r/SQLUtils.scala | 6 |
1 files changed, 4 insertions, 2 deletions
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/api/r/SQLUtils.scala b/sql/core/src/main/scala/org/apache/spark/sql/api/r/SQLUtils.scala index e56c33e4b5..a4c5bf756c 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/api/r/SQLUtils.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/api/r/SQLUtils.scala @@ -47,12 +47,14 @@ private[sql] object SQLUtils extends Logging { jsc: JavaSparkContext, sparkConfigMap: JMap[Object, Object], enableHiveSupport: Boolean): SparkSession = { - val spark = if (SparkSession.hiveClassesArePresent && enableHiveSupport) { + val spark = if (SparkSession.hiveClassesArePresent && enableHiveSupport + && jsc.sc.conf.get(CATALOG_IMPLEMENTATION.key, "hive").toLowerCase == "hive") { SparkSession.builder().sparkContext(withHiveExternalCatalog(jsc.sc)).getOrCreate() } else { if (enableHiveSupport) { logWarning("SparkR: enableHiveSupport is requested for SparkSession but " + - "Spark is not built with Hive; falling back to without Hive support.") + s"Spark is not built with Hive or ${CATALOG_IMPLEMENTATION.key} is not set to 'hive', " + + "falling back to without Hive support.") } SparkSession.builder().sparkContext(jsc.sc).getOrCreate() } |