-
Notifications
You must be signed in to change notification settings - Fork 29k
[SPARK-21637][SPARK-21451][SQL]get spark.hadoop.* properties from sysProps to hiveconf
#18668
New issue
Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.
By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.
Already on GitHub? Sign in to your account
Changes from 8 commits
89d9b86
1ac4cb9
9388f5e
9ba8f53
a0329ca
6dba90a
10d624c
c629cc4
5043eb6
ee47742
a2b23f3
55729fa
46a955d
File filter
Filter by extension
Conversations
Jump to
Diff view
Diff view
There are no files selected for viewing
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -2326,7 +2326,7 @@ from this directory. | |
| # Inheriting Hadoop Cluster Configuration | ||
|
|
||
| If you plan to read and write from HDFS using Spark, there are two Hadoop configuration files that | ||
| should be included on Spark's classpath: | ||
| should be included on Spark's class path: | ||
|
|
||
| * `hdfs-site.xml`, which provides default behaviors for the HDFS client. | ||
| * `core-site.xml`, which sets the default filesystem name. | ||
|
|
@@ -2335,5 +2335,61 @@ The location of these configuration files varies across Hadoop versions, but | |
| a common location is inside of `/etc/hadoop/conf`. Some tools create | ||
| configurations on-the-fly, but offer a mechanisms to download copies of them. | ||
|
|
||
| To make these files visible to Spark, set `HADOOP_CONF_DIR` in `$SPARK_HOME/spark-env.sh` | ||
| To make these files visible to Spark, set `HADOOP_CONF_DIR` in `$SPARK_HOME/conf/spark-env.sh` | ||
|
Member
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. @zsxwing @liancheng Could you please take a look at the documentation? Anything is missing or inaccurate? |
||
| to a location containing the configuration files. | ||
|
|
||
| # Custom Hadoop/Hive Configuration | ||
|
|
||
| If your Spark applications interacting with Hadoop, Hive, or both, there are probably Hadoop/Hive | ||
|
||
| configuration files in Spark's class path. | ||
|
|
||
| Multiple running applications might require different Hadoop/Hive client side configurations. | ||
| You can copy and modify `hdfs-site.xml`, `core-site.xml`, `yarn-site.xml`, `hive-site.xml` in | ||
| Spark's class path for each application, but it is not very convenient and these | ||
| files are best to be shared with common properties to avoid hard-coding certain configurations. | ||
|
||
|
|
||
| The better choice is to use spark hadoop properties in the form of `spark.hadoop.*`. | ||
| They can be considered as same as normal spark properties which can be set in `$SPARK_HOME/conf/spark-defalut.conf` | ||
|
|
||
| In some cases, you may want to avoid hard-coding certain configurations in a `SparkConf`. For | ||
| instance, Spark allows you to simply create an empty conf and set spark/spark hadoop properties. | ||
|
|
||
| {% highlight scala %} | ||
| val conf = new SparkConf().set("spark.hadoop.abc.def","xyz") | ||
| val sc = new SparkContext(conf) | ||
| {% endhighlight %} | ||
|
|
||
| Also, you can modify or add configurations at runtime: | ||
| {% highlight bash %} | ||
| ./bin/spark-submit \ | ||
| --name "My app" \ | ||
| --master local[4] \ | ||
| --conf spark.eventLog.enabled=false \ | ||
| --conf "spark.executor.extraJavaOptions=-XX:+PrintGCDetails -XX:+PrintGCTimeStamps" \ | ||
| --conf spark.hadoop.abc.def=xyz \ | ||
| myApp.jar | ||
| {% endhighlight %} | ||
|
|
||
| ## Typical Hadoop/Hive Configurations | ||
|
||
|
|
||
| <table> | ||
| <tr> | ||
| <td><code>spark.hadoop.<br />mapreduce.fileoutputcommitter.algorithm.version</code></td> | ||
| <td>1</td> | ||
| <td> | ||
| The file output committer algorithm version, valid algorithm version number: 1 or 2. | ||
| Version 2 may have better performance, but version 1 may handle failures better in certain situations, | ||
| as per <a href="https://issues.apache.org/jira/browse/MAPREDUCE-4815">MAPREDUCE-4815</a>. | ||
| </td> | ||
| </tr> | ||
|
|
||
| <tr> | ||
| <td><code>spark.hadoop.<br />fs.hdfs.impl.disable.cache</code></td> | ||
|
||
| <td>false</td> | ||
| <td> | ||
| When true, return a fresh HDFS filesystem instance, bypassing the HDFS cache mechanism. | ||
| This is to prevent the DFSClient from using an old cached token to connect to the NameNode, | ||
| which might fails long-running Spark applications. see <a href="https://issues.apache.org/jira/browse/HDFS-9276">HDFS-9276</a>. | ||
| </td> | ||
| </tr> | ||
| </table> | ||
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -50,6 +50,7 @@ private[hive] object SparkSQLCLIDriver extends Logging { | |
| private val prompt = "spark-sql" | ||
| private val continuedPrompt = "".padTo(prompt.length, ' ') | ||
| private var transport: TSocket = _ | ||
| private final val SPARK_HADOOP_PROP_PREFIX = "spark.hadoop." | ||
|
Member
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Just a question, why the prefix has to be See the related PR: #2379
Contributor
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. good point. I see
Contributor
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more.
Keeping that aside, are you proposing to drop that prefix at L145 ?
Member
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. After thinking more, I think we should just consider |
||
|
|
||
| installSignalHandler() | ||
|
|
||
|
|
@@ -134,6 +135,16 @@ private[hive] object SparkSQLCLIDriver extends Logging { | |
| // Hive 1.2 + not supported in CLI | ||
| throw new RuntimeException("Remote operations not supported") | ||
| } | ||
| // Respect the configurations set by --hiveconf from the command line | ||
| // (based on Hive's CliDriver). | ||
| val hiveConfFromCmd = sessionState.getOverriddenConfigurations.entrySet().asScala | ||
| val newHiveConf = hiveConfFromCmd.map { kv => | ||
| // If the same property is configured by spark.hadoop.xxx, we ignore it and | ||
| // obey settings from spark properties | ||
| val k = kv.getKey | ||
| val v = sys.props.getOrElseUpdate(SPARK_HADOOP_PROP_PREFIX + k, kv.getValue) | ||
|
Member
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Let me try to summarize the impacts of these changes. The initial call of Could you check all the codes in Spark are using
Member
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. When we build
Member
Author
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more.
I realize that
Member
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more.
Member
Author
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. I have checked the whole project that
Member
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Then, that sounds ok to me. |
||
| (k, v) | ||
| } | ||
|
|
||
| val cli = new SparkSQLCLIDriver | ||
| cli.setHiveVariables(oproc.getHiveVariables) | ||
|
|
@@ -157,12 +168,8 @@ private[hive] object SparkSQLCLIDriver extends Logging { | |
| // Execute -i init files (always in silent mode) | ||
| cli.processInitFiles(sessionState) | ||
|
|
||
| // Respect the configurations set by --hiveconf from the command line | ||
| // (based on Hive's CliDriver). | ||
| val it = sessionState.getOverriddenConfigurations.entrySet().iterator() | ||
|
Member
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. What is the reason you move it to line 140?
Member
Author
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. --hiveconf abc.def will be add to system properties as spark.hadoop.abc.def if is not existed , before |
||
| while (it.hasNext) { | ||
| val kv = it.next() | ||
| SparkSQLEnv.sqlContext.setConf(kv.getKey, kv.getValue) | ||
| newHiveConf.foreach{ kv => | ||
|
||
| SparkSQLEnv.sqlContext.setConf(kv._1, kv._2) | ||
| } | ||
|
|
||
| if (sessionState.execString != null) { | ||
|
|
||
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -283,4 +283,17 @@ class CliSuite extends SparkFunSuite with BeforeAndAfterAll with Logging { | |
| "SET conf3;" -> "conftest" | ||
| ) | ||
| } | ||
|
|
||
| test("SPARK-21451: spark.sql.warehouse.dir should respect options in --hiveconf") { | ||
| runCliWithin(1.minute)("set spark.sql.warehouse.dir;" -> warehousePath.getAbsolutePath) | ||
| } | ||
|
|
||
| test("SPARK-21451: Apply spark.hadoop.* configurations") { | ||
|
Member
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Without the fix, this test case still can succeed.
Member
Author
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. @gatorsmile Yes, after sc initialized, spark.hadoop.hive.metastore.warehouse.dir will be translated into a hadoop conf hive.metastore.warehouse.dir as an alternative of warehouse dir. This test case couldn't tell whether this pr works. CliSuite may not see these values only if we explicitly set them to SqlConf. The original code did break another test case anyway. |
||
| val tmpDir = Utils.createTempDir(namePrefix = "SPARK-21451") | ||
| runCliWithin( | ||
| 1.minute, | ||
| Seq(s"--conf", s"spark.hadoop.${ConfVars.METASTOREWAREHOUSE}=$tmpDir"))( | ||
| "set spark.sql.warehouse.dir;" -> tmpDir.getAbsolutePath) | ||
| tmpDir.delete() | ||
| } | ||
| } | ||
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -404,6 +404,13 @@ private[spark] object HiveUtils extends Logging { | |
| propMap.put(ConfVars.METASTORE_EVENT_LISTENERS.varname, "") | ||
| propMap.put(ConfVars.METASTORE_END_FUNCTION_LISTENERS.varname, "") | ||
|
|
||
| // Copy any "spark.hadoop.foo=bar" system properties into conf as "foo=bar" | ||
|
||
| sys.props.foreach { case (key, value) => | ||
|
||
| if (key.startsWith("spark.hadoop.")) { | ||
| propMap.put(key.substring("spark.hadoop.".length), value) | ||
| } | ||
| } | ||
|
|
||
| propMap.toMap | ||
| } | ||
|
|
||
|
|
||
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -33,4 +33,13 @@ class HiveUtilsSuite extends QueryTest with SQLTestUtils with TestHiveSingleton | |
| assert(conf(ConfVars.METASTORE_END_FUNCTION_LISTENERS.varname) === "") | ||
| } | ||
| } | ||
|
|
||
| test("newTemporaryConfiguration respect spark.hadoop.foo=bar in SparkConf") { | ||
| sys.props.put("spark.hadoop.foo", "bar") | ||
|
Contributor
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. The test says we should respect hadoop conf in
Member
Author
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. @cloud-fan at the very beginning, the spark-sumit do the same thing that add properties from --conf and spark-default.conf to sys.props. |
||
| Seq(true, false) foreach { useInMemoryDerby => | ||
| val hiveConf = HiveUtils.newTemporaryConfiguration(useInMemoryDerby) | ||
| assert(!hiveConf.contains("spark.hadoop.foo")) | ||
| assert(hiveConf("foo") === "bar") | ||
| } | ||
| } | ||
| } | ||
Uh oh!
There was an error while loading. Please reload this page.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
nit: everywhere in the documentation
classpathis being used so changing just one instance will make the doc inconsistent. Lets keep this as it was.