forked from delta-io/delta
-
Notifications
You must be signed in to change notification settings - Fork 0
Commit
This commit does not belong to any branch on this repository, and may belong to a fork outside of the repository.
[Spark] Propagate thread locals to Delta thread pools
* The default thread pool executor in Apache Spark does not forward thread locals to threads spawned in a thread pool. * This can cause issues if the threads depend on the thread locals. * To fix this, we introduce a wrapper class around the thread pool executor that forwards thread locals. Closes delta-io#2154 GitOrigin-RevId: 9e9423e4b041232457ffaab18f5f96490bb45b88
- Loading branch information
Showing
6 changed files
with
366 additions
and
5 deletions.
There are no files selected for viewing
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
118 changes: 118 additions & 0 deletions
118
...rg/apache/spark/sql/delta/util/threads/SparkThreadLocalForwardingThreadPoolExecutor.scala
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,118 @@ | ||
/* | ||
* Copyright (2021) The Delta Lake Project Authors. | ||
* | ||
* Licensed 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.delta.util.threads | ||
|
||
import java.util.Properties | ||
import java.util.concurrent._ | ||
|
||
import scala.collection.JavaConverters._ | ||
|
||
import org.apache.spark.{SparkContext, TaskContext} | ||
import org.apache.spark.internal.Logging | ||
import org.apache.spark.util.{Utils => SparkUtils} | ||
|
||
/** | ||
* Implementation of ThreadPoolExecutor that captures the Spark ThreadLocals present at submit time | ||
* and inserts them into the thread before executing the provided runnable. | ||
*/ | ||
class SparkThreadLocalForwardingThreadPoolExecutor( | ||
corePoolSize: Int, | ||
maximumPoolSize: Int, | ||
keepAliveTime: Long, | ||
unit: TimeUnit, | ||
workQueue: BlockingQueue[Runnable], | ||
threadFactory: ThreadFactory, | ||
rejectedExecutionHandler: RejectedExecutionHandler = new ThreadPoolExecutor.AbortPolicy) | ||
extends ThreadPoolExecutor( | ||
corePoolSize, maximumPoolSize, keepAliveTime, | ||
unit, workQueue, threadFactory, rejectedExecutionHandler) { | ||
|
||
override def execute(command: Runnable): Unit = | ||
super.execute(new SparkThreadLocalCapturingRunnable(command)) | ||
} | ||
|
||
|
||
trait SparkThreadLocalCapturingHelper extends Logging { | ||
// At the time of creating this instance we capture the task context and command context. | ||
val capturedTaskContext = TaskContext.get() | ||
val sparkContext = SparkContext.getActive | ||
// Capture an immutable threadsafe snapshot of the current local properties | ||
val capturedProperties = sparkContext | ||
.map(sc => CapturedSparkThreadLocals.toValuesArray( | ||
SparkUtils.cloneProperties(sc.getLocalProperties))) | ||
|
||
def runWithCaptured[T](body: => T): T = { | ||
// Save the previous contexts, overwrite them with the captured contexts, and then restore the | ||
// previous when execution completes. | ||
// This has the unfortunate side effect of writing nulls to these thread locals if they were | ||
// empty beforehand. | ||
val previousTaskContext = TaskContext.get() | ||
val previousProperties = sparkContext.map(_.getLocalProperties) | ||
|
||
TaskContext.setTaskContext(capturedTaskContext) | ||
for { | ||
p <- capturedProperties | ||
sc <- sparkContext | ||
} { | ||
sc.setLocalProperties(CapturedSparkThreadLocals.toProperties(p)) | ||
} | ||
|
||
try { | ||
body | ||
} catch { | ||
case t: Throwable => | ||
logError(s"Exception in thread ${Thread.currentThread().getName}", t) | ||
throw t | ||
} finally { | ||
TaskContext.setTaskContext(previousTaskContext) | ||
for { | ||
p <- previousProperties | ||
sc <- sparkContext | ||
} { | ||
sc.setLocalProperties(p) | ||
} | ||
} | ||
} | ||
} | ||
|
||
class CapturedSparkThreadLocals extends SparkThreadLocalCapturingHelper | ||
|
||
object CapturedSparkThreadLocals { | ||
def apply(): CapturedSparkThreadLocals = { | ||
new CapturedSparkThreadLocals() | ||
} | ||
|
||
def toProperties(props: Array[(String, String)]): Properties = { | ||
val resultProps = new Properties() | ||
for ((key, value) <- props) { | ||
resultProps.put(key, value) | ||
} | ||
resultProps | ||
} | ||
|
||
def toValuesArray(props: Properties): Array[(String, String)] = { | ||
props.asScala.toArray | ||
} | ||
|
||
} | ||
|
||
class SparkThreadLocalCapturingRunnable(runnable: Runnable) | ||
extends Runnable with SparkThreadLocalCapturingHelper { | ||
override def run(): Unit = { | ||
runWithCaptured(runnable.run()) | ||
} | ||
} |
69 changes: 69 additions & 0 deletions
69
spark/src/test/scala/org/apache/spark/sql/delta/util/threads/DeltaThreadPoolSuite.scala
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,69 @@ | ||
/* | ||
* Copyright (2021) The Delta Lake Project Authors. | ||
* | ||
* Licensed 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.delta.util.threads | ||
|
||
import java.util.Properties | ||
|
||
import org.apache.spark.{SparkFunSuite, TaskContext, TaskContextImpl} | ||
import org.apache.spark.sql.test.SharedSparkSession | ||
|
||
class DeltaThreadPoolSuite extends SparkFunSuite with SharedSparkSession { | ||
|
||
val threadPool: DeltaThreadPool = DeltaThreadPool("test", 1) | ||
|
||
def makeTaskContext(id: Int): TaskContext = { | ||
new TaskContextImpl(id, 0, 0, 0, attemptNumber = 45613, 0, null, new Properties(), null) | ||
} | ||
|
||
def testForwarding(testName: String, id: Int)(f: => Unit): Unit = { | ||
test(testName) { | ||
val prevTaskContext = TaskContext.get() | ||
TaskContext.setTaskContext(makeTaskContext(id)) | ||
sparkContext.setLocalProperty("test", id.toString) | ||
|
||
try { | ||
f | ||
} finally { | ||
TaskContext.setTaskContext(prevTaskContext) | ||
} | ||
} | ||
} | ||
|
||
def assertTaskAndProperties(id: Int): Unit = { | ||
assert(TaskContext.get() !== null) | ||
assert(TaskContext.get().stageId() === id) | ||
assert(sparkContext.getLocalProperty("test") === id.toString) | ||
} | ||
|
||
testForwarding("parallelMap captures TaskContext", id = 0) { | ||
threadPool.parallelMap(spark, 0 until 1) { _ => | ||
assertTaskAndProperties(id = 0) | ||
} | ||
} | ||
|
||
testForwarding("submit captures TaskContext and local properties", id = 1) { | ||
threadPool.submit(spark) { | ||
assertTaskAndProperties(id = 1) | ||
} | ||
} | ||
|
||
testForwarding("submitNonFateSharing captures TaskContext and local properties", id = 2) { | ||
threadPool.submitNonFateSharing { _ => | ||
assertTaskAndProperties(id = 2) | ||
} | ||
} | ||
} |
Oops, something went wrong.