-
Notifications
You must be signed in to change notification settings - Fork 29k
[SPARK-8981][CORE][test-hadoop3.2][test-java11] Add MDC support in Executor #26624
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 2 commits
c0e6f03
b43f861
0324746
0c23e26
04078b4
4deda15
0685aba
d5c1aa9
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 |
|---|---|---|
|
|
@@ -17,6 +17,7 @@ | |
|
|
||
| package org.apache.spark.util | ||
|
|
||
| import java.util | ||
| import java.util.concurrent._ | ||
| import java.util.concurrent.{Future => JFuture} | ||
| import java.util.concurrent.locks.ReentrantLock | ||
|
|
@@ -32,6 +33,99 @@ import org.apache.spark.SparkException | |
|
|
||
| private[spark] object ThreadUtils { | ||
|
|
||
| object MDCAwareThreadPoolExecutor { | ||
| def newCachedThreadPool(threadFactory: ThreadFactory): ThreadPoolExecutor = { | ||
| // The values needs to be synced with `Executors.newCachedThreadPool` | ||
| new MDCAwareThreadPoolExecutor( | ||
igreenfield marked this conversation as resolved.
Outdated
Show resolved
Hide resolved
|
||
| 0, | ||
| Integer.MAX_VALUE, | ||
| 60L, | ||
| TimeUnit.SECONDS, | ||
| new SynchronousQueue[Runnable], | ||
| threadFactory) | ||
| } | ||
|
|
||
| def newFixedThreadPool(nThreads: Int, threadFactory: ThreadFactory): ThreadPoolExecutor = { | ||
| // The values needs to be synced with `Executors.newFixedThreadPool` | ||
| new MDCAwareThreadPoolExecutor( | ||
igreenfield marked this conversation as resolved.
Outdated
Show resolved
Hide resolved
|
||
| nThreads, | ||
| nThreads, | ||
| 0L, | ||
| TimeUnit.MILLISECONDS, | ||
| new LinkedBlockingQueue[Runnable], | ||
| threadFactory) | ||
| } | ||
|
|
||
| /** | ||
| * This method differ from the `java.util.concurrent.Executors#newSingleThreadExecutor` in | ||
| * 2 ways: | ||
| * 1. It use `org.apache.spark.util.ThreadUtils.MDCAwareThreadPoolExecutor` | ||
| * as underline `java.util.concurrent.ExecutorService` | ||
| * 2. It does not use the | ||
| * `java.util.concurrent.Executors#FinalizableDelegatedExecutorService` from JDK | ||
| */ | ||
| def newSingleThreadExecutor(threadFactory: ThreadFactory): ExecutorService = { | ||
| // The values needs to be synced with `Executors.newSingleThreadExecutor` | ||
| Executors.unconfigurableExecutorService( | ||
| new MDCAwareThreadPoolExecutor( | ||
igreenfield marked this conversation as resolved.
Outdated
Show resolved
Hide resolved
|
||
| 1, | ||
| 1, | ||
| 0L, | ||
| TimeUnit.MILLISECONDS, | ||
| new LinkedBlockingQueue[Runnable], | ||
| threadFactory) | ||
| ) | ||
| } | ||
|
|
||
| } | ||
|
|
||
| class MDCAwareRunnable(proxy: Runnable) extends Runnable { | ||
| val callerThreadMDC: util.Map[String, String] = getMDCMap | ||
|
|
||
| @inline | ||
|
||
| private def getMDCMap: util.Map[String, String] = { | ||
| org.slf4j.MDC.getCopyOfContextMap match { | ||
| case null => new util.HashMap[String, String]() | ||
|
||
| case m => m | ||
| } | ||
| } | ||
|
|
||
| override def run(): Unit = { | ||
| val threadMDC = getMDCMap | ||
|
||
| org.slf4j.MDC.setContextMap(callerThreadMDC) | ||
| try { | ||
| proxy.run() | ||
| } finally { | ||
| org.slf4j.MDC.setContextMap(threadMDC) | ||
igreenfield marked this conversation as resolved.
Outdated
Show resolved
Hide resolved
|
||
| } | ||
| } | ||
| } | ||
|
|
||
| class MDCAwareScheduledThreadPoolExecutor( | ||
| corePoolSize: Int, | ||
| threadFactory: ThreadFactory) | ||
| extends ScheduledThreadPoolExecutor(corePoolSize, threadFactory) { | ||
|
|
||
| override def execute(runnable: Runnable) { | ||
| super.execute(new MDCAwareRunnable(runnable)) | ||
| } | ||
| } | ||
|
|
||
| class MDCAwareThreadPoolExecutor( | ||
| corePoolSize: Int, | ||
| maximumPoolSize: Int, | ||
| keepAliveTime: Long, | ||
| unit: TimeUnit, | ||
| workQueue: BlockingQueue[Runnable], | ||
| threadFactory: ThreadFactory) | ||
| extends ThreadPoolExecutor(corePoolSize, maximumPoolSize, | ||
| keepAliveTime, unit, workQueue, threadFactory) { | ||
|
|
||
| override def execute(runnable: Runnable) { | ||
| super.execute(new MDCAwareRunnable(runnable)) | ||
| } | ||
| } | ||
|
|
||
| private val sameThreadExecutionContext = | ||
| ExecutionContext.fromExecutorService(sameThreadExecutorService()) | ||
|
|
||
|
|
@@ -130,7 +224,15 @@ private[spark] object ThreadUtils { | |
| */ | ||
| def newDaemonCachedThreadPool(prefix: String): ThreadPoolExecutor = { | ||
|
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 you point out where we call these If your use case is for the driver side, can we document the usage? e.g. users can set MDC properties via
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. without my code, it will not work hence the MDC will not passed to the thread pull.
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. in my usecase I set the MDC in job-server so the driver will use it. but without the MDCAware it will not work. so I don't understand what you want to document.
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. Does your job server use these The executor logs can print the
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. My code just set the MDC and the only way to pass MDC between ThreadPools is the code I add to the ThreadUtils.
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. What is the propagating you are talking about? A common Spark application looks like If you are using a job server and your job server submits Spark jobs in a thread pool, please propagate the MDC properties in that thread pool. But that thread pool is not part of Spark. If you are referencing to other thread pools inside Spark, please point it out. 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. AFAIK, the thread pool is part of spark - unless we have different views of what "spark" is.
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. Please note that, If you use these private APIs in your Spark application, and has new requirements, please fix it in your Spark application. You can simply put these
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. ok, I will move the ThreadUtil change to another pool request, will need to set MDC in
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. @cloud-fan pushed changes. please review |
||
| val threadFactory = namedThreadFactory(prefix) | ||
| Executors.newCachedThreadPool(threadFactory).asInstanceOf[ThreadPoolExecutor] | ||
| MDCAwareThreadPoolExecutor.newCachedThreadPool(threadFactory) | ||
| } | ||
|
|
||
| /** | ||
| * Wrapper over newCachedThreadPool. Thread names are formatted as prefix-ID, where ID is a | ||
igreenfield marked this conversation as resolved.
Outdated
Show resolved
Hide resolved
|
||
| * unique, sequentially assigned integer. | ||
| */ | ||
| def newCachedThreadPool(threadFactory: ThreadFactory): ThreadPoolExecutor = { | ||
| MDCAwareThreadPoolExecutor.newCachedThreadPool(threadFactory) | ||
| } | ||
|
|
||
| /** | ||
|
|
@@ -140,7 +242,7 @@ private[spark] object ThreadUtils { | |
| def newDaemonCachedThreadPool( | ||
| prefix: String, maxThreadNumber: Int, keepAliveSeconds: Int = 60): ThreadPoolExecutor = { | ||
| val threadFactory = namedThreadFactory(prefix) | ||
| val threadPool = new ThreadPoolExecutor( | ||
| val threadPool = new MDCAwareThreadPoolExecutor( | ||
| maxThreadNumber, // corePoolSize: the max number of threads to create before queuing the tasks | ||
| maxThreadNumber, // maximumPoolSize: because we use LinkedBlockingDeque, this one is not used | ||
| keepAliveSeconds, | ||
|
|
@@ -165,15 +267,15 @@ private[spark] object ThreadUtils { | |
| */ | ||
| def newDaemonSingleThreadExecutor(threadName: String): ExecutorService = { | ||
| val threadFactory = new ThreadFactoryBuilder().setDaemon(true).setNameFormat(threadName).build() | ||
| Executors.newSingleThreadExecutor(threadFactory) | ||
| MDCAwareThreadPoolExecutor.newSingleThreadExecutor(threadFactory) | ||
| } | ||
|
|
||
| /** | ||
| * Wrapper over ScheduledThreadPoolExecutor. | ||
| */ | ||
| def newDaemonSingleThreadScheduledExecutor(threadName: String): ScheduledExecutorService = { | ||
| val threadFactory = new ThreadFactoryBuilder().setDaemon(true).setNameFormat(threadName).build() | ||
| val executor = new ScheduledThreadPoolExecutor(1, threadFactory) | ||
| val executor = new MDCAwareScheduledThreadPoolExecutor(1, threadFactory) | ||
| // By default, a cancelled task is not automatically removed from the work queue until its delay | ||
| // elapses. We have to enable it manually. | ||
| executor.setRemoveOnCancelPolicy(true) | ||
|
|
@@ -189,7 +291,7 @@ private[spark] object ThreadUtils { | |
| .setDaemon(true) | ||
| .setNameFormat(s"$threadNamePrefix-%d") | ||
| .build() | ||
| val executor = new ScheduledThreadPoolExecutor(numThreads, threadFactory) | ||
| val executor = new MDCAwareScheduledThreadPoolExecutor(numThreads, threadFactory) | ||
| // By default, a cancelled task is not automatically removed from the work queue until its delay | ||
| // elapses. We have to enable it manually. | ||
| executor.setRemoveOnCancelPolicy(true) | ||
|
|
||
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -2955,6 +2955,12 @@ Spark uses [log4j](http://logging.apache.org/log4j/) for logging. You can config | |
| `log4j.properties` file in the `conf` directory. One way to start is to copy the existing | ||
| `log4j.properties.template` located there. | ||
|
|
||
| By default, Spark adds 1 record to the MDC: `taskName`, which shows something | ||
igreenfield marked this conversation as resolved.
Outdated
Show resolved
Hide resolved
|
||
| like `task 1.0 in stage 0.0`. You can add `%X{taskName}` to your patternLayout in | ||
|
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.
Could you give an example to show how to use it in this document? For example, use an example to show how to specify your application names/identifiers. 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. @gatorsmile were you able to figure this out? My MDC values are not propagating in my logs after following this same procedure. 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. @alefischer13 Just a guess, but it looks like this was changed in 54e702c so that the MDC key still includes the |
||
| order to print it in the logs. | ||
| Moreover, you can use `spark.sparkContext.setLocalProperty("mdc." + name, "value")` to add user specific data into MDC. | ||
| The key in MDC will be the string after the `mdc.` prefix. | ||
|
|
||
| # Overriding configuration directory | ||
|
|
||
| To specify a different configuration directory other than the default "SPARK_HOME/conf", | ||
|
|
||
Uh oh!
There was an error while loading. Please reload this page.