Tests green, config basically in place, need to work on start/stop semantics and countdowns
This commit is contained in:
parent
4478474681
commit
c241703a01
10 changed files with 161 additions and 308 deletions
|
|
@ -58,6 +58,9 @@ case class ThreadPoolConfig(allowCorePoolTimeout: Boolean = ThreadPoolConfig.def
|
|||
flowHandler: ThreadPoolConfig.FlowHandler = ThreadPoolConfig.defaultFlowHandler,
|
||||
queueFactory: ThreadPoolConfig.QueueFactory = ThreadPoolConfig.linkedBlockingQueue()) {
|
||||
|
||||
final def createLazyExecutorService(threadFactory: ThreadFactory): ExecutorService =
|
||||
new LazyExecutorServiceWrapper(createExecutorService(threadFactory))
|
||||
|
||||
final def createExecutorService(threadFactory: ThreadFactory): ExecutorService = {
|
||||
flowHandler match {
|
||||
case Left(rejectHandler) =>
|
||||
|
|
@ -76,6 +79,11 @@ trait DispatcherBuilder {
|
|||
def build: MessageDispatcher
|
||||
}
|
||||
|
||||
object ThreadPoolConfigDispatcherBuilder {
|
||||
def conf_?[T](opt: Option[T])(fun: (T) => ThreadPoolConfigDispatcherBuilder => ThreadPoolConfigDispatcherBuilder):
|
||||
Option[(ThreadPoolConfigDispatcherBuilder) => ThreadPoolConfigDispatcherBuilder] = opt map fun
|
||||
}
|
||||
|
||||
case class ThreadPoolConfigDispatcherBuilder(dispatcherFactory: (ThreadPoolConfig) => MessageDispatcher, config: ThreadPoolConfig) extends DispatcherBuilder {
|
||||
import ThreadPoolConfig._
|
||||
def build = dispatcherFactory(config)
|
||||
|
|
@ -133,214 +141,18 @@ case class ThreadPoolConfigDispatcherBuilder(dispatcherFactory: (ThreadPoolConfi
|
|||
|
||||
def setAllowCoreThreadTimeout(allow: Boolean): ThreadPoolConfigDispatcherBuilder =
|
||||
this.copy(config = config.copy(allowCorePoolTimeout = allow))
|
||||
|
||||
def configure(fs: Option[Function[ThreadPoolConfigDispatcherBuilder,ThreadPoolConfigDispatcherBuilder]]*):
|
||||
ThreadPoolConfigDispatcherBuilder = fs.foldLeft(this)( (c,f) => f.map( _(c) ).getOrElse(c))
|
||||
}
|
||||
|
||||
|
||||
trait ThreadPoolBuilder extends Logging {
|
||||
val name: String
|
||||
|
||||
private val NR_START_THREADS = 16
|
||||
private val NR_MAX_THREADS = 128
|
||||
private val KEEP_ALIVE_TIME = 60000L // default is one minute
|
||||
private val MILLISECONDS = TimeUnit.MILLISECONDS
|
||||
|
||||
private var threadPoolBuilder: ThreadPoolExecutor = _
|
||||
private var boundedExecutorBound = -1
|
||||
protected var mailboxCapacity = -1
|
||||
@volatile private var inProcessOfBuilding = false
|
||||
private var blockingQueue: BlockingQueue[Runnable] = _
|
||||
|
||||
protected lazy val threadFactory = new MonitorableThreadFactory(name)
|
||||
|
||||
@volatile var executor: ExecutorService = _
|
||||
|
||||
def isShutdown = executor.isShutdown
|
||||
|
||||
def buildThreadPool(): ExecutorService = synchronized {
|
||||
ensureNotActive
|
||||
inProcessOfBuilding = false
|
||||
|
||||
log.debug("Creating a %s with config [core-pool:%d,max-pool:%d,timeout:%d,allowCoreTimeout:%s,rejectPolicy:%s]",
|
||||
getClass.getName,
|
||||
threadPoolBuilder.getCorePoolSize,
|
||||
threadPoolBuilder.getMaximumPoolSize,
|
||||
threadPoolBuilder.getKeepAliveTime(MILLISECONDS),
|
||||
threadPoolBuilder.allowsCoreThreadTimeOut,
|
||||
threadPoolBuilder.getRejectedExecutionHandler.getClass.getSimpleName)
|
||||
|
||||
if (boundedExecutorBound > 0) {
|
||||
val boundedExecutor = new BoundedExecutorDecorator(threadPoolBuilder, boundedExecutorBound)
|
||||
boundedExecutorBound = -1 //Why is this here?
|
||||
executor = boundedExecutor
|
||||
} else {
|
||||
executor = threadPoolBuilder
|
||||
}
|
||||
executor
|
||||
}
|
||||
|
||||
def withNewThreadPoolWithCustomBlockingQueue(queue: BlockingQueue[Runnable]): ThreadPoolBuilder = synchronized {
|
||||
ensureNotActive
|
||||
verifyNotInConstructionPhase
|
||||
blockingQueue = queue
|
||||
threadPoolBuilder = new ThreadPoolExecutor(NR_START_THREADS, NR_MAX_THREADS, KEEP_ALIVE_TIME, MILLISECONDS, queue)
|
||||
this
|
||||
}
|
||||
|
||||
/**
|
||||
* Creates a new thread pool in which the number of tasks in the pending queue is bounded. Will block when exceeded.
|
||||
* <p/>
|
||||
* The 'bound' variable should specify the number equal to the size of the thread pool PLUS the number of queued tasks that should be followed.
|
||||
*/
|
||||
def withNewBoundedThreadPoolWithLinkedBlockingQueueWithUnboundedCapacity(bound: Int): ThreadPoolBuilder = synchronized {
|
||||
ensureNotActive
|
||||
verifyNotInConstructionPhase
|
||||
blockingQueue = new LinkedBlockingQueue[Runnable]
|
||||
threadPoolBuilder = new ThreadPoolExecutor(NR_START_THREADS, NR_MAX_THREADS, KEEP_ALIVE_TIME, MILLISECONDS, blockingQueue, threadFactory)
|
||||
boundedExecutorBound = bound
|
||||
this
|
||||
}
|
||||
|
||||
def withNewThreadPoolWithLinkedBlockingQueueWithUnboundedCapacity: ThreadPoolBuilder = synchronized {
|
||||
ensureNotActive
|
||||
verifyNotInConstructionPhase
|
||||
blockingQueue = new LinkedBlockingQueue[Runnable]
|
||||
threadPoolBuilder = new ThreadPoolExecutor(
|
||||
NR_START_THREADS, NR_MAX_THREADS, KEEP_ALIVE_TIME, MILLISECONDS, blockingQueue, threadFactory, new CallerRunsPolicy)
|
||||
this
|
||||
}
|
||||
|
||||
def withNewThreadPoolWithLinkedBlockingQueueWithCapacity(capacity: Int): ThreadPoolBuilder = synchronized {
|
||||
ensureNotActive
|
||||
verifyNotInConstructionPhase
|
||||
blockingQueue = new LinkedBlockingQueue[Runnable](capacity)
|
||||
threadPoolBuilder = new ThreadPoolExecutor(
|
||||
NR_START_THREADS, NR_MAX_THREADS, KEEP_ALIVE_TIME, MILLISECONDS, blockingQueue, threadFactory, new CallerRunsPolicy)
|
||||
this
|
||||
}
|
||||
|
||||
def withNewThreadPoolWithSynchronousQueueWithFairness(fair: Boolean): ThreadPoolBuilder = synchronized {
|
||||
ensureNotActive
|
||||
verifyNotInConstructionPhase
|
||||
blockingQueue = new SynchronousQueue[Runnable](fair)
|
||||
threadPoolBuilder = new ThreadPoolExecutor(
|
||||
NR_START_THREADS, NR_MAX_THREADS, KEEP_ALIVE_TIME, MILLISECONDS, blockingQueue, threadFactory, new CallerRunsPolicy)
|
||||
this
|
||||
}
|
||||
|
||||
def withNewThreadPoolWithArrayBlockingQueueWithCapacityAndFairness(capacity: Int, fair: Boolean): ThreadPoolBuilder = synchronized {
|
||||
ensureNotActive
|
||||
verifyNotInConstructionPhase
|
||||
blockingQueue = new ArrayBlockingQueue[Runnable](capacity, fair)
|
||||
threadPoolBuilder = new ThreadPoolExecutor(
|
||||
NR_START_THREADS, NR_MAX_THREADS, KEEP_ALIVE_TIME, MILLISECONDS, blockingQueue, threadFactory, new CallerRunsPolicy)
|
||||
this
|
||||
}
|
||||
|
||||
def configureIfPossible(f: (ThreadPoolBuilder) => Unit): Boolean = synchronized {
|
||||
if(inProcessOfBuilding) {
|
||||
f(this)
|
||||
true
|
||||
}
|
||||
else {
|
||||
log.warning("Tried to configure an already started ThreadPoolBuilder of type [%s]",getClass.getName)
|
||||
false
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Default is 16.
|
||||
*/
|
||||
def setCorePoolSize(size: Int): ThreadPoolBuilder =
|
||||
setThreadPoolExecutorProperty(_.setCorePoolSize(size))
|
||||
|
||||
/**
|
||||
* Default is 128.
|
||||
*/
|
||||
def setMaxPoolSize(size: Int): ThreadPoolBuilder =
|
||||
setThreadPoolExecutorProperty(_.setMaximumPoolSize(size))
|
||||
|
||||
|
||||
/**
|
||||
* Sets the core pool size to (availableProcessors * multipliers).ceil.toInt
|
||||
*/
|
||||
def setCorePoolSizeFromFactor(multiplier: Double): ThreadPoolBuilder =
|
||||
setThreadPoolExecutorProperty(_.setCorePoolSize(procs(multiplier)))
|
||||
|
||||
/**
|
||||
* Sets the max pool size to (availableProcessors * multipliers).ceil.toInt
|
||||
*/
|
||||
def setMaxPoolSizeFromFactor(multiplier: Double): ThreadPoolBuilder =
|
||||
setThreadPoolExecutorProperty(_.setMaximumPoolSize(procs(multiplier)))
|
||||
|
||||
/**
|
||||
* Sets the bound, -1 is unbounded
|
||||
*/
|
||||
def setExecutorBounds(bounds: Int): Unit = synchronized {
|
||||
this.boundedExecutorBound = bounds
|
||||
}
|
||||
|
||||
/**
|
||||
* Sets the mailbox capacity, -1 is unbounded
|
||||
*/
|
||||
def setMailboxCapacity(capacity: Int): Unit = synchronized {
|
||||
this.mailboxCapacity = capacity
|
||||
}
|
||||
|
||||
protected def procs(multiplier: Double): Int =
|
||||
(Runtime.getRuntime.availableProcessors * multiplier).ceil.toInt
|
||||
|
||||
/**
|
||||
* Default is 60000 (one minute).
|
||||
*/
|
||||
def setKeepAliveTimeInMillis(time: Long): ThreadPoolBuilder =
|
||||
setThreadPoolExecutorProperty(_.setKeepAliveTime(time, MILLISECONDS))
|
||||
|
||||
/**
|
||||
* Default ThreadPoolExecutor.CallerRunsPolicy. To allow graceful backing off when pool is overloaded.
|
||||
*/
|
||||
def setRejectionPolicy(policy: RejectedExecutionHandler): ThreadPoolBuilder =
|
||||
setThreadPoolExecutorProperty(_.setRejectedExecutionHandler(policy))
|
||||
|
||||
/**
|
||||
* Default false, set to true to conserve thread for potentially unused dispatchers
|
||||
*/
|
||||
def setAllowCoreThreadTimeout(allow: Boolean) =
|
||||
setThreadPoolExecutorProperty(_.allowCoreThreadTimeOut(allow))
|
||||
|
||||
/**
|
||||
* Default ThreadPoolExecutor.CallerRunsPolicy. To allow graceful backing off when pool is overloaded.
|
||||
*/
|
||||
protected def setThreadPoolExecutorProperty(f: (ThreadPoolExecutor) => Unit): ThreadPoolBuilder = synchronized {
|
||||
ensureNotActive
|
||||
verifyInConstructionPhase
|
||||
f(threadPoolBuilder)
|
||||
this
|
||||
}
|
||||
|
||||
|
||||
protected def verifyNotInConstructionPhase = {
|
||||
if (inProcessOfBuilding) throw new IllegalActorStateException("Is already in the process of building a thread pool")
|
||||
inProcessOfBuilding = true
|
||||
}
|
||||
|
||||
protected def verifyInConstructionPhase = {
|
||||
if (!inProcessOfBuilding) throw new IllegalActorStateException(
|
||||
"Is not in the process of building a thread pool, start building one by invoking one of the 'newThreadPool*' methods")
|
||||
}
|
||||
|
||||
def ensureNotActive(): Unit
|
||||
}
|
||||
|
||||
|
||||
/**
|
||||
* @author <a href="http://jonasboner.com">Jonas Bonér</a>
|
||||
*/
|
||||
class MonitorableThreadFactory(val name: String) extends ThreadFactory {
|
||||
protected val counter = new AtomicLong
|
||||
|
||||
def newThread(runnable: Runnable) =
|
||||
new MonitorableThread(runnable, name)
|
||||
// new Thread(runnable, name + "-" + counter.getAndIncrement)
|
||||
def newThread(runnable: Runnable) = new MonitorableThread(runnable, name)
|
||||
}
|
||||
|
||||
/**
|
||||
|
|
@ -382,10 +194,10 @@ class MonitorableThread(runnable: Runnable, name: String)
|
|||
/**
|
||||
* @author <a href="http://jonasboner.com">Jonas Bonér</a>
|
||||
*/
|
||||
class BoundedExecutorDecorator(val executor: ExecutorService, bound: Int) extends ExecutorService with Logging {
|
||||
class BoundedExecutorDecorator(val executor: ExecutorService, bound: Int) extends ExecutorServiceDelegate with Logging {
|
||||
protected val semaphore = new Semaphore(bound)
|
||||
|
||||
def execute(command: Runnable) = {
|
||||
override def execute(command: Runnable) = {
|
||||
semaphore.acquire
|
||||
try {
|
||||
executor.execute(new Runnable() {
|
||||
|
|
@ -405,8 +217,14 @@ class BoundedExecutorDecorator(val executor: ExecutorService, bound: Int) extend
|
|||
throw e
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
trait ExecutorServiceDelegate extends ExecutorService {
|
||||
|
||||
def executor: ExecutorService
|
||||
|
||||
def execute(command: Runnable) = executor.execute(command)
|
||||
|
||||
// Delegating methods for the ExecutorService interface
|
||||
def shutdown = executor.shutdown
|
||||
|
||||
def shutdownNow = executor.shutdownNow
|
||||
|
|
@ -431,3 +249,14 @@ class BoundedExecutorDecorator(val executor: ExecutorService, bound: Int) extend
|
|||
|
||||
def invokeAny[T](callables: Collection[_ <: Callable[T]], l: Long, timeUnit: TimeUnit) = executor.invokeAny(callables, l, timeUnit)
|
||||
}
|
||||
|
||||
trait LazyExecutorService extends ExecutorServiceDelegate {
|
||||
|
||||
def createExecutor: ExecutorService
|
||||
|
||||
lazy val executor = createExecutor
|
||||
}
|
||||
|
||||
class LazyExecutorServiceWrapper(executorFactory: => ExecutorService) extends LazyExecutorService {
|
||||
def createExecutor = executorFactory
|
||||
}
|
||||
Loading…
Add table
Add a link
Reference in a new issue