Skip to content
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

[SPARK-21637][SPARK-21451][SQL]get spark.hadoop.* properties from sysProps to hiveconf #18668

Closed
wants to merge 13 commits into from
Closed

Conversation

yaooqinn
Copy link
Member

@yaooqinn yaooqinn commented Jul 18, 2017

What changes were proposed in this pull request?

When we use bin/spark-sql command configuring --conf spark.hadoop.foo=bar, the SparkSQLCliDriver initializes an instance of hiveconf, it does not add foo->bar to it.
this pr gets spark.hadoop.* properties from sysProps to this hiveconf

How was this patch tested?

UT

@yaooqinn
Copy link
Member Author

ping @cloud-fan @gatorsmile

@@ -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"
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

why do we should do so?

Copy link
Member Author

@yaooqinn yaooqinn Jul 19, 2017

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

@cloud-fan if we run bin/spark-sql --conf spark.hadoop.hive.exec.strachdir=/some/dir or in spark-default.conf, SessionState.start(cliSessionState) in SparkSQLCliDriver will not use this dir but the default

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

do we have documents saying that spark.hadoop.xxx is supported? or are you proposing a new feature?

Copy link
Member Author

@yaooqinn yaooqinn Jul 19, 2017

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

lets move this to a util method so that we know this is done in 2 places

@cloud-fan
Copy link
Contributor

ok to test

@SparkQA
Copy link

SparkQA commented Jul 20, 2017

Test build #79792 has finished for PR 18668 at commit 89d9b86.

  • This patch passes all tests.
  • This patch merges cleanly.
  • This patch adds no public classes.

@yaooqinn
Copy link
Member Author

@cloud-fan tests passed, mind take a look?

@yaooqinn
Copy link
Member Author

ping @cloud-fan

@yaooqinn
Copy link
Member Author

ping @cloud-fan again

@@ -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")
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

The test says we should respect hadoop conf in SparkConf, but why we handle system properties?

Copy link
Member Author

Choose a reason for hiding this comment

The 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.

sys.props.put("spark.hadoop.foo", "bar")
Seq(true, false) foreach { useInMemoryDerby =>
val hiveConf = HiveUtils.newTemporaryConfiguration(useInMemoryDerby)
intercept[NoSuchElementException](hiveConf("spark.hadoop.foo") === "bar")
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

nit: assert(!hiveConf.contains("spark.hadoop.foo"))

@cloud-fan
Copy link
Contributor

LGTM except one minor comment

@gatorsmile
Copy link
Member

If this is an issue in SparkSQLCliDriver , could you add a test case to CliSuite?

@SparkQA
Copy link

SparkQA commented Jul 31, 2017

Test build #80063 has finished for PR 18668 at commit 1ac4cb9.

  • This patch passes all tests.
  • This patch merges cleanly.
  • This patch adds no public classes.

@yaooqinn
Copy link
Member Author

yaooqinn commented Jul 31, 2017

@cloud-fan would you plz take a look

Anyway in CliSuite hive.metastore.warehouse at line https://github.com/apache/spark/blob/master/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/CliSuite.scala#L92 since this commit:8f33731 . Such as bin/spark-sql --hiveconf hive.metastore.warehouse.dir=/some/dir will not take affect, because now we respect this property in hadoopconf only but in SparkSQLCliDriver, but we do not add --hiveconf s to hadoopconf. I've change this too.

@yaooqinn
Copy link
Member Author

Hi @gatorsmile, I add some UTs in CliSuite, please check!

@SparkQA
Copy link

SparkQA commented Jul 31, 2017

Test build #80077 has finished for PR 18668 at commit 9388f5e.

  • This patch fails Scala style tests.
  • This patch merges cleanly.
  • This patch adds no public classes.

@gatorsmile
Copy link
Member

test this please

runCliWithin(1.minute)("set spark.sql.warehouse.dir;" -> warehousePath.getAbsolutePath)
}

test("SPARK-21451: Apply spark.hadoop.* configurations") {
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Without the fix, this test case still can succeed.

Copy link
Member Author

@yaooqinn yaooqinn Aug 2, 2017

Choose a reason for hiding this comment

The 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.

@@ -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."
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Just a question, why the prefix has to be spark.hadoop.?

See the related PR: #2379

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

good point. I see spark.hive in some of my configs

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

spark.hadoop. was tribal knowledge and was a sneaky way to stick values into Hadoop Configuration object (which can later also pass on to HiveConf). What does spark.hive. do ? Have never seen such configs and would like to know.

Keeping that aside, are you proposing to drop that prefix at L145 ?

Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

After thinking more, I think we should just consider spark.hadoop. in this PR, unless we get the other feedbacks from the community.

@gatorsmile
Copy link
Member

gatorsmile commented Aug 1, 2017

Since the most related PR #1843 was submitted by @vanzin Could you please review this PR?

Actually, the prefix spark.hadoop. is not documented https://spark.apache.org/docs/latest/configuration.html

Could you explain more details about this?

@SparkQA
Copy link

SparkQA commented Aug 1, 2017

Test build #80110 has finished for PR 18668 at commit 9ba8f53.

  • This patch passes all tests.
  • This patch merges cleanly.
  • This patch adds no public classes.

@vanzin
Copy link
Contributor

vanzin commented Aug 2, 2017

spark.hadoop. has existed for ages, I'm kinda surprised it's not properly documented. My change didn't add it, it just centralized its use.

As for this particular PR, I'm not so sure about it. I'm not exactly familiar with how the "execution" Hive instance uses the metastore and other Hadoop configs, but at the very least this creates a conflict between the configuration of the execution Hive and that of the metadata Hive (which does use spark.hadoop.*, as far as I remember).

So with this change you cannot use spark.hadoop. to set a config option that only applies to the metadata Hive, for example, and I'm not so sure that's desired. And it may negatively affect people who do set those options for the metadata instance today.

@yaooqinn
Copy link
Member Author

yaooqinn commented Aug 2, 2017

There is a bug in HiveClientImpl about reusing cliSessionState, see HiveClientImpl.scala#L140

// In SparkSQLCLIDriver, we have already started a CliSessionState,
// which contains information like configurations from command line. Later
// we call SparkSQLEnv.init() there, which would run into this part again.
// so we should keep conf and reuse the existing instance of CliSessionState

Actually, it is never been reached and reused. session.SessionState will be re-generated every time when you call HiveClient.newSession()

you can run bin/spark-sql --master local simply with info log on, you can see HiveClientImpl.scala#L193 called four times creating session related directories.

  1. HiveExternalCatalog.scala#L65
  2. HiveSessionStateBuilder.scala#L45
  3. SparkSQLEnv.scala#L54 which is unnecessary I guess
  4. SparkSQLCLIDriver.scala#L115 - which should be reused and it has rights to get all hadoop congfigurations

@yaooqinn
Copy link
Member Author

yaooqinn commented Aug 2, 2017

@vanzin

the configuration of the execution Hive

Does this mean a hive client initialized by HiveUtils.newClientForExecution? If true, this is ONLY used in HiveThiftSever2 after SparkContext initialized.

Example:
sbin/start-thriftserver.sh --conf spark.hadoop.hive.server2.thrift.port=11001 --hiveconf hive.server2.thrift.port=11000
Spark Thrift Server will take 11001 as the port. hive.server2.thrift.port firstly be parsed 11000, but it will be re-writted to 11001 HiveThriftServer2.scala#L90 after SparkSQLEnv init the sc. IMO the spark.hadoop.xxx properties can be treat as special spark properties, which should have higher priority than its original form xxx. In SparkSQLCliDriver we shall obey this rule too.

@SparkQA
Copy link

SparkQA commented Aug 2, 2017

Test build #80158 has finished for PR 18668 at commit a0329ca.

  • This patch passes all tests.
  • This patch merges cleanly.
  • This patch adds no public classes.

@vanzin
Copy link
Contributor

vanzin commented Aug 2, 2017

If true, this is ONLY used in HiveThiftSever2 after SparkContext initialized.

I see. Seems like this code changed a bit since I last looked at it closely, when there was an explicit "execution Hive" in HiveContext.

If that's the case then it's probably ok to add this.

@SparkQA
Copy link

SparkQA commented Aug 4, 2017

Test build #80227 has finished for PR 18668 at commit 5043eb6.

  • This patch passes all tests.
  • This patch merges cleanly.
  • This patch adds no public classes.

@@ -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"
Copy link
Member

@gatorsmile gatorsmile Aug 4, 2017

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

@yaooqinn Please follow what @tejasapatil said and create a util function.

Copy link
Member Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

ok

while (it.hasNext) {
val kv = it.next()
SparkSQLEnv.sqlContext.setConf(kv.getKey, kv.getValue)
newHiveConf.foreach{ kv =>
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

foreach{ -> foreach {

Copy link
Member Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

thanks

// 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)
Copy link
Member

Choose a reason for hiding this comment

The 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 newTemporaryConfiguration is before we setting sys.props. The subsequent call of newTemporaryConfiguration in newClientForExecution will be used for Hive execution clients. Thus, the changes will affect Hive execution clients.

Could you check all the codes in Spark are using sys.prop? Will this change impact them?

Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

When we build SparkConf in SparkSQLEnv, we get the conf from system prop because loadDefaults is set to true. That is the way we pass -hiveconf values to sc.hadoopConfiguration.

Copy link
Member Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

newClientForExecution is used ONLY in HiveThriftServer2, where it is used to get a hiveconf. There is no more a execution hive client, IMO this method be removed. This activity happens after SparkSQLEnv.init, so it is OK for spark.hadoop. properties.

I realize that --hiveconf should be added to sys.props as spark.hadoop.xxx before SparkSQLEnv.init

Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

newClientForExecution is used for us to read/write hive serde tables. This is the major concern I have. Let us add another parameter in newTemporaryConfiguration . When newClientForExecution is calling newTemporaryConfiguration , we should not get the hive conf from sys.prop.

Copy link
Member Author

@yaooqinn yaooqinn Aug 4, 2017

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I have checked the whole project that newClientForExecution is only used at HiveThriftServer2.scala#L58, HiveThriftServer2.scala#L86

Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Then, that sounds ok to me.

@@ -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()
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

What is the reason you move it to line 140?

Copy link
Member Author

Choose a reason for hiding this comment

The 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 SparkSQLEnv.init

@gatorsmile
Copy link
Member

Please open another JIRA for the issue of #18668 (comment). Also put the JIRA number in this PR. It can help us track the issues. Thanks!

@@ -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) =>
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

As I mentioned above, we should not do this for newClientForExecution.

@SparkQA
Copy link

SparkQA commented Aug 4, 2017

Test build #80239 has finished for PR 18668 at commit ee47742.

  • This patch fails Scala style tests.
  • This patch merges cleanly.
  • This patch adds no public classes.

@yaooqinn yaooqinn changed the title [SPARK-21451][SQL]get spark.hadoop.* properties from sysProps to hiveconf [SPARK-21637][SPARK-21451][SQL]get spark.hadoop.* properties from sysProps to hiveconf Aug 4, 2017
@SparkQA
Copy link

SparkQA commented Aug 4, 2017

Test build #80241 has finished for PR 18668 at commit a2b23f3.

  • This patch fails PySpark unit tests.
  • This patch merges cleanly.
  • This patch adds no public classes.

@jiangxb1987
Copy link
Contributor

retest this please

@@ -404,6 +405,8 @@ private[spark] object HiveUtils extends Logging {
propMap.put(ConfVars.METASTORE_EVENT_LISTENERS.varname, "")
propMap.put(ConfVars.METASTORE_END_FUNCTION_LISTENERS.varname, "")

SparkHadoopUtil.get.appendSparkHadoopConfigs(propMap)
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

We are unable to know this is getting the values from sys.props. How about changing the interface to?

// xyz
SparkHadoopUtil.get.appendSparkHadoopConfigs(sys.props.toMap, propMap)

Copy link
Member Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

ok

sys.props.foreach { case (key, value) =>
if (key.startsWith("spark.hadoop.")) {
propMap.put(key.substring("spark.hadoop.".length), value)
}
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

We can shorten it to something like

    for ((key, value) <- conf if key.startsWith("spark.hadoop.")) {
      propMap.put(key.substring("spark.hadoop.".length), value)
    }

Copy link
Member Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

ok

@SparkQA
Copy link

SparkQA commented Aug 5, 2017

Test build #80278 has finished for PR 18668 at commit 55729fa.

  • This patch fails Spark unit tests.
  • This patch merges cleanly.
  • This patch adds no public classes.

// Copy any "spark.hadoop.foo=bar" system properties into destMap as "foo=bar"
srcMap.foreach { case (key, value) if key.startsWith("spark.hadoop.") =>
destMap.put(key.substring("spark.hadoop.".length), value)
}
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Your change is different from what I posted before

    for ((key, value) <- conf if key.startsWith("spark.hadoop.")) {
      propMap.put(key.substring("spark.hadoop.".length), value)
    }

Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Your solution requires another case _ =>

Copy link
Member Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

ok

@SparkQA
Copy link

SparkQA commented Aug 5, 2017

Test build #80284 has finished for PR 18668 at commit 46a955d.

  • This patch fails SparkR unit tests.
  • This patch merges cleanly.
  • This patch adds no public classes.

@yaooqinn
Copy link
Member Author

yaooqinn commented Aug 5, 2017

retest this please

1 similar comment
@gatorsmile
Copy link
Member

retest this please

@gatorsmile
Copy link
Member

LGTM pending Jenkins.

@SparkQA
Copy link

SparkQA commented Aug 5, 2017

Test build #80288 has finished for PR 18668 at commit 46a955d.

  • This patch passes all tests.
  • This patch merges cleanly.
  • This patch adds no public classes.

@gatorsmile
Copy link
Member

Thanks everyone! Merging it to master.

If any other comment, we can address it in the follow-up PRs.

@asfgit asfgit closed this in 41568e9 Aug 6, 2017
Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment
Labels
None yet
Projects
None yet
Development

Successfully merging this pull request may close these issues.

8 participants