2009-12-11 16:37:44 +01:00
|
|
|
/**
|
2018-01-04 17:26:29 +00:00
|
|
|
* Copyright (C) 2009-2018 Lightbend Inc. <https://www.lightbend.com>
|
2009-12-11 16:37:44 +01:00
|
|
|
*/
|
|
|
|
|
|
2010-10-26 12:49:25 +02:00
|
|
|
package akka.dispatch
|
2009-12-11 16:37:44 +01:00
|
|
|
|
2010-03-01 22:03:17 +01:00
|
|
|
import java.util.Collection
|
2016-02-22 20:18:15 +01:00
|
|
|
import scala.concurrent.{ BlockContext, CanAwait }
|
2012-09-21 14:50:06 +02:00
|
|
|
import scala.concurrent.duration.Duration
|
2017-02-22 17:10:37 +01:00
|
|
|
import akka.dispatch.forkjoin._
|
2012-07-17 17:21:08 +02:00
|
|
|
import java.util.concurrent.{
|
|
|
|
|
ArrayBlockingQueue,
|
|
|
|
|
BlockingQueue,
|
|
|
|
|
Callable,
|
|
|
|
|
ExecutorService,
|
|
|
|
|
LinkedBlockingQueue,
|
|
|
|
|
RejectedExecutionHandler,
|
|
|
|
|
RejectedExecutionException,
|
|
|
|
|
SynchronousQueue,
|
|
|
|
|
TimeUnit,
|
|
|
|
|
ThreadFactory,
|
|
|
|
|
ThreadPoolExecutor
|
|
|
|
|
}
|
|
|
|
|
import java.util.concurrent.atomic.{ AtomicReference, AtomicLong }
|
2010-10-22 17:50:48 +02:00
|
|
|
|
|
|
|
|
object ThreadPoolConfig {
|
2011-05-18 17:25:30 +02:00
|
|
|
type QueueFactory = () ⇒ BlockingQueue[Runnable]
|
2010-10-22 17:50:48 +02:00
|
|
|
|
|
|
|
|
val defaultAllowCoreThreadTimeout: Boolean = false
|
2011-05-18 17:25:30 +02:00
|
|
|
val defaultCorePoolSize: Int = 16
|
|
|
|
|
val defaultMaxPoolSize: Int = 128
|
|
|
|
|
val defaultTimeout: Duration = Duration(60000L, TimeUnit.MILLISECONDS)
|
2012-01-20 12:30:19 +01:00
|
|
|
val defaultRejectionPolicy: RejectedExecutionHandler = new SaneRejectedExecutionHandler()
|
2010-10-22 17:50:48 +02:00
|
|
|
|
2012-06-04 13:34:30 +02:00
|
|
|
def scaledPoolSize(floor: Int, multiplier: Double, ceiling: Int): Int =
|
|
|
|
|
math.min(math.max((Runtime.getRuntime.availableProcessors * multiplier).ceil.toInt, floor), ceiling)
|
2010-10-22 17:50:48 +02:00
|
|
|
|
2012-06-04 13:34:30 +02:00
|
|
|
def arrayBlockingQueue(capacity: Int, fair: Boolean): QueueFactory = () ⇒ new ArrayBlockingQueue[Runnable](capacity, fair)
|
2010-10-22 17:50:48 +02:00
|
|
|
|
2012-06-04 13:34:30 +02:00
|
|
|
def synchronousQueue(fair: Boolean): QueueFactory = () ⇒ new SynchronousQueue[Runnable](fair)
|
2010-10-22 17:50:48 +02:00
|
|
|
|
2012-06-04 13:34:30 +02:00
|
|
|
def linkedBlockingQueue(): QueueFactory = () ⇒ new LinkedBlockingQueue[Runnable]()
|
2010-10-22 17:50:48 +02:00
|
|
|
|
2012-06-04 13:34:30 +02:00
|
|
|
def linkedBlockingQueue(capacity: Int): QueueFactory = () ⇒ new LinkedBlockingQueue[Runnable](capacity)
|
2010-10-22 17:50:48 +02:00
|
|
|
|
2012-06-04 13:34:30 +02:00
|
|
|
def reusableQueue(queue: BlockingQueue[Runnable]): QueueFactory = () ⇒ queue
|
2010-10-22 17:50:48 +02:00
|
|
|
|
2012-06-04 13:34:30 +02:00
|
|
|
def reusableQueue(queueFactory: QueueFactory): QueueFactory = reusableQueue(queueFactory())
|
2010-10-22 17:50:48 +02:00
|
|
|
}
|
|
|
|
|
|
2011-10-12 18:21:12 +02:00
|
|
|
/**
|
|
|
|
|
* Function0 without the fun stuff (mostly for the sake of the Java API side of things)
|
|
|
|
|
*/
|
2011-07-01 22:13:56 +02:00
|
|
|
trait ExecutorServiceFactory {
|
|
|
|
|
def createExecutorService: ExecutorService
|
|
|
|
|
}
|
|
|
|
|
|
2011-10-12 18:21:12 +02:00
|
|
|
/**
|
|
|
|
|
* Generic way to specify an ExecutorService to a Dispatcher, create it with the given name if desired
|
|
|
|
|
*/
|
2011-07-01 22:13:56 +02:00
|
|
|
trait ExecutorServiceFactoryProvider {
|
2012-02-08 11:53:55 +01:00
|
|
|
def createExecutorServiceFactory(id: String, threadFactory: ThreadFactory): ExecutorServiceFactory
|
2011-07-01 22:13:56 +02:00
|
|
|
}
|
|
|
|
|
|
2011-10-12 18:21:12 +02:00
|
|
|
/**
|
|
|
|
|
* A small configuration DSL to create ThreadPoolExecutors that can be provided as an ExecutorServiceFactoryProvider to Dispatcher
|
|
|
|
|
*/
|
2016-06-02 14:06:57 +02:00
|
|
|
final case class ThreadPoolConfig(
|
|
|
|
|
allowCorePoolTimeout: Boolean = ThreadPoolConfig.defaultAllowCoreThreadTimeout,
|
|
|
|
|
corePoolSize: Int = ThreadPoolConfig.defaultCorePoolSize,
|
|
|
|
|
maxPoolSize: Int = ThreadPoolConfig.defaultMaxPoolSize,
|
|
|
|
|
threadTimeout: Duration = ThreadPoolConfig.defaultTimeout,
|
|
|
|
|
queueFactory: ThreadPoolConfig.QueueFactory = ThreadPoolConfig.linkedBlockingQueue(),
|
|
|
|
|
rejectionPolicy: RejectedExecutionHandler = ThreadPoolConfig.defaultRejectionPolicy)
|
2011-07-01 22:13:56 +02:00
|
|
|
extends ExecutorServiceFactoryProvider {
|
2011-12-13 20:06:07 +01:00
|
|
|
class ThreadPoolExecutorServiceFactory(val threadFactory: ThreadFactory) extends ExecutorServiceFactory {
|
2011-11-15 10:25:05 +01:00
|
|
|
def createExecutorService: ExecutorService = {
|
2012-02-13 18:14:35 +01:00
|
|
|
val service: ThreadPoolExecutor = new ThreadPoolExecutor(
|
2012-01-20 12:30:19 +01:00
|
|
|
corePoolSize,
|
|
|
|
|
maxPoolSize,
|
|
|
|
|
threadTimeout.length,
|
|
|
|
|
threadTimeout.unit,
|
|
|
|
|
queueFactory(),
|
|
|
|
|
threadFactory,
|
2012-02-13 18:14:35 +01:00
|
|
|
rejectionPolicy) with LoadMetrics {
|
|
|
|
|
def atFullThrottle(): Boolean = this.getActiveCount >= this.getPoolSize
|
|
|
|
|
}
|
2011-11-15 10:25:05 +01:00
|
|
|
service.allowCoreThreadTimeOut(allowCorePoolTimeout)
|
|
|
|
|
service
|
2010-10-22 17:50:48 +02:00
|
|
|
}
|
|
|
|
|
}
|
2012-10-02 09:31:23 +02:00
|
|
|
final def createExecutorServiceFactory(id: String, threadFactory: ThreadFactory): ExecutorServiceFactory = {
|
|
|
|
|
val tf = threadFactory match {
|
|
|
|
|
case m: MonitorableThreadFactory ⇒
|
|
|
|
|
// add the dispatcher id to the thread names
|
2012-12-06 22:50:40 +01:00
|
|
|
m.withName(m.name + "-" + id)
|
2012-10-02 09:31:23 +02:00
|
|
|
case other ⇒ other
|
|
|
|
|
}
|
|
|
|
|
new ThreadPoolExecutorServiceFactory(tf)
|
|
|
|
|
}
|
2010-10-22 17:50:48 +02:00
|
|
|
}
|
|
|
|
|
|
2011-10-12 18:21:12 +02:00
|
|
|
/**
|
|
|
|
|
* A DSL to configure and create a MessageDispatcher with a ThreadPoolExecutor
|
|
|
|
|
*/
|
2014-03-07 13:20:01 +01:00
|
|
|
final case class ThreadPoolConfigBuilder(config: ThreadPoolConfig) {
|
2010-10-22 17:50:48 +02:00
|
|
|
import ThreadPoolConfig._
|
|
|
|
|
|
2012-01-30 13:44:56 +01:00
|
|
|
def withNewThreadPoolWithCustomBlockingQueue(newQueueFactory: QueueFactory): ThreadPoolConfigBuilder =
|
2011-11-15 10:25:05 +01:00
|
|
|
this.copy(config = config.copy(queueFactory = newQueueFactory))
|
2010-10-22 17:50:48 +02:00
|
|
|
|
2012-01-30 13:44:56 +01:00
|
|
|
def withNewThreadPoolWithCustomBlockingQueue(queue: BlockingQueue[Runnable]): ThreadPoolConfigBuilder =
|
2011-05-18 17:25:30 +02:00
|
|
|
withNewThreadPoolWithCustomBlockingQueue(reusableQueue(queue))
|
2010-10-22 17:50:48 +02:00
|
|
|
|
2012-01-30 13:44:56 +01:00
|
|
|
def withNewThreadPoolWithLinkedBlockingQueueWithUnboundedCapacity: ThreadPoolConfigBuilder =
|
2011-11-15 10:25:05 +01:00
|
|
|
this.copy(config = config.copy(queueFactory = linkedBlockingQueue()))
|
2010-10-22 17:50:48 +02:00
|
|
|
|
2012-01-30 13:44:56 +01:00
|
|
|
def withNewThreadPoolWithLinkedBlockingQueueWithCapacity(capacity: Int): ThreadPoolConfigBuilder =
|
2011-11-15 10:25:05 +01:00
|
|
|
this.copy(config = config.copy(queueFactory = linkedBlockingQueue(capacity)))
|
2010-10-22 17:50:48 +02:00
|
|
|
|
2012-01-30 13:44:56 +01:00
|
|
|
def withNewThreadPoolWithSynchronousQueueWithFairness(fair: Boolean): ThreadPoolConfigBuilder =
|
2011-11-15 10:25:05 +01:00
|
|
|
this.copy(config = config.copy(queueFactory = synchronousQueue(fair)))
|
2010-10-22 17:50:48 +02:00
|
|
|
|
2012-01-30 13:44:56 +01:00
|
|
|
def withNewThreadPoolWithArrayBlockingQueueWithCapacityAndFairness(capacity: Int, fair: Boolean): ThreadPoolConfigBuilder =
|
2011-11-15 10:25:05 +01:00
|
|
|
this.copy(config = config.copy(queueFactory = arrayBlockingQueue(capacity, fair)))
|
2010-10-22 17:50:48 +02:00
|
|
|
|
2015-12-17 09:40:03 +01:00
|
|
|
def setFixedPoolSize(size: Int): ThreadPoolConfigBuilder =
|
|
|
|
|
this.copy(config = config.copy(corePoolSize = size, maxPoolSize = size))
|
|
|
|
|
|
2012-01-30 13:44:56 +01:00
|
|
|
def setCorePoolSize(size: Int): ThreadPoolConfigBuilder =
|
2015-12-17 09:40:03 +01:00
|
|
|
this.copy(config = config.copy(corePoolSize = size, maxPoolSize = math.max(size, config.maxPoolSize)))
|
2010-10-22 17:50:48 +02:00
|
|
|
|
2012-01-30 13:44:56 +01:00
|
|
|
def setMaxPoolSize(size: Int): ThreadPoolConfigBuilder =
|
2015-12-17 09:40:03 +01:00
|
|
|
this.copy(config = config.copy(maxPoolSize = math.max(size, config.corePoolSize)))
|
2010-10-22 17:50:48 +02:00
|
|
|
|
2012-01-30 13:44:56 +01:00
|
|
|
def setCorePoolSizeFromFactor(min: Int, multiplier: Double, max: Int): ThreadPoolConfigBuilder =
|
2011-12-07 11:25:27 +01:00
|
|
|
setCorePoolSize(scaledPoolSize(min, multiplier, max))
|
2010-10-22 17:50:48 +02:00
|
|
|
|
2012-01-30 13:44:56 +01:00
|
|
|
def setMaxPoolSizeFromFactor(min: Int, multiplier: Double, max: Int): ThreadPoolConfigBuilder =
|
2011-12-07 11:25:27 +01:00
|
|
|
setMaxPoolSize(scaledPoolSize(min, multiplier, max))
|
2010-10-22 17:50:48 +02:00
|
|
|
|
2012-01-30 13:44:56 +01:00
|
|
|
def setKeepAliveTimeInMillis(time: Long): ThreadPoolConfigBuilder =
|
2011-05-18 17:25:30 +02:00
|
|
|
setKeepAliveTime(Duration(time, TimeUnit.MILLISECONDS))
|
2010-10-22 17:50:48 +02:00
|
|
|
|
2012-01-30 13:44:56 +01:00
|
|
|
def setKeepAliveTime(time: Duration): ThreadPoolConfigBuilder =
|
2010-10-22 17:50:48 +02:00
|
|
|
this.copy(config = config.copy(threadTimeout = time))
|
|
|
|
|
|
2012-01-30 13:44:56 +01:00
|
|
|
def setAllowCoreThreadTimeout(allow: Boolean): ThreadPoolConfigBuilder =
|
2010-10-22 17:50:48 +02:00
|
|
|
this.copy(config = config.copy(allowCorePoolTimeout = allow))
|
2009-12-11 16:37:44 +01:00
|
|
|
|
2012-01-30 13:44:56 +01:00
|
|
|
def setQueueFactory(newQueueFactory: QueueFactory): ThreadPoolConfigBuilder =
|
2011-08-24 12:41:21 +02:00
|
|
|
this.copy(config = config.copy(queueFactory = newQueueFactory))
|
|
|
|
|
|
2012-06-04 10:35:27 +02:00
|
|
|
def configure(fs: Option[Function[ThreadPoolConfigBuilder, ThreadPoolConfigBuilder]]*): ThreadPoolConfigBuilder =
|
|
|
|
|
fs.foldLeft(this)((c, f) ⇒ f.map(_(c)).getOrElse(c))
|
2010-10-22 17:50:48 +02:00
|
|
|
}
|
2010-05-21 20:08:49 +02:00
|
|
|
|
2012-01-20 12:30:19 +01:00
|
|
|
object MonitorableThreadFactory {
|
|
|
|
|
val doNothing: Thread.UncaughtExceptionHandler =
|
|
|
|
|
new Thread.UncaughtExceptionHandler() { def uncaughtException(thread: Thread, cause: Throwable) = () }
|
2012-07-17 17:21:08 +02:00
|
|
|
|
|
|
|
|
private[akka] class AkkaForkJoinWorkerThread(_pool: ForkJoinPool) extends ForkJoinWorkerThread(_pool) with BlockContext {
|
2012-07-20 16:01:18 +02:00
|
|
|
override def blockOn[T](thunk: ⇒ T)(implicit permission: CanAwait): T = {
|
2012-07-17 17:21:08 +02:00
|
|
|
val result = new AtomicReference[Option[T]](None)
|
|
|
|
|
ForkJoinPool.managedBlock(new ForkJoinPool.ManagedBlocker {
|
|
|
|
|
def block(): Boolean = {
|
2012-07-20 16:01:18 +02:00
|
|
|
result.set(Some(thunk))
|
2012-07-17 17:21:08 +02:00
|
|
|
true
|
|
|
|
|
}
|
|
|
|
|
def isReleasable = result.get.isDefined
|
|
|
|
|
})
|
|
|
|
|
result.get.get // Exception intended if None
|
|
|
|
|
}
|
|
|
|
|
}
|
2012-01-20 12:30:19 +01:00
|
|
|
}
|
|
|
|
|
|
2016-06-02 14:06:57 +02:00
|
|
|
final case class MonitorableThreadFactory(
|
|
|
|
|
name: String,
|
|
|
|
|
daemonic: Boolean,
|
|
|
|
|
contextClassLoader: Option[ClassLoader],
|
|
|
|
|
exceptionHandler: Thread.UncaughtExceptionHandler = MonitorableThreadFactory.doNothing,
|
|
|
|
|
protected val counter: AtomicLong = new AtomicLong)
|
2012-01-27 12:44:40 +01:00
|
|
|
extends ThreadFactory with ForkJoinPool.ForkJoinWorkerThreadFactory {
|
2009-12-11 16:37:44 +01:00
|
|
|
|
2012-02-08 11:53:55 +01:00
|
|
|
def newThread(pool: ForkJoinPool): ForkJoinWorkerThread = {
|
2012-07-17 17:21:08 +02:00
|
|
|
val t = wire(new MonitorableThreadFactory.AkkaForkJoinWorkerThread(pool))
|
2012-02-08 11:53:55 +01:00
|
|
|
// Name of the threads for the ForkJoinPool are not customizable. Change it here.
|
2012-02-08 14:03:31 +01:00
|
|
|
t.setName(name + "-" + counter.incrementAndGet())
|
2012-02-08 11:53:55 +01:00
|
|
|
t
|
|
|
|
|
}
|
2012-01-30 16:34:25 +01:00
|
|
|
|
2012-02-08 11:53:55 +01:00
|
|
|
def newThread(runnable: Runnable): Thread = wire(new Thread(runnable, name + "-" + counter.incrementAndGet()))
|
2012-01-27 12:44:40 +01:00
|
|
|
|
2012-12-06 22:50:40 +01:00
|
|
|
def withName(newName: String): MonitorableThreadFactory = copy(newName)
|
|
|
|
|
|
2012-01-30 16:34:25 +01:00
|
|
|
protected def wire[T <: Thread](t: T): T = {
|
2012-01-20 12:30:19 +01:00
|
|
|
t.setUncaughtExceptionHandler(exceptionHandler)
|
2011-07-11 18:54:15 +02:00
|
|
|
t.setDaemon(daemonic)
|
2012-02-13 18:14:35 +01:00
|
|
|
contextClassLoader foreach t.setContextClassLoader
|
2011-07-11 18:54:15 +02:00
|
|
|
t
|
|
|
|
|
}
|
2010-10-22 17:50:48 +02:00
|
|
|
}
|
2009-12-11 16:37:44 +01:00
|
|
|
|
2011-10-12 18:21:12 +02:00
|
|
|
/**
|
|
|
|
|
* As the name says
|
|
|
|
|
*/
|
2011-02-28 22:54:32 +01:00
|
|
|
trait ExecutorServiceDelegate extends ExecutorService {
|
2010-10-24 00:36:56 +02:00
|
|
|
|
|
|
|
|
def executor: ExecutorService
|
|
|
|
|
|
|
|
|
|
def execute(command: Runnable) = executor.execute(command)
|
2009-12-11 16:37:44 +01:00
|
|
|
|
2018-07-25 20:38:27 +09:00
|
|
|
def shutdown(): Unit = { executor.shutdown() }
|
2009-12-11 16:37:44 +01:00
|
|
|
|
2011-04-29 10:20:16 +02:00
|
|
|
def shutdownNow() = executor.shutdownNow()
|
2009-12-11 16:37:44 +01:00
|
|
|
|
2010-10-22 17:50:48 +02:00
|
|
|
def isShutdown = executor.isShutdown
|
2009-12-11 16:37:44 +01:00
|
|
|
|
2012-05-03 21:14:47 +02:00
|
|
|
def isTerminated = executor.isTerminated
|
2009-12-11 16:37:44 +01:00
|
|
|
|
2010-10-22 17:50:48 +02:00
|
|
|
def awaitTermination(l: Long, timeUnit: TimeUnit) = executor.awaitTermination(l, timeUnit)
|
2009-12-11 16:37:44 +01:00
|
|
|
|
2010-10-22 17:50:48 +02:00
|
|
|
def submit[T](callable: Callable[T]) = executor.submit(callable)
|
2009-12-11 16:37:44 +01:00
|
|
|
|
2010-10-22 17:50:48 +02:00
|
|
|
def submit[T](runnable: Runnable, t: T) = executor.submit(runnable, t)
|
2009-12-11 16:37:44 +01:00
|
|
|
|
2010-10-22 17:50:48 +02:00
|
|
|
def submit(runnable: Runnable) = executor.submit(runnable)
|
2009-12-11 16:37:44 +01:00
|
|
|
|
2010-10-22 17:50:48 +02:00
|
|
|
def invokeAll[T](callables: Collection[_ <: Callable[T]]) = executor.invokeAll(callables)
|
2009-12-11 16:37:44 +01:00
|
|
|
|
2010-10-22 17:50:48 +02:00
|
|
|
def invokeAll[T](callables: Collection[_ <: Callable[T]], l: Long, timeUnit: TimeUnit) = executor.invokeAll(callables, l, timeUnit)
|
2009-12-11 16:37:44 +01:00
|
|
|
|
2010-10-22 17:50:48 +02:00
|
|
|
def invokeAny[T](callables: Collection[_ <: Callable[T]]) = executor.invokeAny(callables)
|
|
|
|
|
|
|
|
|
|
def invokeAny[T](callables: Collection[_ <: Callable[T]], l: Long, timeUnit: TimeUnit) = executor.invokeAny(callables, l, timeUnit)
|
2010-05-21 20:08:49 +02:00
|
|
|
}
|
2011-11-15 10:25:05 +01:00
|
|
|
|
2011-12-28 12:43:28 +01:00
|
|
|
/**
|
|
|
|
|
* The RejectedExecutionHandler used by Akka, it improves on CallerRunsPolicy
|
|
|
|
|
* by throwing a RejectedExecutionException if the executor isShutdown.
|
|
|
|
|
* (CallerRunsPolicy silently discards the runnable in this case, which is arguably broken)
|
|
|
|
|
*/
|
2011-11-15 10:25:05 +01:00
|
|
|
class SaneRejectedExecutionHandler extends RejectedExecutionHandler {
|
|
|
|
|
def rejectedExecution(runnable: Runnable, threadPoolExecutor: ThreadPoolExecutor): Unit = {
|
|
|
|
|
if (threadPoolExecutor.isShutdown) throw new RejectedExecutionException("Shutdown")
|
|
|
|
|
else runnable.run()
|
|
|
|
|
}
|
2013-01-09 01:47:48 +01:00
|
|
|
}
|