-
Notifications
You must be signed in to change notification settings - Fork 29k
[SPARK-24250][SQL] support accessing SQLConf inside tasks #21299
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
e39b7d0
7c7caf8
dc63dbe
2ecabe4
bf8b42d
a100dea
01e288a
cf75856
cfb76f5
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 |
|---|---|---|
| @@ -0,0 +1,66 @@ | ||
| /* | ||
| * Licensed to the Apache Software Foundation (ASF) under one or more | ||
| * contributor license agreements. See the NOTICE file distributed with | ||
| * this work for additional information regarding copyright ownership. | ||
| * The ASF licenses this file to You under the Apache License, Version 2.0 | ||
| * (the "License"); you may not use this file except in compliance with | ||
| * the License. You may obtain a copy of the License at | ||
| * | ||
| * http://www.apache.org/licenses/LICENSE-2.0 | ||
| * | ||
| * Unless required by applicable law or agreed to in writing, software | ||
| * distributed under the License is distributed on an "AS IS" BASIS, | ||
| * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. | ||
| * See the License for the specific language governing permissions and | ||
| * limitations under the License. | ||
| */ | ||
|
|
||
| package org.apache.spark.sql.internal | ||
|
|
||
| import java.util.{Map => JMap} | ||
|
|
||
| import org.apache.spark.{TaskContext, TaskContextImpl} | ||
| import org.apache.spark.internal.config.{ConfigEntry, ConfigProvider, ConfigReader} | ||
|
|
||
| /** | ||
| * A readonly SQLConf that will be created by tasks running at the executor side. It reads the | ||
| * configs from the local properties which are propagated from driver to executors. | ||
| */ | ||
| class ReadOnlySQLConf(context: TaskContext) extends SQLConf { | ||
|
|
||
| @transient override val settings: JMap[String, String] = { | ||
| context.asInstanceOf[TaskContextImpl].getLocalProperties().asInstanceOf[JMap[String, String]] | ||
| } | ||
|
|
||
| @transient override protected val reader: ConfigReader = { | ||
| new ConfigReader(new TaskContextConfigProvider(context)) | ||
| } | ||
|
|
||
| override protected def setConfWithCheck(key: String, value: String): Unit = { | ||
| throw new UnsupportedOperationException("Cannot mutate ReadOnlySQLConf.") | ||
| } | ||
|
|
||
| override def unsetConf(key: String): Unit = { | ||
| throw new UnsupportedOperationException("Cannot mutate ReadOnlySQLConf.") | ||
| } | ||
|
|
||
| override def unsetConf(entry: ConfigEntry[_]): Unit = { | ||
| throw new UnsupportedOperationException("Cannot mutate ReadOnlySQLConf.") | ||
| } | ||
|
|
||
| override def clear(): Unit = { | ||
| throw new UnsupportedOperationException("Cannot mutate ReadOnlySQLConf.") | ||
| } | ||
|
|
||
| override def clone(): SQLConf = { | ||
| throw new UnsupportedOperationException("Cannot clone/copy ReadOnlySQLConf.") | ||
| } | ||
|
|
||
| override def copy(entries: (ConfigEntry[_], Any)*): SQLConf = { | ||
| throw new UnsupportedOperationException("Cannot clone/copy ReadOnlySQLConf.") | ||
| } | ||
| } | ||
|
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. Do we need to allow us to do
Contributor
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 don't think we need to clone or copy |
||
|
|
||
| class TaskContextConfigProvider(context: TaskContext) extends ConfigProvider { | ||
| override def get(key: String): Option[String] = Option(context.getLocalProperty(key)) | ||
| } | ||
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -68,16 +68,18 @@ object SQLExecution { | |
| // sparkContext.getCallSite() would first try to pick up any call site that was previously | ||
| // set, then fall back to Utils.getCallSite(); call Utils.getCallSite() directly on | ||
| // streaming queries would give us call site like "run at <unknown>:0" | ||
| val callSite = sparkSession.sparkContext.getCallSite() | ||
| val callSite = sc.getCallSite() | ||
|
|
||
| sparkSession.sparkContext.listenerBus.post(SparkListenerSQLExecutionStart( | ||
| executionId, callSite.shortForm, callSite.longForm, queryExecution.toString, | ||
| SparkPlanInfo.fromSparkPlan(queryExecution.executedPlan), System.currentTimeMillis())) | ||
| try { | ||
| body | ||
| } finally { | ||
| sparkSession.sparkContext.listenerBus.post(SparkListenerSQLExecutionEnd( | ||
| executionId, System.currentTimeMillis())) | ||
| withSQLConfPropagated(sparkSession) { | ||
| sc.listenerBus.post(SparkListenerSQLExecutionStart( | ||
| executionId, callSite.shortForm, callSite.longForm, queryExecution.toString, | ||
| SparkPlanInfo.fromSparkPlan(queryExecution.executedPlan), System.currentTimeMillis())) | ||
| try { | ||
| body | ||
| } finally { | ||
| sc.listenerBus.post(SparkListenerSQLExecutionEnd( | ||
| executionId, System.currentTimeMillis())) | ||
| } | ||
| } | ||
| } finally { | ||
| executionIdToQueryExecution.remove(executionId) | ||
|
|
@@ -90,13 +92,42 @@ object SQLExecution { | |
| * thread from the original one, this method can be used to connect the Spark jobs in this action | ||
| * with the known executionId, e.g., `BroadcastExchangeExec.relationFuture`. | ||
| */ | ||
| def withExecutionId[T](sc: SparkContext, executionId: String)(body: => T): T = { | ||
| def withExecutionId[T](sparkSession: SparkSession, executionId: String)(body: => T): T = { | ||
| val sc = sparkSession.sparkContext | ||
| val oldExecutionId = sc.getLocalProperty(SQLExecution.EXECUTION_ID_KEY) | ||
| withSQLConfPropagated(sparkSession) { | ||
| try { | ||
| sc.setLocalProperty(SQLExecution.EXECUTION_ID_KEY, executionId) | ||
| body | ||
| } finally { | ||
| sc.setLocalProperty(SQLExecution.EXECUTION_ID_KEY, oldExecutionId) | ||
| } | ||
| } | ||
| } | ||
|
|
||
| def withSQLConfPropagated[T](sparkSession: SparkSession)(body: => T): T = { | ||
|
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. Maybe cleaner in following way: |
||
| // Set all the specified SQL configs to local properties, so that they can be available at | ||
| // the executor side. | ||
|
||
| val allConfigs = sparkSession.sessionState.conf.getAllConfs | ||
| val originalLocalProps = scala.collection.mutable.HashMap.empty[String, String] | ||
| for ((key, value) <- allConfigs) { | ||
| // Excludes external configs defined by users. | ||
| if (key.startsWith("spark")) { | ||
| Option(sparkSession.sparkContext.getLocalProperty(key)).foreach { | ||
| // If users already set a value in local properties, keep it and restore it at the end. | ||
| origin => originalLocalProps(key) = origin | ||
| } | ||
| sparkSession.sparkContext.setLocalProperty(key, value) | ||
| } | ||
| } | ||
| try { | ||
| sc.setLocalProperty(SQLExecution.EXECUTION_ID_KEY, executionId) | ||
| body | ||
| } finally { | ||
| sc.setLocalProperty(SQLExecution.EXECUTION_ID_KEY, oldExecutionId) | ||
| allConfigs.foreach { | ||
| case (key, _) => | ||
| val origin = originalLocalProps.getOrElse(key, null) | ||
| sparkSession.sparkContext.setLocalProperty(key, origin) | ||
| } | ||
| } | ||
| } | ||
| } | ||
| Original file line number | Diff line number | Diff line change |
|---|---|---|
| @@ -0,0 +1,66 @@ | ||
| /* | ||
| * Licensed to the Apache Software Foundation (ASF) under one or more | ||
| * contributor license agreements. See the NOTICE file distributed with | ||
| * this work for additional information regarding copyright ownership. | ||
| * The ASF licenses this file to You under the Apache License, Version 2.0 | ||
| * (the "License"); you may not use this file except in compliance with | ||
| * the License. You may obtain a copy of the License at | ||
| * | ||
| * http://www.apache.org/licenses/LICENSE-2.0 | ||
| * | ||
| * Unless required by applicable law or agreed to in writing, software | ||
| * distributed under the License is distributed on an "AS IS" BASIS, | ||
| * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. | ||
| * See the License for the specific language governing permissions and | ||
| * limitations under the License. | ||
| */ | ||
|
|
||
| package org.apache.spark.sql.internal | ||
|
|
||
| import org.apache.spark.SparkFunSuite | ||
| import org.apache.spark.sql.SparkSession | ||
| import org.apache.spark.sql.test.SQLTestUtils | ||
|
|
||
| class ExecutorSideSQLConfSuite extends SparkFunSuite with SQLTestUtils { | ||
| import testImplicits._ | ||
|
|
||
| protected var spark: SparkSession = null | ||
|
|
||
| // Create a new [[SparkSession]] running in local-cluster mode. | ||
| override def beforeAll(): Unit = { | ||
| super.beforeAll() | ||
| spark = SparkSession.builder() | ||
| .master("local-cluster[2,1,1024]") | ||
| .appName("testing") | ||
| .getOrCreate() | ||
| } | ||
|
|
||
| override def afterAll(): Unit = { | ||
| spark.stop() | ||
| spark = null | ||
| } | ||
|
|
||
| test("ReadonlySQLConf is correctly created at the executor side") { | ||
| SQLConf.get.setConfString("spark.sql.x", "a") | ||
| try { | ||
| val checks = spark.range(10).mapPartitions { it => | ||
| val conf = SQLConf.get | ||
| Iterator(conf.isInstanceOf[ReadOnlySQLConf] && conf.getConfString("spark.sql.x") == "a") | ||
| }.collect() | ||
| assert(checks.forall(_ == true)) | ||
| } finally { | ||
| SQLConf.get.unsetConf("spark.sql.x") | ||
| } | ||
| } | ||
|
|
||
| test("case-sensitive config should work for json schema inference") { | ||
| withSQLConf(SQLConf.CASE_SENSITIVE.key -> "true") { | ||
| withTempPath { path => | ||
| val pathString = path.getCanonicalPath | ||
| spark.range(10).select('id.as("ID")).write.json(pathString) | ||
| spark.range(10).write.mode("append").json(pathString) | ||
| assert(spark.read.json(pathString).columns.toSet == Set("id", "ID")) | ||
| } | ||
| } | ||
| } | ||
| } |
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.
This is nice :)