ViewVC Help
View File | Revision Log | Show Annotations | Download File | Root Listing
root/jsr166/jsr166/src/main/java/util/concurrent/ThreadPoolExecutor.java
Revision: 1.4
Committed: Thu May 29 13:21:28 2003 UTC (21 years ago) by dl
Branch: MAIN
CVS Tags: JSR166_PRERELEASE_0_1
Changes since 1.3: +46 -1 lines
Log Message:
Cancellation support in TPE

File Contents

# User Rev Content
1 tim 1.1 /*
2 dl 1.2 * Written by Doug Lea with assistance from members of JCP JSR-166
3     * Expert Group and released to the public domain. Use, modify, and
4     * redistribute this code in any way without acknowledgement.
5 tim 1.1 */
6    
7     package java.util.concurrent;
8    
9 dl 1.2 import java.util.*;
10 tim 1.1
11     /**
12 dl 1.2 * An {@link ExecutorService} that executes each submitted task on one
13 tim 1.1 * of several pooled threads.
14     *
15     * <p>Thread pools address two different problems at the same time:
16     * they usually provide improved performance when executing large
17     * numbers of asynchronous tasks, due to reduced per-task invocation
18     * overhead, and they provide a means of bounding and managing the
19     * resources, including threads, consumed in executing a collection of
20     * tasks.
21     *
22     * <p>This class is very configurable and can be configured to create
23     * a new thread for each task, or even to execute tasks sequentially
24     * in a single thread, in addition to its most common configuration,
25     * which reuses a pool of threads.
26     *
27     * <p>To be useful across a wide range of contexts, this class
28     * provides many adjustable parameters and extensibility hooks.
29     * However, programmers are urged to use the more convenient factory
30     * methods <tt>newCachedThreadPool</tt> (unbounded thread pool, with
31     * automatic thread reclamation), <tt>newFixedThreadPool</tt> (fixed
32 dl 1.2 * size thread pool), <tt>newSingleThreadPoolExecutor</tt> (single
33 tim 1.1 * background thread for execution of tasks), and
34     * <tt>newThreadPerTaskExeceutor</tt> (execute each task in a new
35     * thread), that preconfigure settings for the most common usage
36     * scenarios.
37     *
38     * <p>This class also maintain some basic statistics, such as the
39 dl 1.2 * number of completed tasks, that may be useful for monitoring and
40     * tuning executors.
41 tim 1.1 *
42     * <h3>Tuning guide</h3>
43     * <dl>
44 dl 1.2 *
45     * <dt>Core and maximum pool size</dt>
46     *
47     * <dd>A ThreadPoolExecutor will automatically adjust the pool size
48     * according to the bounds set by corePoolSize and maximumPoolSize.
49     * When a new task is submitted, and fewer than corePoolSize threads
50     * are running, a new thread is created to handle the request, even if
51     * other worker threads are idle. If there are more than the
52     * corePoolSize but less than maximumPoolSize threads running, a new
53     * thread will be created only if the queue is full. By setting
54     * corePoolSize and maximumPoolSize the same, you create a fixed-size
55     * thread pool.</dd>
56     *
57     * <dt>Keep-alive</dt>
58     *
59     * <dd>The keepAliveTime determines what happens to idle threads. If
60     * the pool currently has more than the core number of threads, excess
61     * threads will be terminated if they have been idle for more than the
62     * keepAliveTime.</dd>
63     *
64     * <dt>Queueing</dt>
65     *
66     * <dd>You are free to specify the queuing mechanism used to handle
67     * submitted tasks. The newCachedThreadPool factory method uses
68     * queueless synchronous channels to to hand off work to threads.
69 tim 1.1 * This is a safe, conservative policy that avoids lockups when
70     * handling sets of requests that might have internal dependencies.
71     * The newFixedThreadPool factory method uses a LinkedBlockingQueue,
72     * which will cause new tasks to be queued in cases where all
73     * MaximumPoolSize threads are busy. Queues are sometimes appropriate
74     * when each task is completely independent of others, so tasks cannot
75     * affect each others execution. For example, in an http server. When
76     * given a choice, this pool always prefers adding a new thread rather
77     * than queueing if there are currently fewer than the current
78 dl 1.2 * getCorePoolSize threads running, but otherwise always prefers
79 tim 1.1 * queuing a request rather than adding a new thread.
80     *
81     * <p>While queuing can be useful in smoothing out transient bursts of
82     * requests, especially in socket-based services, it is not very well
83     * behaved when commands continue to arrive on average faster than
84 dl 1.2 * they can be processed.
85 tim 1.1 *
86     * Queue sizes and maximum pool sizes can often be traded off for each
87     * other. Using large queues and small pools minimizes CPU usage, OS
88     * resources, and context-switching overhead, but can lead to
89     * artifically low throughput. If tasks frequently block (for example
90     * if they are I/O bound), a JVM and underlying OS may be able to
91     * schedule time for more threads than you otherwise allow. Use of
92     * small queues or queueless handoffs generally requires larger pool
93     * sizes, which keeps CPUs busier but may encounter unacceptable
94     * scheduling overhead, which also decreases throughput.
95     * </dd>
96 dl 1.2 *
97 tim 1.1 * <dt>Creating new threads</dt>
98 dl 1.2 *
99     * <dd>New threads are created using a ThreadFactory. By default,
100     * threads are created simply with the new Thread(Runnable)
101     * constructor, but by supplying a different ThreadFactory, you can
102     * alter the thread's name, thread group, priority, daemon status,
103     * etc. </dd>
104     *
105 tim 1.1 * <dt>Before and after intercepts</dt>
106 dl 1.2 *
107     * <dd>This class has overridable methods that which are called before
108     * and after execution of each task. These can be used to manipulate
109     * the execution environment (for example, reinitializing
110     * ThreadLocals), gather statistics, or perform logging. </dd>
111     *
112 tim 1.1 * <dt>Blocked execution</dt>
113 dl 1.2 *
114     * <dd>There are a number of factors which can bound the number of
115     * tasks which can execute at once, including the maximum pool size
116     * and the queuing mechanism used. If the executor determines that a
117     * task cannot be executed because it has been refused by the queue
118     * and no threads are available, or because the executor has been shut
119     * down, the RejectedExecutionHandler's rejectedExecution method is
120     * invoked. </dd>
121     *
122 tim 1.1 * <dt>Termination</dt>
123 dl 1.2 *
124     * <dd>ThreadPoolExecutor supports two shutdown options, immediate and
125     * graceful. In an immediate shutdown, any threads currently
126     * executing are interrupted, and any tasks not yet begun are returned
127     * from the shutdownNow call. In a graceful shutdown, all queued
128     * tasks are allowed to run, but new tasks may not be submitted.
129 tim 1.1 * </dd>
130 dl 1.2 *
131 tim 1.1 * </dl>
132     *
133     * @since 1.5
134 dl 1.2 * @see RejectedExecutionHandler
135 tim 1.1 * @see Executors
136     * @see ThreadFactory
137     *
138     * @spec JSR-166
139 dl 1.4 * @revised $Date: 2003/05/28 23:00:40 $
140 dl 1.3 * @editor $Author: dl $
141 tim 1.1 *
142     */
143 dl 1.2 public class ThreadPoolExecutor implements ExecutorService {
144     /**
145     * Queue used for holding tasks and handing off to worker threads.
146     */
147     private final BlockingQueue<Runnable> workQueue;
148    
149     /**
150     * Lock held on updates to poolSize, corePoolSize, maximumPoolSize, and
151     * workers set.
152     */
153     private final ReentrantLock mainLock = new ReentrantLock();
154    
155     /**
156     * Wait condition to support awaitTermination
157     */
158     private final Condition termination = mainLock.newCondition();
159    
160     /**
161     * Set containing all worker threads in pool.
162     */
163     private final Set<Worker> workers = new HashSet<Worker>();
164    
165     /**
166     * Timeout in nanosecods for idle threads waiting for work.
167     * Threads use this timeout only when there are more than
168     * corePoolSize present. Otherwise they wait forever for new work.
169     */
170     private volatile long keepAliveTime;
171    
172     /**
173     * Core pool size, updated only while holding mainLock,
174     * but volatile to allow concurrent readability even
175     * during updates.
176     */
177     private volatile int corePoolSize;
178    
179     /**
180     * Maximum pool size, updated only while holding mainLock
181     * but volatile to allow concurrent readability even
182     * during updates.
183     */
184     private volatile int maximumPoolSize;
185    
186     /**
187     * Current pool size, updated only while holding mainLock
188     * but volatile to allow concurrent readability even
189     * during updates.
190     */
191     private volatile int poolSize;
192    
193     /**
194     * Shutdown status, becomes (and remains) nonzero when shutdown called.
195     */
196     private volatile int shutdownStatus;
197    
198     // Special values for status
199     private static final int NOT_SHUTDOWN = 0;
200     private static final int SHUTDOWN_WHEN_IDLE = 1;
201     private static final int SHUTDOWN_NOW = 2;
202    
203     /**
204     * Latch that becomes true when all threads terminate after shutdown.
205     */
206     private volatile boolean isTerminated;
207    
208     /**
209     * Handler called when saturated or shutdown in execute.
210     */
211     private volatile RejectedExecutionHandler handler = defaultHandler;
212    
213     /**
214     * Factory for new threads.
215     */
216     private volatile ThreadFactory threadFactory = defaultThreadFactory;
217    
218     /**
219     * Tracks largest attained pool size.
220     */
221     private int largestPoolSize;
222    
223     /**
224     * Counter for completed tasks. Updated only on termination of
225     * worker threads.
226     */
227     private long completedTaskCount;
228    
229     private static final ThreadFactory defaultThreadFactory =
230     new ThreadFactory() {
231     public Thread newThread(Runnable r) {
232     return new Thread(r);
233     }
234     };
235    
236     private static final RejectedExecutionHandler defaultHandler =
237     new AbortPolicy();
238    
239     /**
240     * Create and return a new thread running firstTask as its first
241     * task. Call only while holding mainLock
242     */
243     private Thread addThread(Runnable firstTask) {
244     Worker w = new Worker(firstTask);
245     Thread t = threadFactory.newThread(w);
246     w.thread = t;
247     workers.add(w);
248     int nt = ++poolSize;
249     if (nt > largestPoolSize)
250     largestPoolSize = nt;
251     return t;
252     }
253    
254     /**
255     * Create and start a new thread running firstTask as its first
256     * task, only if less than corePoolSize threads are running.
257     * @return true if successful.
258     */
259 dl 1.3 boolean addIfUnderCorePoolSize(Runnable task) {
260 dl 1.2 Thread t = null;
261     mainLock.lock();
262     try {
263     if (poolSize < corePoolSize)
264     t = addThread(task);
265     }
266     finally {
267     mainLock.unlock();
268     }
269     if (t == null)
270     return false;
271     t.start();
272     return true;
273     }
274    
275     /**
276     * Create and start a new thread only if less than maximumPoolSize
277     * threads are running. The new thread runs as its first task the
278     * next task in queue, or if there is none, the given task.
279     * @return null on failure, else the first task to be run by new thread.
280     */
281     private Runnable addIfUnderMaximumPoolSize(Runnable task) {
282     Thread t = null;
283     Runnable next = null;
284     mainLock.lock();
285     try {
286     if (poolSize < maximumPoolSize) {
287     next = workQueue.poll();
288     if (next == null)
289     next = task;
290     t = addThread(next);
291     }
292     }
293     finally {
294     mainLock.unlock();
295     }
296     if (t == null)
297     return null;
298     t.start();
299     return next;
300     }
301    
302    
303     /**
304     * Get the next task for a worker thread to run.
305     */
306     private Runnable getTask() throws InterruptedException {
307     for (;;) {
308     int stat = shutdownStatus;
309     if (stat == SHUTDOWN_NOW)
310     return null;
311     long timeout = keepAliveTime;
312     if (timeout <= 0) // must die immediately for 0 timeout
313     return null;
314     if (stat == SHUTDOWN_WHEN_IDLE) // help drain queue before dying
315     return workQueue.poll();
316     if (poolSize <= corePoolSize) // untimed wait if core
317     return workQueue.take();
318     Runnable task = workQueue.poll(timeout, TimeUnit.NANOSECONDS);
319     if (task != null)
320     return task;
321     if (poolSize > corePoolSize) // timed out
322     return null;
323     // else, after timeout, pool shrank so shouldn't die, so retry
324     }
325     }
326    
327     /**
328     * Perform bookkeeping for a terminated worker thread.
329     */
330     private void workerDone(Worker w) {
331     boolean allDone = false;
332     mainLock.lock();
333     try {
334     completedTaskCount += w.completedTasks;
335     workers.remove(w);
336    
337     if (--poolSize > 0)
338     return;
339    
340     // If this was last thread, deal with potential shutdown
341     int stat = shutdownStatus;
342    
343     // If there are queued tasks but no threads, create replacement.
344     if (stat != SHUTDOWN_NOW) {
345     Runnable r = workQueue.poll();
346     if (r != null) {
347     addThread(r).start();
348     return;
349     }
350     }
351    
352     // if no tasks and not shutdown, can exit without replacement
353     if (stat == NOT_SHUTDOWN)
354     return;
355    
356     allDone = true;
357     isTerminated = true;
358     termination.signalAll();
359     }
360     finally {
361     mainLock.unlock();
362     }
363    
364     if (allDone) // call outside lock
365     terminated();
366     }
367    
368     /**
369     * Worker threads
370     */
371     private class Worker implements Runnable {
372    
373     /**
374     * The runLock is acquired and released surrounding each task
375     * execution. It mainly protects against interrupts that are
376     * intended to cancel the worker thread from instead
377     * interrupting the task being run.
378     */
379     private final ReentrantLock runLock = new ReentrantLock();
380    
381     /**
382     * Initial task to run before entering run loop
383     */
384     private Runnable firstTask;
385    
386     /**
387     * Per thread completed task counter; accumulated
388     * into completedTaskCount upon termination.
389     */
390     volatile long completedTasks;
391    
392     /**
393     * Thread this worker is running in. Acts as a final field,
394     * but cannot be set until thread is created.
395     */
396     Thread thread;
397    
398     Worker(Runnable firstTask) {
399     this.firstTask = firstTask;
400     }
401    
402     boolean isActive() {
403     return runLock.isLocked();
404     }
405    
406     /**
407     * Interrupt thread if not running a task
408     */
409     void interruptIfIdle() {
410     if (runLock.tryLock()) {
411     try {
412     thread.interrupt();
413     }
414     finally {
415     runLock.unlock();
416     }
417     }
418     }
419    
420     /**
421     * Cause thread to die even if running a task.
422     */
423     void interruptNow() {
424     thread.interrupt();
425     }
426    
427     /**
428     * Run a single task between before/after methods.
429     */
430     private void runTask(Runnable task) {
431     runLock.lock();
432     try {
433     // Abort now if immediate cancel. Otherwise, we have
434     // committed to run this task.
435     if (shutdownStatus == SHUTDOWN_NOW)
436     return;
437    
438     Thread.interrupted(); // clear interrupt status on entry
439     boolean ran = false;
440     beforeExecute(thread, task);
441     try {
442     task.run();
443     ran = true;
444     afterExecute(task, null);
445     ++completedTasks;
446     }
447     catch(RuntimeException ex) {
448     if (!ran)
449     afterExecute(task, ex);
450     // else the exception occurred within
451     // afterExecute itself in which case we don't
452     // want to call it again.
453     throw ex;
454     }
455     }
456     finally {
457     runLock.unlock();
458     }
459     }
460    
461     /**
462     * Main run loop
463     */
464     public void run() {
465     try {
466     for (;;) {
467     Runnable task;
468     if (firstTask != null) {
469     task = firstTask;
470     firstTask = null;
471     }
472     else {
473     task = getTask();
474     if (task == null)
475     break;
476     }
477     runTask(task);
478     task = null; // unnecessary but can help GC
479     }
480     }
481     catch(InterruptedException ie) {
482     // fall through
483     }
484     finally {
485     workerDone(this);
486     }
487     }
488     }
489 tim 1.1
490 dl 1.4 private static class DequeuableFutureTask<V> extends FutureTask<V> {
491     private final ThreadPoolExecutor tpe;
492     public DequeuableFutureTask(Callable<V> callable, ThreadPoolExecutor tpe) {
493     super(callable);
494     this.tpe = tpe;
495     }
496    
497     public DequeuableFutureTask(final Runnable runnable, final V result, ThreadPoolExecutor tpe) {
498     super(runnable, result);
499     this.tpe = tpe;
500     }
501    
502     public boolean cancel(boolean mayInterruptIfRunning) {
503     if (!isDone())
504     tpe.dequeue(this);
505     return super.cancel(mayInterruptIfRunning);
506     }
507     }
508    
509 tim 1.1 /**
510     * Creates a new <tt>ThreadPoolExecutor</tt> with the given initial
511     * parameters. It may be more convenient to use one of the factory
512     * methods instead of this general purpose constructor.
513     *
514 dl 1.2 * @param corePoolSize the number of threads to keep in the
515 tim 1.1 * pool, even if they are idle.
516 dl 1.2 * @param maximumPoolSize the maximum number of threads to allow in the
517 tim 1.1 * pool.
518     * @param keepAliveTime when the number of threads is greater than
519 dl 1.2 * the core, this is the maximum time that excess idle threads
520 tim 1.1 * will wait for new tasks before terminating.
521 dl 1.2 * @param unit the time unit for the keepAliveTime
522 tim 1.1 * argument.
523     * @param workQueue the queue to use for holding tasks before the
524     * are executed. This queue will hold only the <tt>Runnable</tt>
525     * tasks submitted by the <tt>execute</tt> method.
526 dl 1.2 * @throws IllegalArgumentException if corePoolSize, or
527     * keepAliveTime less than zero, or if maximumPoolSize less than or
528     * equal to zero, or if corePoolSize greater than maximumPoolSize.
529 tim 1.1 * @throws NullPointerException if <tt>workQueue</tt> is null
530     */
531 dl 1.2 public ThreadPoolExecutor(int corePoolSize,
532     int maximumPoolSize,
533 tim 1.1 long keepAliveTime,
534 dl 1.2 TimeUnit unit,
535     BlockingQueue<Runnable> workQueue) {
536     this(corePoolSize, maximumPoolSize, keepAliveTime, unit, workQueue,
537     defaultThreadFactory, defaultHandler);
538     }
539 tim 1.1
540 dl 1.2 /**
541     * Creates a new <tt>ThreadPoolExecutor</tt> with the given initial
542     * parameters.
543     *
544     * @param corePoolSize the number of threads to keep in the
545     * pool, even if they are idle.
546     * @param maximumPoolSize the maximum number of threads to allow in the
547     * pool.
548     * @param keepAliveTime when the number of threads is greater than
549     * the core, this is the maximum time that excess idle threads
550     * will wait for new tasks before terminating.
551     * @param unit the time unit for the keepAliveTime
552     * argument.
553     * @param workQueue the queue to use for holding tasks before the
554     * are executed. This queue will hold only the <tt>Runnable</tt>
555     * tasks submitted by the <tt>execute</tt> method.
556     * @param threadFactory the factory to use when the executor
557     * creates a new thread.
558     * @throws IllegalArgumentException if corePoolSize, or
559     * keepAliveTime less than zero, or if maximumPoolSize less than or
560     * equal to zero, or if corePoolSize greater than maximumPoolSize.
561     * @throws NullPointerException if <tt>workQueue</tt>
562     * or <tt>threadFactory</tt> are null.
563     */
564     public ThreadPoolExecutor(int corePoolSize,
565     int maximumPoolSize,
566     long keepAliveTime,
567     TimeUnit unit,
568     BlockingQueue<Runnable> workQueue,
569     ThreadFactory threadFactory) {
570 tim 1.1
571 dl 1.2 this(corePoolSize, maximumPoolSize, keepAliveTime, unit, workQueue,
572     threadFactory, defaultHandler);
573     }
574 tim 1.1
575 dl 1.2 /**
576     * Creates a new <tt>ThreadPoolExecutor</tt> with the given initial
577     * parameters.
578     *
579     * @param corePoolSize the number of threads to keep in the
580     * pool, even if they are idle.
581     * @param maximumPoolSize the maximum number of threads to allow in the
582     * pool.
583     * @param keepAliveTime when the number of threads is greater than
584     * the core, this is the maximum time that excess idle threads
585     * will wait for new tasks before terminating.
586     * @param unit the time unit for the keepAliveTime
587     * argument.
588     * @param workQueue the queue to use for holding tasks before the
589     * are executed. This queue will hold only the <tt>Runnable</tt>
590     * tasks submitted by the <tt>execute</tt> method.
591     * @param handler the handler to use when execution is blocked
592     * because the thread bounds and queue capacities are reached.
593     * @throws IllegalArgumentException if corePoolSize, or
594     * keepAliveTime less than zero, or if maximumPoolSize less than or
595     * equal to zero, or if corePoolSize greater than maximumPoolSize.
596     * @throws NullPointerException if <tt>workQueue</tt>
597     * or <tt>handler</tt> are null.
598     */
599     public ThreadPoolExecutor(int corePoolSize,
600     int maximumPoolSize,
601     long keepAliveTime,
602     TimeUnit unit,
603     BlockingQueue<Runnable> workQueue,
604     RejectedExecutionHandler handler) {
605     this(corePoolSize, maximumPoolSize, keepAliveTime, unit, workQueue,
606     defaultThreadFactory, handler);
607     }
608 tim 1.1
609 dl 1.2 /**
610     * Creates a new <tt>ThreadPoolExecutor</tt> with the given initial
611     * parameters.
612     *
613     * @param corePoolSize the number of threads to keep in the
614     * pool, even if they are idle.
615     * @param maximumPoolSize the maximum number of threads to allow in the
616     * pool.
617     * @param keepAliveTime when the number of threads is greater than
618     * the core, this is the maximum time that excess idle threads
619     * will wait for new tasks before terminating.
620     * @param unit the time unit for the keepAliveTime
621     * argument.
622     * @param workQueue the queue to use for holding tasks before the
623     * are executed. This queue will hold only the <tt>Runnable</tt>
624     * tasks submitted by the <tt>execute</tt> method.
625     * @param threadFactory the factory to use when the executor
626     * creates a new thread.
627     * @param handler the handler to use when execution is blocked
628     * because the thread bounds and queue capacities are reached.
629     * @throws IllegalArgumentException if corePoolSize, or
630     * keepAliveTime less than zero, or if maximumPoolSize less than or
631     * equal to zero, or if corePoolSize greater than maximumPoolSize.
632     * @throws NullPointerException if <tt>workQueue</tt>
633     * or <tt>threadFactory</tt> or <tt>handler</tt> are null.
634     */
635     public ThreadPoolExecutor(int corePoolSize,
636     int maximumPoolSize,
637     long keepAliveTime,
638     TimeUnit unit,
639     BlockingQueue<Runnable> workQueue,
640     ThreadFactory threadFactory,
641     RejectedExecutionHandler handler) {
642     if (corePoolSize < 0 ||
643     maximumPoolSize <= 0 ||
644     maximumPoolSize < corePoolSize ||
645     keepAliveTime < 0)
646     throw new IllegalArgumentException();
647     if (workQueue == null || threadFactory == null || handler == null)
648     throw new NullPointerException();
649     this.corePoolSize = corePoolSize;
650     this.maximumPoolSize = maximumPoolSize;
651     this.workQueue = workQueue;
652     this.keepAliveTime = unit.toNanos(keepAliveTime);
653     this.threadFactory = threadFactory;
654     this.handler = handler;
655 tim 1.1 }
656    
657 dl 1.2
658     /**
659     * Executes the given task sometime in the future. The task
660     * may execute in a new thread or in an existing pooled thread.
661     *
662     * If the task cannot be submitted for execution, either because this
663     * executor has been shutdown or because its capacity has been reached,
664     * the task is handled by the current <tt>RejectedExecutionHandler</tt>.
665     *
666     * @param command the task to execute
667     * @throws RejectedExecutionException at discretion of
668     * <tt>RejectedExecutionHandler</tt>, if task cannot be accepted for execution
669     */
670     public void execute(Runnable command) {
671     for (;;) {
672     if (shutdownStatus != NOT_SHUTDOWN) {
673     handler.rejectedExecution(command, this);
674     return;
675     }
676     if (poolSize < corePoolSize && addIfUnderCorePoolSize(command))
677     return;
678     if (workQueue.offer(command))
679     return;
680     Runnable r = addIfUnderMaximumPoolSize(command);
681     if (r == command)
682     return;
683     if (r == null) {
684     handler.rejectedExecution(command, this);
685     return;
686     }
687     // else retry
688     }
689 tim 1.1 }
690 dl 1.4
691     public <T> FutureTask<T> executeAsFuture(Callable<T> task) {
692     FutureTask<T> future = new DequeuableFutureTask<T>(task, this);
693     execute(future);
694     return future;
695     }
696    
697     public <T> FutureTask<T> executeAsFuture(Runnable task, T returnValue) {
698     FutureTask<T> future = new DequeuableFutureTask<T>(task, returnValue, this);
699     execute(future);
700     return future;
701     }
702    
703    
704 dl 1.2
705     public void shutdown() {
706     mainLock.lock();
707     try {
708     if (shutdownStatus == NOT_SHUTDOWN) // don't override shutdownNow
709     shutdownStatus = SHUTDOWN_WHEN_IDLE;
710 tim 1.1
711 dl 1.2 for (Iterator<Worker> it = workers.iterator(); it.hasNext(); )
712     it.next().interruptIfIdle();
713     }
714     finally {
715     mainLock.unlock();
716     }
717 tim 1.1 }
718    
719 dl 1.2 public List shutdownNow() {
720     mainLock.lock();
721     try {
722     shutdownStatus = SHUTDOWN_NOW;
723     for (Iterator<Worker> it = workers.iterator(); it.hasNext(); )
724     it.next().interruptNow();
725     }
726     finally {
727     mainLock.unlock();
728     }
729     return Arrays.asList(workQueue.toArray());
730 tim 1.1 }
731    
732 dl 1.2 public boolean isShutdown() {
733     return shutdownStatus != NOT_SHUTDOWN;
734 tim 1.1 }
735    
736 dl 1.2 public boolean isTerminated() {
737     return isTerminated;
738     }
739 tim 1.1
740 dl 1.2 public boolean awaitTermination(long timeout, TimeUnit unit)
741     throws InterruptedException {
742     mainLock.lock();
743     try {
744     return termination.await(timeout, unit);
745     }
746     finally {
747     mainLock.unlock();
748     }
749     }
750    
751     /**
752     * Sets the thread factory used to create new threads.
753     *
754     * @param threadFactory the new thread factory
755     */
756     public void setThreadFactory(ThreadFactory threadFactory) {
757     this.threadFactory = threadFactory;
758 tim 1.1 }
759    
760 dl 1.2 /**
761     * Returns the thread factory used to create new threads.
762     *
763     * @return the current thread factory
764     */
765     public ThreadFactory getThreadFactory() {
766     return threadFactory;
767 tim 1.1 }
768    
769 dl 1.2 /**
770     * Sets a new handler for unexecutable tasks.
771     *
772     * @param handler the new handler
773     */
774     public void setRejectedExecutionHandler(RejectedExecutionHandler handler) {
775     this.handler = handler;
776     }
777 tim 1.1
778 dl 1.2 /**
779     * Returns the current handler for unexecutable tasks.
780     *
781     * @return the current handler
782     */
783     public RejectedExecutionHandler getRejectedExecutionHandler() {
784     return handler;
785 tim 1.1 }
786    
787 dl 1.2 /**
788     * Returns the task queue used by this executor. Note that
789     * this queue may be in active use. Retrieveing the task queue
790     * does not prevent queued tasks from executing.
791     *
792     * @return the task queue
793     */
794     public BlockingQueue<Runnable> getQueue() {
795     return workQueue;
796 tim 1.1 }
797 dl 1.4
798     /**
799     * Removes this task from internal queue if it is present, thus
800     * causing it not to be run if it has not already started. This
801     * method may be useful as one part of a cancellation scheme.
802     *
803     * #return true if the task was removed
804     */
805     public boolean dequeue(Runnable task) {
806     return getQueue().remove(task);
807     }
808    
809 tim 1.1
810     /**
811 dl 1.2 * Sets the core number of threads. This overrides any value set
812     * in the constructor. If the new value is smaller than the
813     * current value, excess existing threads will be terminated when
814     * they next become idle.
815 tim 1.1 *
816 dl 1.2 * @param corePoolSize the new core size
817     * @throws IllegalArgumentException if <tt>corePoolSize</tt> less than zero
818 tim 1.1 */
819 dl 1.2 public void setCorePoolSize(int corePoolSize) {
820     if (corePoolSize < 0)
821     throw new IllegalArgumentException();
822     mainLock.lock();
823     try {
824     int extra = this.corePoolSize - corePoolSize;
825     this.corePoolSize = corePoolSize;
826     if (extra > 0 && poolSize > corePoolSize) {
827     Iterator<Worker> it = workers.iterator();
828     while (it.hasNext() &&
829     extra > 0 &&
830     poolSize > corePoolSize &&
831     workQueue.remainingCapacity() == 0) {
832     it.next().interruptIfIdle();
833     --extra;
834     }
835     }
836    
837     }
838     finally {
839     mainLock.unlock();
840     }
841     }
842 tim 1.1
843     /**
844 dl 1.2 * Returns the core number of threads.
845 tim 1.1 *
846 dl 1.2 * @return the core number of threads
847 tim 1.1 */
848 dl 1.2 public int getCorePoolSize() {
849     return corePoolSize;
850     }
851 tim 1.1
852     /**
853     * Sets the maximum allowed number of threads. This overrides any
854 dl 1.2 * value set in the constructor. If the new value is smaller than
855     * the current value, excess existing threads will be
856     * terminated when they next become idle.
857 tim 1.1 *
858 dl 1.2 * @param maximumPoolSize the new maximum
859     * @throws IllegalArgumentException if maximumPoolSize less than zero or
860     * the {@link #getCorePoolSize core pool size}
861     */
862     public void setMaximumPoolSize(int maximumPoolSize) {
863     if (maximumPoolSize <= 0 || maximumPoolSize < corePoolSize)
864     throw new IllegalArgumentException();
865     mainLock.lock();
866     try {
867     int extra = this.maximumPoolSize - maximumPoolSize;
868     this.maximumPoolSize = maximumPoolSize;
869     if (extra > 0 && poolSize > maximumPoolSize) {
870     Iterator<Worker> it = workers.iterator();
871     while (it.hasNext() &&
872     extra > 0 &&
873     poolSize > maximumPoolSize) {
874     it.next().interruptIfIdle();
875     --extra;
876     }
877     }
878     }
879     finally {
880     mainLock.unlock();
881     }
882     }
883 tim 1.1
884     /**
885     * Returns the maximum allowed number of threads.
886     *
887 dl 1.2 * @return the maximum allowed number of threads
888 tim 1.1 */
889 dl 1.2 public int getMaximumPoolSize() {
890     return maximumPoolSize;
891     }
892 tim 1.1
893     /**
894     * Sets the time limit for which threads may remain idle before
895 dl 1.2 * being terminated. If there are more than the core number of
896 tim 1.1 * threads currently in the pool, after waiting this amount of
897     * time without processing a task, excess threads will be
898     * terminated. This overrides any value set in the constructor.
899     * @param time the time to wait. A time value of zero will cause
900     * excess threads to terminate immediately after executing tasks.
901 dl 1.2 * @param unit the time unit of the time argument
902 tim 1.1 * @throws IllegalArgumentException if msecs less than zero
903     */
904 dl 1.2 public void setKeepAliveTime(long time, TimeUnit unit) {
905     if (time < 0)
906     throw new IllegalArgumentException();
907     this.keepAliveTime = unit.toNanos(time);
908     }
909 tim 1.1
910     /**
911     * Returns the thread keep-alive time, which is the amount of time
912 dl 1.2 * which threads in excess of the core pool size may remain
913     * idle before being terminated.
914 tim 1.1 *
915 dl 1.2 * @param unit the desired time unit of the result
916 tim 1.1 * @return the time limit
917     */
918 dl 1.2 public long getKeepAliveTime(TimeUnit unit) {
919     return unit.convert(keepAliveTime, TimeUnit.NANOSECONDS);
920     }
921 tim 1.1
922     /* Statistics */
923    
924     /**
925     * Returns the current number of threads in the pool.
926     *
927     * @return the number of threads
928     */
929 dl 1.2 public int getPoolSize() {
930     return poolSize;
931     }
932 tim 1.1
933     /**
934 dl 1.2 * Returns the approximate number of threads that are actively
935 tim 1.1 * executing tasks.
936     *
937     * @return the number of threads
938     */
939 dl 1.2 public int getActiveCount() {
940     mainLock.lock();
941     try {
942     int n = 0;
943     for (Iterator<Worker> it = workers.iterator(); it.hasNext(); ) {
944     if (it.next().isActive())
945     ++n;
946     }
947     return n;
948     }
949     finally {
950     mainLock.unlock();
951     }
952     }
953 tim 1.1
954     /**
955 dl 1.2 * Returns the largest number of threads that have ever
956     * simultaneously been in the pool.
957 tim 1.1 *
958     * @return the number of threads
959     */
960 dl 1.2 public int getLargestPoolSize() {
961     mainLock.lock();
962     try {
963     return largestPoolSize;
964     }
965     finally {
966     mainLock.unlock();
967     }
968     }
969 tim 1.1
970     /**
971 dl 1.2 * Returns the approximate total number of tasks that have been
972     * scheduled for execution. Because the states of tasks and
973     * threads may change dynamically during computation, the returned
974     * value is only an approximation.
975 tim 1.1 *
976     * @return the number of tasks
977     */
978 dl 1.2 public long getTaskCount() {
979     mainLock.lock();
980     try {
981     long n = completedTaskCount;
982     for (Iterator<Worker> it = workers.iterator(); it.hasNext(); ) {
983     Worker w = it.next();
984     n += w.completedTasks;
985     if (w.isActive())
986     ++n;
987     }
988     return n + workQueue.size();
989     }
990     finally {
991     mainLock.unlock();
992     }
993     }
994 tim 1.1
995     /**
996 dl 1.2 * Returns the approximate total number of tasks that have
997     * completed execution. Because the states of tasks and threads
998     * may change dynamically during computation, the returned value
999     * is only an approximation.
1000 tim 1.1 *
1001     * @return the number of tasks
1002     */
1003 dl 1.2 public long getCompletedTaskCount() {
1004     mainLock.lock();
1005     try {
1006     long n = completedTaskCount;
1007     for (Iterator<Worker> it = workers.iterator(); it.hasNext(); )
1008     n += it.next().completedTasks;
1009     return n;
1010     }
1011     finally {
1012     mainLock.unlock();
1013     }
1014     }
1015 tim 1.1
1016     /**
1017 dl 1.2 * Method invoked prior to executing the given Runnable in given
1018     * thread. This method may be used to re-initialize ThreadLocals,
1019     * or to perform logging.
1020 tim 1.1 *
1021 dl 1.2 * @param t the thread that will run task r.
1022     * @param r the task that will be executed.
1023 tim 1.1 */
1024 dl 1.2 protected void beforeExecute(Thread t, Runnable r) { }
1025 tim 1.1
1026     /**
1027 dl 1.2 * Method invoked upon completion of execution of the given
1028     * Runnable. If non-null, the Throwable is the uncaught exception
1029     * that caused execution to terminate abruptly.
1030 tim 1.1 *
1031 dl 1.2 * @param r the runnable that has completed.
1032     * @param t the exception that cause termination, or null if
1033     * execution completed normally.
1034 tim 1.1 */
1035 dl 1.2 protected void afterExecute(Runnable r, Throwable t) { }
1036 tim 1.1
1037 dl 1.2 /**
1038     * Method invoked when the Executor has terminated. Default
1039     * implementation does nothing.
1040     */
1041     protected void terminated() { }
1042 tim 1.1
1043     /**
1044     * A handler for unexecutable tasks that runs these tasks directly in the
1045     * calling thread of the <tt>execute</tt> method. This is the default
1046 dl 1.2 * <tt>RejectedExecutionHandler</tt>.
1047 tim 1.1 */
1048 dl 1.2 public static class CallerRunsPolicy implements RejectedExecutionHandler {
1049 tim 1.1
1050     /**
1051     * Constructs a <tt>CallerRunsPolicy</tt>.
1052     */
1053     public CallerRunsPolicy() { }
1054    
1055 dl 1.2 public void rejectedExecution(Runnable r, ThreadPoolExecutor e) {
1056     if (!e.isShutdown()) {
1057 tim 1.1 r.run();
1058     }
1059     }
1060     }
1061    
1062     /**
1063 dl 1.2 * A handler for unexecutable tasks that throws a <tt>RejectedExecutionException</tt>.
1064 tim 1.1 */
1065 dl 1.2 public static class AbortPolicy implements RejectedExecutionHandler {
1066 tim 1.1
1067     /**
1068     * Constructs a <tt>AbortPolicy</tt>.
1069     */
1070     public AbortPolicy() { }
1071    
1072 dl 1.2 public void rejectedExecution(Runnable r, ThreadPoolExecutor e) {
1073     throw new RejectedExecutionException();
1074 tim 1.1 }
1075     }
1076    
1077     /**
1078     * A handler for unexecutable tasks that waits until the task can be
1079     * submitted for execution.
1080     */
1081 dl 1.2 public static class WaitPolicy implements RejectedExecutionHandler {
1082 tim 1.1 /**
1083     * Constructs a <tt>WaitPolicy</tt>.
1084     */
1085     public WaitPolicy() { }
1086    
1087 dl 1.2 public void rejectedExecution(Runnable r, ThreadPoolExecutor e) {
1088     if (!e.isShutdown()) {
1089     try {
1090     e.getQueue().put(r);
1091     }
1092     catch (InterruptedException ie) {
1093     Thread.currentThread().interrupt();
1094     throw new RejectedExecutionException(ie);
1095     }
1096 tim 1.1 }
1097     }
1098     }
1099    
1100     /**
1101     * A handler for unexecutable tasks that silently discards these tasks.
1102     */
1103 dl 1.2 public static class DiscardPolicy implements RejectedExecutionHandler {
1104 tim 1.1
1105     /**
1106     * Constructs <tt>DiscardPolicy</tt>.
1107     */
1108     public DiscardPolicy() { }
1109    
1110 dl 1.2 public void rejectedExecution(Runnable r, ThreadPoolExecutor e) {
1111 tim 1.1 }
1112     }
1113    
1114     /**
1115     * A handler for unexecutable tasks that discards the oldest unhandled request.
1116     */
1117 dl 1.2 public static class DiscardOldestPolicy implements RejectedExecutionHandler {
1118 tim 1.1 /**
1119 dl 1.2 * Constructs a <tt>DiscardOldestPolicy</tt> for the given executor.
1120 tim 1.1 */
1121     public DiscardOldestPolicy() { }
1122    
1123 dl 1.2 public void rejectedExecution(Runnable r, ThreadPoolExecutor e) {
1124     if (!e.isShutdown()) {
1125     e.getQueue().poll();
1126     e.execute(r);
1127 tim 1.1 }
1128     }
1129     }
1130     }