viirya commented on a change in pull request #26624:
URL: https://github.com/apache/spark/pull/26624#discussion_r419925692
##########
File path: core/src/main/scala/org/apache/spark/util/ThreadUtils.scala
##########
@@ -17,21 +17,114 @@
package org.apache.spark.util
+import java.util
import java.util.concurrent._
import java.util.concurrent.locks.ReentrantLock
+import com.google.common.util.concurrent.{MoreExecutors, ThreadFactoryBuilder}
import scala.concurrent.{Awaitable, ExecutionContext,
ExecutionContextExecutor, Future}
import scala.concurrent.duration.{Duration, FiniteDuration}
import scala.language.higherKinds
import scala.util.control.NonFatal
-import com.google.common.util.concurrent.ThreadFactoryBuilder
-
import org.apache.spark.SparkException
import org.apache.spark.rpc.RpcAbortException
private[spark] object ThreadUtils {
+ object MDCAwareThreadPoolExecutor {
+ def newCachedThreadPool(threadFactory: ThreadFactory): ThreadPoolExecutor
= {
+ // The values needs to be synced with `Executors.newCachedThreadPool`
+ new MDCAwareThreadPoolExecutor(
+ 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(
+ 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(
+ 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
Review comment:
Is any reason to keep current thread's MDC? I think when you use
MDCAwareRunnable, you always pass in some MDC or getting empty map. Do we use
current thread's MDC for logging?
----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
For queries about this service, please contact Infrastructure at:
[email protected]
---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]