Skip to content
Closed
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
2 changes: 2 additions & 0 deletions core/src/main/scala/org/apache/spark/TaskContextImpl.scala
Original file line number Diff line number Diff line change
Expand Up @@ -178,4 +178,6 @@ private[spark] class TaskContextImpl(

private[spark] def fetchFailed: Option[FetchFailedException] = _fetchFailedException

// TODO: shall we publish it and define it in `TaskContext`?
private[spark] def getLocalProperties(): Properties = localProperties
}
Original file line number Diff line number Diff line change
Expand Up @@ -206,7 +206,7 @@ class DAGScheduler(
private val messageScheduler =
ThreadUtils.newDaemonSingleThreadScheduledExecutor("dag-scheduler-message")

private[scheduler] val eventProcessLoop = new DAGSchedulerEventProcessLoop(this)
private[spark] val eventProcessLoop = new DAGSchedulerEventProcessLoop(this)
taskScheduler.setDAGScheduler(this)

/**
Expand Down
3 changes: 2 additions & 1 deletion core/src/main/scala/org/apache/spark/util/EventLoop.scala
Original file line number Diff line number Diff line change
Expand Up @@ -37,7 +37,8 @@ private[spark] abstract class EventLoop[E](name: String) extends Logging {

private val stopped = new AtomicBoolean(false)

private val eventThread = new Thread(name) {
// Exposed for testing.
private[spark] val eventThread = new Thread(name) {
setDaemon(true)

override def run(): Unit = {
Expand Down
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
Expand Up @@ -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
Expand Down Expand Up @@ -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).
Expand All @@ -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.
Copy link
Member

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.

*/
def get: SQLConf = confGetter.get()()
def get: SQLConf = {
if (TaskContext.get != null) {
new ReadOnlySQLConf(TaskContext.get())
} else {
if (Utils.isTesting && SparkContext.getActive.isDefined) {
Copy link
Member

Choose a reason for hiding this comment

The 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.")
}
Copy link
Member

@viirya viirya May 20, 2018

Choose a reason for hiding this comment

The 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()
Expand Down Expand Up @@ -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 ******************* */

Expand Down Expand Up @@ -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)
}

Expand Down
21 changes: 18 additions & 3 deletions sql/core/src/main/scala/org/apache/spark/sql/SparkSession.scala
Original file line number Diff line number Diff line change
Expand Up @@ -24,7 +24,7 @@ import scala.collection.JavaConverters._
import scala.reflect.runtime.universe.TypeTag
import scala.util.control.NonFatal

import org.apache.spark.{SPARK_VERSION, SparkConf, SparkContext}
import org.apache.spark.{SPARK_VERSION, SparkConf, SparkContext, TaskContext}
import org.apache.spark.annotation.{DeveloperApi, Experimental, InterfaceStability}
import org.apache.spark.api.java.JavaRDD
import org.apache.spark.internal.Logging
Expand Down Expand Up @@ -898,6 +898,7 @@ object SparkSession extends Logging {
* @since 2.0.0
*/
def getOrCreate(): SparkSession = synchronized {
assertOnDriver()
// Get the session from current thread's active session.
var session = activeThreadSession.get()
if ((session ne null) && !session.sparkContext.isStopped) {
Expand Down Expand Up @@ -1022,14 +1023,20 @@ object SparkSession extends Logging {
*
* @since 2.2.0
*/
def getActiveSession: Option[SparkSession] = Option(activeThreadSession.get)
def getActiveSession: Option[SparkSession] = {
assertOnDriver()
Option(activeThreadSession.get)
}

/**
* Returns the default SparkSession that is returned by the builder.
*
* @since 2.2.0
*/
def getDefaultSession: Option[SparkSession] = Option(defaultSession.get)
def getDefaultSession: Option[SparkSession] = {
assertOnDriver()
Option(defaultSession.get)
}

/**
* Returns the currently active SparkSession, otherwise the default one. If there is no default
Expand Down Expand Up @@ -1062,6 +1069,14 @@ object SparkSession extends Logging {
}
}

private def assertOnDriver(): Unit = {
if (Utils.isTesting && TaskContext.get != null) {
// we're accessing it during task execution, fail.
throw new IllegalStateException(
"SparkSession should only be created and accessed on the driver.")
}
}

/**
* Helper method to create an instance of `SessionState` based on `className` from conf.
* The result is either `SessionState` or a Hive based `SessionState`.
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -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)
Expand All @@ -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)
Copy link
Member

Choose a reason for hiding this comment

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

Is that possible the same key already exists?

Copy link
Contributor Author

Choose a reason for hiding this comment

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

Choose a reason for hiding this comment

The 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)
    }
  }

Copy link
Contributor Author

Choose a reason for hiding this comment

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

originalLocalProps already contains entries with null value.

sc.setLocalProperty(key, value)
}
}
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -629,7 +629,7 @@ case class SubqueryExec(name: String, child: SparkPlan) extends UnaryExecNode {
Future {
// This will run in another thread. Set the execution id so that we can connect these jobs
// with the correct execution.
SQLExecution.withExecutionId(sparkContext, executionId) {
SQLExecution.withExecutionId(sqlContext.sparkSession, executionId) {
val beforeCollect = System.nanoTime()
// Note that we use .executeCollect() because we don't want to convert data to Scala types
val rows: Array[InternalRow] = child.executeCollect()
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -34,6 +34,7 @@ import org.apache.spark.rdd.{BinaryFileRDD, RDD}
import org.apache.spark.sql.{Dataset, Encoders, SparkSession}
import org.apache.spark.sql.catalyst.InternalRow
import org.apache.spark.sql.catalyst.json.{CreateJacksonParser, JacksonParser, JSONOptions}
import org.apache.spark.sql.execution.SQLExecution
import org.apache.spark.sql.execution.datasources._
import org.apache.spark.sql.execution.datasources.text.TextFileFormat
import org.apache.spark.sql.types.StructType
Expand Down Expand Up @@ -104,22 +105,19 @@ object TextInputJsonDataSource extends JsonDataSource {
CreateJacksonParser.internalRow(enc, _: JsonFactory, _: InternalRow)
}.getOrElse(CreateJacksonParser.internalRow(_: JsonFactory, _: InternalRow))

JsonInferSchema.infer(rdd, parsedOptions, rowParser)
SQLExecution.withSQLConfPropagated(json.sparkSession) {
JsonInferSchema.infer(rdd, parsedOptions, rowParser)
}
}

private def createBaseDataset(
sparkSession: SparkSession,
inputPaths: Seq[FileStatus],
parsedOptions: JSONOptions): Dataset[String] = {
val paths = inputPaths.map(_.getPath.toString)
val textOptions = Map.empty[String, String] ++
parsedOptions.encoding.map("encoding" -> _) ++
parsedOptions.lineSeparator.map("lineSep" -> _)

sparkSession.baseRelationToDataFrame(
DataSource.apply(
sparkSession,
paths = paths,
paths = inputPaths.map(_.getPath.toString),
className = classOf[TextFileFormat].getName,
options = parsedOptions.parameters
).resolveRelation(checkFilesExist = false))
Expand Down Expand Up @@ -165,7 +163,9 @@ object MultiLineJsonDataSource extends JsonDataSource {
.map(enc => createParser(enc, _: JsonFactory, _: PortableDataStream))
.getOrElse(createParser(_: JsonFactory, _: PortableDataStream))

JsonInferSchema.infer[PortableDataStream](sampled, parsedOptions, parser)
SQLExecution.withSQLConfPropagated(sparkSession) {
JsonInferSchema.infer[PortableDataStream](sampled, parsedOptions, parser)
}
}

private def createBaseRdd(
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -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 =>
Expand All @@ -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)
Copy link
Contributor Author

Choose a reason for hiding this comment

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

I re-run the JsonBenmark and no performance regression is observed.

Copy link
Member

Choose a reason for hiding this comment

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

LGTM

Copy link
Contributor

Choose a reason for hiding this comment

The 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
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -69,7 +69,7 @@ case class BroadcastExchangeExec(
Future {
// This will run in another thread. Set the execution id so that we can connect these jobs
// with the correct execution.
SQLExecution.withExecutionId(sparkContext, executionId) {
SQLExecution.withExecutionId(sqlContext.sparkSession, executionId) {
try {
val beforeCollect = System.nanoTime()
// Use executeCollect/executeCollectIterator to avoid conversion to Scala types
Expand Down
Loading