-
Notifications
You must be signed in to change notification settings - Fork 28.9k
[SPARK-24250][SQL] support accessing SQLConf inside tasks #21376
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 all commits
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.") | ||
| } | ||
| } | ||
|
|
||
| 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 |
|---|---|---|
|
|
@@ -27,7 +27,7 @@ import scala.util.matching.Regex | |
|
|
||
| import org.apache.hadoop.fs.Path | ||
|
|
||
| import org.apache.spark.{SparkContext, SparkEnv} | ||
| import org.apache.spark.{SparkContext, TaskContext} | ||
| import org.apache.spark.internal.Logging | ||
| import org.apache.spark.internal.config._ | ||
| import org.apache.spark.network.util.ByteUnit | ||
|
|
@@ -95,7 +95,9 @@ object SQLConf { | |
|
|
||
| /** | ||
| * Returns the active config object within the current scope. If there is an active SparkSession, | ||
| * the proper SQLConf associated with the thread's session is used. | ||
| * the proper SQLConf associated with the thread's active session is used. If it's called from | ||
| * tasks in the executor side, a SQLConf will be created from job local properties, which are set | ||
| * and propagated from the driver side. | ||
| * | ||
| * The way this works is a little bit convoluted, due to the fact that config was added initially | ||
| * only for physical plans (and as a result not in sql/catalyst module). | ||
|
|
@@ -107,7 +109,22 @@ object SQLConf { | |
| * run tests in parallel. At the time this feature was implemented, this was a no-op since we | ||
| * run unit tests (that does not involve SparkSession) in serial order. | ||
| */ | ||
| def get: SQLConf = confGetter.get()() | ||
| def get: SQLConf = { | ||
| if (TaskContext.get != null) { | ||
| new ReadOnlySQLConf(TaskContext.get()) | ||
| } else { | ||
| if (Utils.isTesting && SparkContext.getActive.isDefined) { | ||
|
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. good check!! |
||
| // DAGScheduler event loop thread does not have an active SparkSession, the `confGetter` | ||
| // will return `fallbackConf` which is unexpected. Here we prevent it from happening. | ||
| val schedulerEventLoopThread = | ||
| SparkContext.getActive.get.dagScheduler.eventProcessLoop.eventThread | ||
| if (schedulerEventLoopThread.getId == Thread.currentThread().getId) { | ||
| throw new RuntimeException("Cannot get SQLConf inside scheduler event loop thread.") | ||
| } | ||
|
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. nit: Should we also add a note in PR description that we prevent accessing SQLConf in scheduler event loop? |
||
| } | ||
| confGetter.get()() | ||
| } | ||
| } | ||
|
|
||
| val OPTIMIZER_MAX_ITERATIONS = buildConf("spark.sql.optimizer.maxIterations") | ||
| .internal() | ||
|
|
@@ -1292,17 +1309,11 @@ object SQLConf { | |
| class SQLConf extends Serializable with Logging { | ||
| import SQLConf._ | ||
|
|
||
| if (Utils.isTesting && SparkEnv.get != null) { | ||
| // assert that we're only accessing it on the driver. | ||
| assert(SparkEnv.get.executorId == SparkContext.DRIVER_IDENTIFIER, | ||
| "SQLConf should only be created and accessed on the driver.") | ||
| } | ||
|
|
||
| /** Only low degree of contention is expected for conf, thus NOT using ConcurrentHashMap. */ | ||
| @transient protected[spark] val settings = java.util.Collections.synchronizedMap( | ||
| new java.util.HashMap[String, String]()) | ||
|
|
||
| @transient private val reader = new ConfigReader(settings) | ||
| @transient protected val reader = new ConfigReader(settings) | ||
|
|
||
| /** ************************ Spark SQL Params/Hints ******************* */ | ||
|
|
||
|
|
@@ -1765,7 +1776,7 @@ class SQLConf extends Serializable with Logging { | |
| settings.containsKey(key) | ||
| } | ||
|
|
||
| private def setConfWithCheck(key: String, value: String): Unit = { | ||
| protected def setConfWithCheck(key: String, value: String): Unit = { | ||
| settings.put(key, value) | ||
| } | ||
|
|
||
|
|
||
| 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,41 @@ 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) | ||
| } | ||
| } | ||
| } | ||
|
|
||
| /** | ||
| * Wrap an action with specified SQL configs. These configs will be propagated to the executor | ||
| * side via job local properties. | ||
| */ | ||
| def withSQLConfPropagated[T](sparkSession: SparkSession)(body: => T): T = { | ||
| val sc = sparkSession.sparkContext | ||
| // 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 = allConfigs.collect { | ||
| case (key, value) if key.startsWith("spark") => | ||
| val originalValue = sc.getLocalProperty(key) | ||
| sc.setLocalProperty(key, value) | ||
|
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. Is that possible the same key already exists?
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. If users happen to set the same key in the local properties and want to access them in tasks, we will break it. It's very unlikely to happen and I'd say SQL config keys should be reserved for internal usage only. |
||
| (key, originalValue) | ||
| } | ||
|
|
||
| try { | ||
| sc.setLocalProperty(SQLExecution.EXECUTION_ID_KEY, executionId) | ||
| body | ||
| } finally { | ||
| sc.setLocalProperty(SQLExecution.EXECUTION_ID_KEY, oldExecutionId) | ||
| for ((key, value) <- originalLocalProps) { | ||
|
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. before we set the original one, should we reset the new key with null values? def setLocalProperty(key: String, value: String) {
if (value == null) {
localProperties.get.remove(key)
} else {
localProperties.get.setProperty(key, value)
}
}
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.
|
||
| sc.setLocalProperty(key, value) | ||
| } | ||
| } | ||
| } | ||
| } | ||
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -45,8 +45,9 @@ private[sql] object JsonInferSchema { | |
| val parseMode = configOptions.parseMode | ||
| val columnNameOfCorruptRecord = configOptions.columnNameOfCorruptRecord | ||
|
|
||
| // perform schema inference on each row and merge afterwards | ||
| val rootType = json.mapPartitions { iter => | ||
| // In each RDD partition, perform schema inference on each row and merge afterwards. | ||
| val typeMerger = compatibleRootType(columnNameOfCorruptRecord, parseMode) | ||
| val mergedTypesFromPartitions = json.mapPartitions { iter => | ||
| val factory = new JsonFactory() | ||
| configOptions.setJacksonOptions(factory) | ||
| iter.flatMap { row => | ||
|
|
@@ -66,9 +67,13 @@ private[sql] object JsonInferSchema { | |
| s"Parse Mode: ${FailFastMode.name}.", e) | ||
| } | ||
| } | ||
| } | ||
| }.fold(StructType(Nil))( | ||
| compatibleRootType(columnNameOfCorruptRecord, parseMode)) | ||
| }.reduceOption(typeMerger).toIterator | ||
| } | ||
|
|
||
| // Here we get RDD local iterator then fold, instead of calling `RDD.fold` directly, because | ||
| // `RDD.fold` will run the fold function in DAGScheduler event loop thread, which may not have | ||
| // active SparkSession and `SQLConf.get` may point to the wrong configs. | ||
| val rootType = mergedTypesFromPartitions.toLocalIterator.fold(StructType(Nil))(typeMerger) | ||
|
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 re-run the
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. LGTM
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. can the same problem happen also in other places? this seems to be quite a tricky issue which may happen in general. Can we avoid it somehow? |
||
|
|
||
| canonicalizeType(rootType) match { | ||
| case Some(st: StructType) => st | ||
|
|
||
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.
Need to update the above comments.