ViewVC Help
View File | Revision Log | Show Annotations | Download File | Root Listing
root/jsr166/jsr166/src/main/java/util/concurrent/ThreadPoolExecutor.java
Revision: 1.21
Committed: Wed Sep 3 13:15:14 2003 UTC (20 years, 9 months ago) by dl
Branch: MAIN
Changes since 1.20: +175 -95 lines
Log Message:
Rewrite sections of class javadoc; eliminate WaitPolicy

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 dl 1.9 import java.util.concurrent.locks.*;
9 dl 1.2 import java.util.*;
10 tim 1.1
11     /**
12 dl 1.17 * An {@link ExecutorService} that executes each submitted task using
13     * one of possibly several pooled threads.
14 tim 1.1 *
15 dl 1.17 * <p>Thread pools address two different problems: they usually
16     * provide improved performance when executing large numbers of
17     * asynchronous tasks, due to reduced per-task invocation overhead,
18     * and they provide a means of bounding and managing the resources,
19     * including threads, consumed when executing a collection of tasks.
20 dl 1.20 * Each <tt>ThreadPoolExecutor</tt> also maintains some basic
21     * statistics, such as the number of completed tasks, that may be
22     * useful for monitoring and tuning.
23 dl 1.17 *
24 tim 1.1 * <p>To be useful across a wide range of contexts, this class
25 dl 1.17 * provides many adjustable parameters and extensibility hooks. For
26     * example, it can be configured to create a new thread for each task,
27     * or even to execute tasks sequentially in a single thread, in
28     * addition to its most common configuration, which reuses a pool of
29     * threads. However, programmers are urged to use the more convenient
30 dl 1.20 * {@link Executors} factory methods {@link
31     * Executors#newCachedThreadPool} (unbounded thread pool, with
32     * automatic thread reclamation), {@link Executors#newFixedThreadPool}
33     * (fixed size thread pool) and {@link
34     * Executors#newSingleThreadExecutor} (single background thread), that
35 dl 1.21 * preconfigure settings for the most common usage scenarios. Use the
36     * following guide when manually configuring and tuning
37     * <tt>ThreadPoolExecutors</tt>:
38 dl 1.17 *
39 tim 1.1 * <dl>
40 dl 1.2 *
41 dl 1.21 * <dt>Core and maximum pool sizes</dt>
42 dl 1.2 *
43 dl 1.19 * <dd>A <tt>ThreadPoolExecutor</tt> will automatically adjust the
44 dl 1.21 * pool size
45     * (see {@link ThreadPoolExecutor#getPoolSize})
46     * according to the bounds set by corePoolSize
47     * (see {@link ThreadPoolExecutor#getCorePoolSize})
48     * and
49     * maximumPoolSize
50     * (see {@link ThreadPoolExecutor#getMaximumPoolSize}).
51     * When a new task is submitted in method {@link
52     * ThreadPoolExecutor#execute}, and fewer than corePoolSize threads
53     * are running, a new thread is created to handle the request, even if
54     * other worker threads are idle. If there are more than
55     * corePoolSize but less than maximumPoolSize threads running, a new
56     * thread will be created only if the queue is full. By setting
57     * corePoolSize and maximumPoolSize the same, you create a fixed-size
58     * thread pool. By setting maximumPoolSize to an essentially unbounded
59     * value such as <tt>Integer.MAX_VALUE</tt>, you allow the pool to
60     * accomodate an arbitrary number of concurrent tasks. Most typically,
61     * core and maximum pool sizes are set only upon construction, but they
62     * may also be changed dynamically using {@link
63     * ThreadPoolExecutor#setCorePoolSize} and {@link
64     * ThreadPoolExecutor#setMaximumPoolSize}. <dd>
65 dl 1.2 *
66 dl 1.21 * <dt> On-demand construction
67 dl 1.2 *
68 dl 1.21 * <dd> By default, even core threads are initially created and
69     * started only when needed by new tasks, but this can be overridden
70     * dynamically using method {@link
71     * ThreadPoolExecutor#prestartCoreThread} or
72     * {@link ThreadPoolExecutor#prestartAllCoreThreads}. </dd>
73 dl 1.2 *
74 tim 1.1 * <dt>Creating new threads</dt>
75 dl 1.2 *
76 dl 1.19 * <dd>New threads are created using a {@link ThreadFactory}. By
77     * default, threads are created simply with the <tt>new
78     * Thread(Runnable)</tt> constructor, but by supplying a different
79     * ThreadFactory, you can alter the thread's name, thread group,
80     * priority, daemon status, etc. </dd>
81 dl 1.2 *
82 dl 1.21 * <dt>Keep-alive times</dt>
83     *
84     * <dd>If the pool currently has more than corePoolSize threads,
85     * excess threads will be terminated if they have been idle for more
86     * than the keepAliveTime (see {@link
87     * ThreadPoolExecutor#getKeepAliveTime}). This provides a means of
88     * reducing resource consumption when the pool is not being actively
89     * used. If the pool becomes more active later, new threads will be
90     * constructed. This parameter can also be changed dynamically
91     * using method {@link ThreadPoolExecutor#setKeepAliveTime}. Using
92     * a value of <tt>Long.MAX_VALUE</tt> {@link TimeUnit#NANOSECONDS}
93     * effectively disables idle threads from ever terminating prior
94     * to shut down.
95     * </dd>
96     *
97     * <dt>Queueing</dt>
98     *
99     * <dd>Any {@link BlockingQueue} may be used to transfer and hold
100     * submitted tasks. The use of this queue interacts with pool sizing:
101 dl 1.2 *
102 dl 1.21 * <ul>
103     *
104     * <li> If fewer than corePoolSize threads are running, a
105     * <tt>ThreadPoolExecutor</tt> always prefers adding a new thread
106     * rather than queueing.</li>
107     *
108     * <li> If corePoolSize or more threads are running, a
109     * <tt>ThreadPoolExecutor</tt>
110     * always prefers queuing a request rather than adding a new thread.</li>
111     *
112     * <li> If a request cannot be queued, a new thread is created unless
113     * this would exceed maximumPoolSize, in which case, the task will be
114     * rejected.</li>
115     *
116     * </ul>
117     *
118     * There are three general strategies for queuing:
119     * <ol>
120     *
121     * <li> <em> Direct handoffs.</em> A good default choice for a work
122     * queue is a {@link SynchronousQueue} that hands off tasks to threads
123     * without otherwise holding them. Here, an attempt to queue a task
124     * will fail if no threads are immediately available to run it, so a
125     * new thread will be constructed. This policy avoids lockups when
126     * handling sets of requests that might have internal dependencies.
127     * Direct handoffs generally require unbounded maximumPoolSizes to
128     * avoid rejection of new submitted tasks, which in turn admit the
129     * possibility of unbounded thread growth when commands continue to
130     * arrive on average faster than they can be processed. </li>
131     *
132     * <li><em> Unbounded queues.</em> Using an unbounded queue (for
133     * example a {@link LinkedBlockingQueue} without a predefined
134     * capacity) will cause new tasks to be queued in cases where all
135     * corePoolSize threads are busy, so no more than corePoolSize threads
136     * will be created. This may be appropriate when each task is
137     * completely independent of others, so tasks cannot affect each
138     * others execution; for example, in a web page server. While this
139     * style of queuing can be useful in smoothing out transient bursts of
140     * requests, it admits the possibility of unbounded work queue growth
141     * when commands continue to arrive on average faster than they can be
142     * processed. </li>
143     *
144     * <li><em>Bounded queues.</em> A bounded queue (for example, an
145     * {@link ArrayBlockingQueue}) helps prevent resource exhaustion when
146     * used with finite maximumPoolSizes, but can be more difficult to
147     * tune and control. Queue sizes and maximum pool sizes may be traded
148     * off for each other: Using large queues and small pools minimizes
149     * CPU usage, OS resources, and context-switching overhead, but can
150     * lead to artifically low throughput. If tasks frequently block (for
151     * example if they are I/O bound), a system may be able to schedule
152     * time for more threads than you otherwise allow. Use of small queues
153     * or queueless handoffs generally requires larger pool sizes, which
154     * keeps CPUs busier but may encounter unacceptable scheduling
155     * overhead, which also decreases throughput. </li>
156     *
157     * </ol>
158     *
159     * </dd>
160     *
161     * <dt>Rejected tasks</dt>
162     *
163     * <dd> New tasks submitted in method {@link
164     * ThreadPoolExecutor#execute} will be <em>rejected</em> when the
165     * Executor has been shut down, and also when the Executor uses finite
166     * bounds for both maximum threads and work queue capacity, and is
167     * saturated. In both cases, the <tt>execute</tt> method invokes its
168     * {@link RejectedExecutionHandler} {@link
169     * RejectedExecutionHandler#rejectedExecution} method. Four
170     * predefined handler policies are provided:
171     *
172     * <ol>
173     *
174     * <li> In the
175     * default {@link ThreadPoolExecutor.AbortPolicy}, the handler throws a
176     * runtime {@link RejectedExecutionException} upon rejection. </li>
177     *
178     * <li> In {@link
179     * ThreadPoolExecutor.CallerRunsPolicy}, the thread that invokes
180     * <tt>execute</tt> itself runs the task. This provides a simple
181     * feedback control mechanism that will slow down the rate that new
182     * tasks are submitted. </li>
183     *
184     * <li> In {@link ThreadPoolExecutor.DiscardPolicy},
185     * a task that cannot be executed is simply dropped. </li>
186     *
187     * <li>In {@link
188     * ThreadPoolExecutor.DiscardOldestPolicy}, if the executor is not
189     * shut down, the task at the head of the work queue is dropped, and
190     * then execution is retried (which can fail again, causing this to be
191     * repeated.) </li>
192     *
193     * </ol>
194     *
195     * It is possible to define and use other kinds of {@link
196     * RejectedExecutionHandler} classes. Doing so requires some care
197     * especially when policies are designed to work only under particular
198     * capacity or queueing policies. </dd>
199     *
200     * <dt>Hook methods</dt>
201     *
202     * <dd>This class has <tt>protected</tt> overridable {@link
203     * ThreadPoolExecutor#beforeExecute} and {@link
204     * ThreadPoolExecutor#afterExecute} methods that are called before and
205 dl 1.19 * after execution of each task. These can be used to manipulate the
206     * execution environment, for example, reinitializing ThreadLocals,
207 dl 1.21 * gathering statistics, or adding log entries. Additionally, method
208     * {@link ThreadPoolExecutor#terminated} can be overridden to perform
209     * any special processing that needs to be done once the Executor has
210     * fully terminated.</dd>
211 dl 1.2 *
212 dl 1.21 * <dt>Queue maintenance</dt>
213 dl 1.2 *
214 dl 1.21 * <dd> Method {@link ThreadPoolExecutor#getQueue} allows access
215     * to the work queue for purposes of monitoring and debugging.
216     * Use of this method for any other purpose is strongly discouraged.
217     * Two supplied methods, {@link ThreadPoolExecutor#remove} and
218     * {@link ThreadPoolExecutor#purge} are available to assist in
219     * storage reclamation when large numbers of not-yet-executed
220     * tasks become cancelled.</dd>
221 tim 1.1 * </dl>
222     *
223     * @since 1.5
224 dl 1.8 * @author Doug Lea
225 tim 1.1 */
226 dl 1.2 public class ThreadPoolExecutor implements ExecutorService {
227     /**
228     * Queue used for holding tasks and handing off to worker threads.
229 tim 1.10 */
230 dl 1.2 private final BlockingQueue<Runnable> workQueue;
231    
232     /**
233     * Lock held on updates to poolSize, corePoolSize, maximumPoolSize, and
234     * workers set.
235 tim 1.10 */
236 dl 1.2 private final ReentrantLock mainLock = new ReentrantLock();
237    
238     /**
239     * Wait condition to support awaitTermination
240 tim 1.10 */
241 dl 1.2 private final Condition termination = mainLock.newCondition();
242    
243     /**
244     * Set containing all worker threads in pool.
245 tim 1.10 */
246 dl 1.17 private final HashSet<Worker> workers = new HashSet<Worker>();
247 dl 1.2
248     /**
249     * Timeout in nanosecods for idle threads waiting for work.
250     * Threads use this timeout only when there are more than
251     * corePoolSize present. Otherwise they wait forever for new work.
252 tim 1.10 */
253 dl 1.2 private volatile long keepAliveTime;
254    
255     /**
256     * Core pool size, updated only while holding mainLock,
257     * but volatile to allow concurrent readability even
258     * during updates.
259 tim 1.10 */
260 dl 1.2 private volatile int corePoolSize;
261    
262     /**
263     * Maximum pool size, updated only while holding mainLock
264     * but volatile to allow concurrent readability even
265     * during updates.
266 tim 1.10 */
267 dl 1.2 private volatile int maximumPoolSize;
268    
269     /**
270     * Current pool size, updated only while holding mainLock
271     * but volatile to allow concurrent readability even
272     * during updates.
273 tim 1.10 */
274 dl 1.2 private volatile int poolSize;
275    
276     /**
277 dl 1.16 * Lifecycle state
278 tim 1.10 */
279 dl 1.16 private volatile int runState;
280 dl 1.2
281 dl 1.16 // Special values for runState
282 dl 1.8 /** Normal, not-shutdown mode */
283 dl 1.16 private static final int RUNNING = 0;
284 dl 1.8 /** Controlled shutdown mode */
285 dl 1.16 private static final int SHUTDOWN = 1;
286     /** Immediate shutdown mode */
287     private static final int STOP = 2;
288     /** Final state */
289     private static final int TERMINATED = 3;
290 dl 1.2
291     /**
292     * Handler called when saturated or shutdown in execute.
293 tim 1.10 */
294 dl 1.2 private volatile RejectedExecutionHandler handler = defaultHandler;
295    
296     /**
297     * Factory for new threads.
298 tim 1.10 */
299 dl 1.2 private volatile ThreadFactory threadFactory = defaultThreadFactory;
300    
301     /**
302     * Tracks largest attained pool size.
303 tim 1.10 */
304 dl 1.2 private int largestPoolSize;
305    
306     /**
307     * Counter for completed tasks. Updated only on termination of
308     * worker threads.
309 tim 1.10 */
310 dl 1.2 private long completedTaskCount;
311    
312 dl 1.8 /**
313 dl 1.16 * The default thread factory
314 dl 1.8 */
315 tim 1.10 private static final ThreadFactory defaultThreadFactory =
316 dl 1.2 new ThreadFactory() {
317     public Thread newThread(Runnable r) {
318     return new Thread(r);
319     }
320     };
321    
322 dl 1.8 /**
323     * The default rejectect execution handler
324     */
325 tim 1.10 private static final RejectedExecutionHandler defaultHandler =
326 dl 1.2 new AbortPolicy();
327    
328     /**
329 dl 1.17 * Invoke the rejected execution handler for the given command.
330 dl 1.13 */
331     void reject(Runnable command) {
332     handler.rejectedExecution(command, this);
333     }
334    
335     /**
336 dl 1.2 * Create and return a new thread running firstTask as its first
337     * task. Call only while holding mainLock
338 dl 1.8 * @param firstTask the task the new thread should run first (or
339     * null if none)
340     * @return the new thread
341 dl 1.2 */
342     private Thread addThread(Runnable firstTask) {
343     Worker w = new Worker(firstTask);
344     Thread t = threadFactory.newThread(w);
345     w.thread = t;
346     workers.add(w);
347     int nt = ++poolSize;
348     if (nt > largestPoolSize)
349     largestPoolSize = nt;
350     return t;
351     }
352    
353 dl 1.16
354 dl 1.15
355 dl 1.2 /**
356     * Create and start a new thread running firstTask as its first
357     * task, only if less than corePoolSize threads are running.
358 dl 1.8 * @param firstTask the task the new thread should run first (or
359     * null if none)
360 dl 1.2 * @return true if successful.
361     */
362 dl 1.16 private boolean addIfUnderCorePoolSize(Runnable firstTask) {
363 dl 1.2 Thread t = null;
364     mainLock.lock();
365     try {
366 tim 1.10 if (poolSize < corePoolSize)
367 dl 1.8 t = addThread(firstTask);
368 tim 1.14 } finally {
369 dl 1.2 mainLock.unlock();
370     }
371     if (t == null)
372     return false;
373     t.start();
374     return true;
375     }
376    
377     /**
378     * Create and start a new thread only if less than maximumPoolSize
379     * threads are running. The new thread runs as its first task the
380     * next task in queue, or if there is none, the given task.
381 dl 1.8 * @param firstTask the task the new thread should run first (or
382     * null if none)
383 dl 1.2 * @return null on failure, else the first task to be run by new thread.
384     */
385 dl 1.8 private Runnable addIfUnderMaximumPoolSize(Runnable firstTask) {
386 dl 1.2 Thread t = null;
387     Runnable next = null;
388     mainLock.lock();
389     try {
390     if (poolSize < maximumPoolSize) {
391     next = workQueue.poll();
392     if (next == null)
393 dl 1.8 next = firstTask;
394 dl 1.2 t = addThread(next);
395     }
396 tim 1.14 } finally {
397 dl 1.2 mainLock.unlock();
398     }
399     if (t == null)
400     return null;
401     t.start();
402     return next;
403     }
404    
405    
406     /**
407     * Get the next task for a worker thread to run.
408 dl 1.8 * @return the task
409     * @throws InterruptedException if interrupted while waiting for task
410 dl 1.2 */
411     private Runnable getTask() throws InterruptedException {
412     for (;;) {
413 dl 1.16 switch(runState) {
414     case RUNNING: {
415     if (poolSize <= corePoolSize) // untimed wait if core
416     return workQueue.take();
417    
418     long timeout = keepAliveTime;
419     if (timeout <= 0) // die immediately for 0 timeout
420     return null;
421     Runnable r = workQueue.poll(timeout, TimeUnit.NANOSECONDS);
422     if (r != null)
423     return r;
424     if (poolSize > corePoolSize) // timed out
425     return null;
426     // else, after timeout, pool shrank so shouldn't die, so retry
427     break;
428     }
429    
430     case SHUTDOWN: {
431     // Help drain queue
432     Runnable r = workQueue.poll();
433     if (r != null)
434     return r;
435    
436     // Check if can terminate
437     if (workQueue.isEmpty()) {
438     interruptIdleWorkers();
439     return null;
440     }
441    
442     // There could still be delayed tasks in queue.
443     // Wait for one, re-checking state upon interruption
444     try {
445     return workQueue.take();
446     }
447     catch(InterruptedException ignore) {
448     }
449     break;
450     }
451    
452     case STOP:
453 dl 1.2 return null;
454 dl 1.16 default:
455     assert false;
456     }
457     }
458     }
459    
460     /**
461     * Wake up all threads that might be waiting for tasks.
462     */
463     void interruptIdleWorkers() {
464     mainLock.lock();
465     try {
466     for (Iterator<Worker> it = workers.iterator(); it.hasNext(); )
467     it.next().interruptIfIdle();
468     } finally {
469     mainLock.unlock();
470 dl 1.2 }
471     }
472    
473     /**
474     * Perform bookkeeping for a terminated worker thread.
475 tim 1.10 * @param w the worker
476 dl 1.2 */
477     private void workerDone(Worker w) {
478     mainLock.lock();
479     try {
480     completedTaskCount += w.completedTasks;
481     workers.remove(w);
482 tim 1.10 if (--poolSize > 0)
483 dl 1.2 return;
484    
485 dl 1.16 // Else, this is the last thread. Deal with potential shutdown.
486    
487     int state = runState;
488     assert state != TERMINATED;
489 tim 1.10
490 dl 1.16 if (state != STOP) {
491     // If there are queued tasks but no threads, create
492     // replacement.
493 dl 1.2 Runnable r = workQueue.poll();
494     if (r != null) {
495     addThread(r).start();
496     return;
497     }
498 dl 1.16
499     // If there are some (presumably delayed) tasks but
500     // none pollable, create an idle replacement to wait.
501     if (!workQueue.isEmpty()) {
502     addThread(null).start();
503     return;
504     }
505    
506     // Otherwise, we can exit without replacement
507     if (state == RUNNING)
508     return;
509 dl 1.2 }
510    
511 dl 1.16 // Either state is STOP, or state is SHUTDOWN and there is
512     // no work to do. So we can terminate.
513     runState = TERMINATED;
514 dl 1.2 termination.signalAll();
515 dl 1.16 // fall through to call terminate() outside of lock.
516 tim 1.14 } finally {
517 dl 1.2 mainLock.unlock();
518     }
519    
520 dl 1.16 assert runState == TERMINATED;
521     terminated();
522 dl 1.2 }
523    
524     /**
525 tim 1.10 * Worker threads
526 dl 1.2 */
527     private class Worker implements Runnable {
528    
529     /**
530     * The runLock is acquired and released surrounding each task
531     * execution. It mainly protects against interrupts that are
532     * intended to cancel the worker thread from instead
533     * interrupting the task being run.
534     */
535     private final ReentrantLock runLock = new ReentrantLock();
536    
537     /**
538     * Initial task to run before entering run loop
539     */
540     private Runnable firstTask;
541    
542     /**
543     * Per thread completed task counter; accumulated
544     * into completedTaskCount upon termination.
545     */
546     volatile long completedTasks;
547    
548     /**
549     * Thread this worker is running in. Acts as a final field,
550     * but cannot be set until thread is created.
551     */
552     Thread thread;
553    
554     Worker(Runnable firstTask) {
555     this.firstTask = firstTask;
556     }
557    
558     boolean isActive() {
559     return runLock.isLocked();
560     }
561    
562     /**
563     * Interrupt thread if not running a task
564 tim 1.10 */
565 dl 1.2 void interruptIfIdle() {
566     if (runLock.tryLock()) {
567     try {
568     thread.interrupt();
569 tim 1.14 } finally {
570 dl 1.2 runLock.unlock();
571     }
572     }
573     }
574    
575     /**
576     * Cause thread to die even if running a task.
577 tim 1.10 */
578 dl 1.2 void interruptNow() {
579     thread.interrupt();
580     }
581    
582     /**
583     * Run a single task between before/after methods.
584     */
585     private void runTask(Runnable task) {
586     runLock.lock();
587     try {
588     // Abort now if immediate cancel. Otherwise, we have
589     // committed to run this task.
590 dl 1.16 if (runState == STOP)
591 dl 1.2 return;
592    
593     Thread.interrupted(); // clear interrupt status on entry
594     boolean ran = false;
595     beforeExecute(thread, task);
596     try {
597     task.run();
598     ran = true;
599     afterExecute(task, null);
600     ++completedTasks;
601 tim 1.14 } catch(RuntimeException ex) {
602 dl 1.2 if (!ran)
603     afterExecute(task, ex);
604 dl 1.17 // Else the exception occurred within
605 dl 1.2 // afterExecute itself in which case we don't
606     // want to call it again.
607     throw ex;
608     }
609 tim 1.14 } finally {
610 dl 1.2 runLock.unlock();
611     }
612     }
613    
614     /**
615     * Main run loop
616     */
617     public void run() {
618     try {
619     for (;;) {
620     Runnable task;
621     if (firstTask != null) {
622     task = firstTask;
623     firstTask = null;
624 tim 1.14 } else {
625 dl 1.2 task = getTask();
626     if (task == null)
627     break;
628     }
629     runTask(task);
630     task = null; // unnecessary but can help GC
631     }
632 tim 1.14 } catch(InterruptedException ie) {
633 dl 1.2 // fall through
634 tim 1.14 } finally {
635 dl 1.2 workerDone(this);
636     }
637     }
638     }
639 tim 1.1
640 dl 1.17 // Public methods
641    
642 tim 1.1 /**
643 dl 1.17 * Creates a new <tt>ThreadPoolExecutor</tt> with the given
644     * initial parameters. It may be more convenient to use one of
645 dholmes 1.18 * the {@link Executors} factory methods instead of this general
646 dl 1.17 * purpose constructor.
647 tim 1.1 *
648 dl 1.2 * @param corePoolSize the number of threads to keep in the
649 tim 1.1 * pool, even if they are idle.
650 dl 1.2 * @param maximumPoolSize the maximum number of threads to allow in the
651 tim 1.1 * pool.
652     * @param keepAliveTime when the number of threads is greater than
653 dl 1.2 * the core, this is the maximum time that excess idle threads
654 tim 1.1 * will wait for new tasks before terminating.
655 dl 1.2 * @param unit the time unit for the keepAliveTime
656 tim 1.1 * argument.
657     * @param workQueue the queue to use for holding tasks before the
658     * are executed. This queue will hold only the <tt>Runnable</tt>
659     * tasks submitted by the <tt>execute</tt> method.
660 dl 1.2 * @throws IllegalArgumentException if corePoolSize, or
661     * keepAliveTime less than zero, or if maximumPoolSize less than or
662     * equal to zero, or if corePoolSize greater than maximumPoolSize.
663 tim 1.1 * @throws NullPointerException if <tt>workQueue</tt> is null
664     */
665 dl 1.2 public ThreadPoolExecutor(int corePoolSize,
666     int maximumPoolSize,
667 tim 1.1 long keepAliveTime,
668 dl 1.2 TimeUnit unit,
669     BlockingQueue<Runnable> workQueue) {
670 tim 1.10 this(corePoolSize, maximumPoolSize, keepAliveTime, unit, workQueue,
671 dl 1.2 defaultThreadFactory, defaultHandler);
672     }
673 tim 1.1
674 dl 1.2 /**
675     * Creates a new <tt>ThreadPoolExecutor</tt> with the given initial
676     * parameters.
677     *
678     * @param corePoolSize the number of threads to keep in the
679     * pool, even if they are idle.
680     * @param maximumPoolSize the maximum number of threads to allow in the
681     * pool.
682     * @param keepAliveTime when the number of threads is greater than
683     * the core, this is the maximum time that excess idle threads
684     * will wait for new tasks before terminating.
685     * @param unit the time unit for the keepAliveTime
686     * argument.
687     * @param workQueue the queue to use for holding tasks before the
688     * are executed. This queue will hold only the <tt>Runnable</tt>
689     * tasks submitted by the <tt>execute</tt> method.
690     * @param threadFactory the factory to use when the executor
691 tim 1.10 * creates a new thread.
692 dl 1.2 * @throws IllegalArgumentException if corePoolSize, or
693     * keepAliveTime less than zero, or if maximumPoolSize less than or
694     * equal to zero, or if corePoolSize greater than maximumPoolSize.
695 tim 1.10 * @throws NullPointerException if <tt>workQueue</tt>
696 dl 1.2 * or <tt>threadFactory</tt> are null.
697     */
698     public ThreadPoolExecutor(int corePoolSize,
699     int maximumPoolSize,
700     long keepAliveTime,
701     TimeUnit unit,
702     BlockingQueue<Runnable> workQueue,
703     ThreadFactory threadFactory) {
704 tim 1.1
705 tim 1.10 this(corePoolSize, maximumPoolSize, keepAliveTime, unit, workQueue,
706 dl 1.2 threadFactory, defaultHandler);
707     }
708 tim 1.1
709 dl 1.2 /**
710     * Creates a new <tt>ThreadPoolExecutor</tt> with the given initial
711     * parameters.
712     *
713     * @param corePoolSize the number of threads to keep in the
714     * pool, even if they are idle.
715     * @param maximumPoolSize the maximum number of threads to allow in the
716     * pool.
717     * @param keepAliveTime when the number of threads is greater than
718     * the core, this is the maximum time that excess idle threads
719     * will wait for new tasks before terminating.
720     * @param unit the time unit for the keepAliveTime
721     * argument.
722     * @param workQueue the queue to use for holding tasks before the
723     * are executed. This queue will hold only the <tt>Runnable</tt>
724     * tasks submitted by the <tt>execute</tt> method.
725     * @param handler the handler to use when execution is blocked
726     * because the thread bounds and queue capacities are reached.
727     * @throws IllegalArgumentException if corePoolSize, or
728     * keepAliveTime less than zero, or if maximumPoolSize less than or
729     * equal to zero, or if corePoolSize greater than maximumPoolSize.
730 tim 1.10 * @throws NullPointerException if <tt>workQueue</tt>
731 dl 1.2 * or <tt>handler</tt> are null.
732     */
733     public ThreadPoolExecutor(int corePoolSize,
734     int maximumPoolSize,
735     long keepAliveTime,
736     TimeUnit unit,
737     BlockingQueue<Runnable> workQueue,
738     RejectedExecutionHandler handler) {
739 tim 1.10 this(corePoolSize, maximumPoolSize, keepAliveTime, unit, workQueue,
740 dl 1.2 defaultThreadFactory, handler);
741     }
742 tim 1.1
743 dl 1.2 /**
744     * Creates a new <tt>ThreadPoolExecutor</tt> with the given initial
745     * parameters.
746     *
747     * @param corePoolSize the number of threads to keep in the
748     * pool, even if they are idle.
749     * @param maximumPoolSize the maximum number of threads to allow in the
750     * pool.
751     * @param keepAliveTime when the number of threads is greater than
752     * the core, this is the maximum time that excess idle threads
753     * will wait for new tasks before terminating.
754     * @param unit the time unit for the keepAliveTime
755     * argument.
756     * @param workQueue the queue to use for holding tasks before the
757     * are executed. This queue will hold only the <tt>Runnable</tt>
758     * tasks submitted by the <tt>execute</tt> method.
759     * @param threadFactory the factory to use when the executor
760 tim 1.10 * creates a new thread.
761 dl 1.2 * @param handler the handler to use when execution is blocked
762     * because the thread bounds and queue capacities are reached.
763     * @throws IllegalArgumentException if corePoolSize, or
764     * keepAliveTime less than zero, or if maximumPoolSize less than or
765     * equal to zero, or if corePoolSize greater than maximumPoolSize.
766 tim 1.10 * @throws NullPointerException if <tt>workQueue</tt>
767 dl 1.2 * or <tt>threadFactory</tt> or <tt>handler</tt> are null.
768     */
769     public ThreadPoolExecutor(int corePoolSize,
770     int maximumPoolSize,
771     long keepAliveTime,
772     TimeUnit unit,
773     BlockingQueue<Runnable> workQueue,
774     ThreadFactory threadFactory,
775     RejectedExecutionHandler handler) {
776 tim 1.10 if (corePoolSize < 0 ||
777 dl 1.2 maximumPoolSize <= 0 ||
778 tim 1.10 maximumPoolSize < corePoolSize ||
779 dl 1.2 keepAliveTime < 0)
780     throw new IllegalArgumentException();
781     if (workQueue == null || threadFactory == null || handler == null)
782     throw new NullPointerException();
783     this.corePoolSize = corePoolSize;
784     this.maximumPoolSize = maximumPoolSize;
785     this.workQueue = workQueue;
786     this.keepAliveTime = unit.toNanos(keepAliveTime);
787     this.threadFactory = threadFactory;
788     this.handler = handler;
789 tim 1.1 }
790    
791 dl 1.2
792     /**
793     * Executes the given task sometime in the future. The task
794     * may execute in a new thread or in an existing pooled thread.
795     *
796     * If the task cannot be submitted for execution, either because this
797     * executor has been shutdown or because its capacity has been reached,
798 tim 1.10 * the task is handled by the current <tt>RejectedExecutionHandler</tt>.
799 dl 1.2 *
800     * @param command the task to execute
801     * @throws RejectedExecutionException at discretion of
802 dl 1.8 * <tt>RejectedExecutionHandler</tt>, if task cannot be accepted
803     * for execution
804 dl 1.2 */
805 tim 1.10 public void execute(Runnable command) {
806 dl 1.2 for (;;) {
807 dl 1.16 if (runState != RUNNING) {
808 dl 1.13 reject(command);
809 dl 1.2 return;
810     }
811     if (poolSize < corePoolSize && addIfUnderCorePoolSize(command))
812     return;
813     if (workQueue.offer(command))
814     return;
815     Runnable r = addIfUnderMaximumPoolSize(command);
816     if (r == command)
817     return;
818     if (r == null) {
819 dl 1.13 reject(command);
820 dl 1.2 return;
821     }
822     // else retry
823     }
824 tim 1.1 }
825 dl 1.4
826 dl 1.2 public void shutdown() {
827     mainLock.lock();
828     try {
829 dl 1.16 if (runState == RUNNING) // don't override shutdownNow
830     runState = SHUTDOWN;
831 dl 1.2 for (Iterator<Worker> it = workers.iterator(); it.hasNext(); )
832     it.next().interruptIfIdle();
833 tim 1.14 } finally {
834 dl 1.2 mainLock.unlock();
835     }
836 tim 1.1 }
837    
838 dl 1.16
839 dl 1.2 public List shutdownNow() {
840     mainLock.lock();
841     try {
842 dl 1.16 if (runState != TERMINATED)
843     runState = STOP;
844 dl 1.2 for (Iterator<Worker> it = workers.iterator(); it.hasNext(); )
845     it.next().interruptNow();
846 tim 1.14 } finally {
847 dl 1.2 mainLock.unlock();
848     }
849     return Arrays.asList(workQueue.toArray());
850 tim 1.1 }
851    
852 dl 1.2 public boolean isShutdown() {
853 dl 1.16 return runState != RUNNING;
854     }
855    
856     /**
857     * Return true if this executor is in the process of terminating
858     * after <tt>shutdown</tt> or <tt>shutdownNow</tt> but has not
859     * completely terminated. This method may be useful for
860     * debugging. A return of <tt>true</tt> reported a sufficient
861     * period after shutdown may indicate that submitted tasks have
862     * ignored or suppressed interruption, causing this executor not
863     * to properly terminate.
864     * @return true if terminating but not yet terminated.
865     */
866     public boolean isTerminating() {
867     return runState == STOP;
868 tim 1.1 }
869    
870 dl 1.2 public boolean isTerminated() {
871 dl 1.16 return runState == TERMINATED;
872 dl 1.2 }
873 tim 1.1
874 dl 1.2 public boolean awaitTermination(long timeout, TimeUnit unit)
875     throws InterruptedException {
876     mainLock.lock();
877     try {
878     return termination.await(timeout, unit);
879 tim 1.14 } finally {
880 dl 1.2 mainLock.unlock();
881     }
882 dl 1.15 }
883    
884     /**
885     * Invokes <tt>shutdown</tt> when this executor is no longer
886     * referenced.
887     */
888     protected void finalize() {
889     shutdown();
890 dl 1.2 }
891 tim 1.10
892 dl 1.2 /**
893     * Sets the thread factory used to create new threads.
894     *
895     * @param threadFactory the new thread factory
896 tim 1.11 * @see #getThreadFactory
897 dl 1.2 */
898     public void setThreadFactory(ThreadFactory threadFactory) {
899     this.threadFactory = threadFactory;
900 tim 1.1 }
901    
902 dl 1.2 /**
903     * Returns the thread factory used to create new threads.
904     *
905     * @return the current thread factory
906 tim 1.11 * @see #setThreadFactory
907 dl 1.2 */
908     public ThreadFactory getThreadFactory() {
909     return threadFactory;
910 tim 1.1 }
911    
912 dl 1.2 /**
913     * Sets a new handler for unexecutable tasks.
914     *
915     * @param handler the new handler
916 tim 1.11 * @see #getRejectedExecutionHandler
917 dl 1.2 */
918     public void setRejectedExecutionHandler(RejectedExecutionHandler handler) {
919     this.handler = handler;
920     }
921 tim 1.1
922 dl 1.2 /**
923     * Returns the current handler for unexecutable tasks.
924     *
925     * @return the current handler
926 tim 1.11 * @see #setRejectedExecutionHandler
927 dl 1.2 */
928     public RejectedExecutionHandler getRejectedExecutionHandler() {
929     return handler;
930 tim 1.1 }
931    
932 dl 1.2 /**
933 dl 1.17 * Returns the task queue used by this executor. Access to the
934     * task queue is intended primarily for debugging and monitoring.
935     * This queue may be in active use. Retrieveing the task queue
936 dl 1.2 * does not prevent queued tasks from executing.
937     *
938     * @return the task queue
939     */
940     public BlockingQueue<Runnable> getQueue() {
941     return workQueue;
942 tim 1.1 }
943 dl 1.4
944     /**
945     * Removes this task from internal queue if it is present, thus
946     * causing it not to be run if it has not already started. This
947     * method may be useful as one part of a cancellation scheme.
948 tim 1.10 *
949 dl 1.8 * @param task the task to remove
950     * @return true if the task was removed
951 dl 1.4 */
952 dl 1.5 public boolean remove(Runnable task) {
953 dl 1.4 return getQueue().remove(task);
954     }
955    
956 dl 1.7
957     /**
958 dl 1.16 * Tries to remove from the work queue all {@link Cancellable}
959     * tasks that have been cancelled. This method can be useful as a
960     * storage reclamation operation, that has no other impact on
961     * functionality. Cancelled tasks are never executed, but may
962     * accumulate in work queues until worker threads can actively
963     * remove them. Invoking this method instead tries to remove them now.
964     * However, this method may fail to remove all such tasks in
965     * the presence of interference by other threads.
966 dl 1.7 */
967    
968     public void purge() {
969 dl 1.16 // Fail if we encounter interference during traversal
970     try {
971     Iterator<Runnable> it = getQueue().iterator();
972     while (it.hasNext()) {
973     Runnable r = it.next();
974     if (r instanceof Cancellable) {
975     Cancellable c = (Cancellable)r;
976     if (c.isCancelled())
977     it.remove();
978     }
979 dl 1.7 }
980     }
981 dl 1.16 catch(ConcurrentModificationException ex) {
982     return;
983     }
984 dl 1.7 }
985 tim 1.1
986     /**
987 dl 1.2 * Sets the core number of threads. This overrides any value set
988     * in the constructor. If the new value is smaller than the
989     * current value, excess existing threads will be terminated when
990     * they next become idle.
991 tim 1.1 *
992 dl 1.2 * @param corePoolSize the new core size
993 tim 1.10 * @throws IllegalArgumentException if <tt>corePoolSize</tt>
994 dl 1.8 * less than zero
995 tim 1.11 * @see #getCorePoolSize
996 tim 1.1 */
997 dl 1.2 public void setCorePoolSize(int corePoolSize) {
998     if (corePoolSize < 0)
999     throw new IllegalArgumentException();
1000     mainLock.lock();
1001     try {
1002     int extra = this.corePoolSize - corePoolSize;
1003     this.corePoolSize = corePoolSize;
1004     if (extra > 0 && poolSize > corePoolSize) {
1005     Iterator<Worker> it = workers.iterator();
1006 tim 1.10 while (it.hasNext() &&
1007     extra > 0 &&
1008 dl 1.2 poolSize > corePoolSize &&
1009     workQueue.remainingCapacity() == 0) {
1010     it.next().interruptIfIdle();
1011     --extra;
1012     }
1013     }
1014 tim 1.10
1015 tim 1.14 } finally {
1016 dl 1.2 mainLock.unlock();
1017     }
1018     }
1019 tim 1.1
1020     /**
1021 dl 1.2 * Returns the core number of threads.
1022 tim 1.1 *
1023 dl 1.2 * @return the core number of threads
1024 tim 1.11 * @see #setCorePoolSize
1025 tim 1.1 */
1026 tim 1.10 public int getCorePoolSize() {
1027 dl 1.2 return corePoolSize;
1028 dl 1.16 }
1029    
1030     /**
1031     * Start a core thread, causing it to idly wait for work. This
1032     * overrides the default policy of starting core threads only when
1033     * new tasks are executed. This method will return <tt>false</tt>
1034     * if all core threads have already been started.
1035     * @return true if a thread was started
1036     */
1037     public boolean prestartCoreThread() {
1038     return addIfUnderCorePoolSize(null);
1039     }
1040    
1041     /**
1042     * Start all core threads, causing them to idly wait for work. This
1043     * overrides the default policy of starting core threads only when
1044     * new tasks are executed.
1045     * @return the number of threads started.
1046     */
1047     public int prestartAllCoreThreads() {
1048     int n = 0;
1049     while (addIfUnderCorePoolSize(null))
1050     ++n;
1051     return n;
1052 dl 1.2 }
1053 tim 1.1
1054     /**
1055     * Sets the maximum allowed number of threads. This overrides any
1056 dl 1.2 * value set in the constructor. If the new value is smaller than
1057     * the current value, excess existing threads will be
1058     * terminated when they next become idle.
1059 tim 1.1 *
1060 dl 1.2 * @param maximumPoolSize the new maximum
1061     * @throws IllegalArgumentException if maximumPoolSize less than zero or
1062     * the {@link #getCorePoolSize core pool size}
1063 tim 1.11 * @see #getMaximumPoolSize
1064 dl 1.2 */
1065     public void setMaximumPoolSize(int maximumPoolSize) {
1066     if (maximumPoolSize <= 0 || maximumPoolSize < corePoolSize)
1067     throw new IllegalArgumentException();
1068     mainLock.lock();
1069     try {
1070     int extra = this.maximumPoolSize - maximumPoolSize;
1071     this.maximumPoolSize = maximumPoolSize;
1072     if (extra > 0 && poolSize > maximumPoolSize) {
1073     Iterator<Worker> it = workers.iterator();
1074 tim 1.10 while (it.hasNext() &&
1075     extra > 0 &&
1076 dl 1.2 poolSize > maximumPoolSize) {
1077     it.next().interruptIfIdle();
1078     --extra;
1079     }
1080     }
1081 tim 1.14 } finally {
1082 dl 1.2 mainLock.unlock();
1083     }
1084     }
1085 tim 1.1
1086     /**
1087     * Returns the maximum allowed number of threads.
1088     *
1089 dl 1.2 * @return the maximum allowed number of threads
1090 tim 1.11 * @see #setMaximumPoolSize
1091 tim 1.1 */
1092 tim 1.10 public int getMaximumPoolSize() {
1093 dl 1.2 return maximumPoolSize;
1094     }
1095 tim 1.1
1096     /**
1097     * Sets the time limit for which threads may remain idle before
1098 dl 1.2 * being terminated. If there are more than the core number of
1099 tim 1.1 * threads currently in the pool, after waiting this amount of
1100     * time without processing a task, excess threads will be
1101     * terminated. This overrides any value set in the constructor.
1102     * @param time the time to wait. A time value of zero will cause
1103     * excess threads to terminate immediately after executing tasks.
1104 dl 1.2 * @param unit the time unit of the time argument
1105 dl 1.17 * @throws IllegalArgumentException if time less than zero
1106 tim 1.11 * @see #getKeepAliveTime
1107 tim 1.1 */
1108 dl 1.2 public void setKeepAliveTime(long time, TimeUnit unit) {
1109     if (time < 0)
1110     throw new IllegalArgumentException();
1111     this.keepAliveTime = unit.toNanos(time);
1112     }
1113 tim 1.1
1114     /**
1115     * Returns the thread keep-alive time, which is the amount of time
1116 dl 1.2 * which threads in excess of the core pool size may remain
1117 tim 1.10 * idle before being terminated.
1118 tim 1.1 *
1119 dl 1.2 * @param unit the desired time unit of the result
1120 tim 1.1 * @return the time limit
1121 tim 1.11 * @see #setKeepAliveTime
1122 tim 1.1 */
1123 tim 1.10 public long getKeepAliveTime(TimeUnit unit) {
1124 dl 1.2 return unit.convert(keepAliveTime, TimeUnit.NANOSECONDS);
1125     }
1126 tim 1.1
1127     /* Statistics */
1128    
1129     /**
1130     * Returns the current number of threads in the pool.
1131     *
1132     * @return the number of threads
1133     */
1134 tim 1.10 public int getPoolSize() {
1135 dl 1.2 return poolSize;
1136     }
1137 tim 1.1
1138     /**
1139 dl 1.2 * Returns the approximate number of threads that are actively
1140 tim 1.1 * executing tasks.
1141     *
1142     * @return the number of threads
1143     */
1144 tim 1.10 public int getActiveCount() {
1145 dl 1.2 mainLock.lock();
1146     try {
1147     int n = 0;
1148     for (Iterator<Worker> it = workers.iterator(); it.hasNext(); ) {
1149     if (it.next().isActive())
1150     ++n;
1151     }
1152     return n;
1153 tim 1.14 } finally {
1154 dl 1.2 mainLock.unlock();
1155     }
1156     }
1157 tim 1.1
1158     /**
1159 dl 1.2 * Returns the largest number of threads that have ever
1160     * simultaneously been in the pool.
1161 tim 1.1 *
1162     * @return the number of threads
1163     */
1164 tim 1.10 public int getLargestPoolSize() {
1165 dl 1.2 mainLock.lock();
1166     try {
1167     return largestPoolSize;
1168 tim 1.14 } finally {
1169 dl 1.2 mainLock.unlock();
1170     }
1171     }
1172 tim 1.1
1173     /**
1174 dl 1.2 * Returns the approximate total number of tasks that have been
1175     * scheduled for execution. Because the states of tasks and
1176     * threads may change dynamically during computation, the returned
1177 dl 1.17 * value is only an approximation, but one that does not ever
1178     * decrease across successive calls.
1179 tim 1.1 *
1180     * @return the number of tasks
1181     */
1182 tim 1.10 public long getTaskCount() {
1183 dl 1.2 mainLock.lock();
1184     try {
1185     long n = completedTaskCount;
1186     for (Iterator<Worker> it = workers.iterator(); it.hasNext(); ) {
1187     Worker w = it.next();
1188     n += w.completedTasks;
1189     if (w.isActive())
1190     ++n;
1191     }
1192     return n + workQueue.size();
1193 tim 1.14 } finally {
1194 dl 1.2 mainLock.unlock();
1195     }
1196     }
1197 tim 1.1
1198     /**
1199 dl 1.2 * Returns the approximate total number of tasks that have
1200     * completed execution. Because the states of tasks and threads
1201     * may change dynamically during computation, the returned value
1202 dl 1.17 * is only an approximation, but one that does not ever decrease
1203     * across successive calls.
1204 tim 1.1 *
1205     * @return the number of tasks
1206     */
1207 tim 1.10 public long getCompletedTaskCount() {
1208 dl 1.2 mainLock.lock();
1209     try {
1210     long n = completedTaskCount;
1211 tim 1.10 for (Iterator<Worker> it = workers.iterator(); it.hasNext(); )
1212 dl 1.2 n += it.next().completedTasks;
1213     return n;
1214 tim 1.14 } finally {
1215 dl 1.2 mainLock.unlock();
1216     }
1217     }
1218 tim 1.1
1219     /**
1220 dl 1.17 * Method invoked prior to executing the given Runnable in the
1221     * given thread. This method may be used to re-initialize
1222     * ThreadLocals, or to perform logging. Note: To properly nest
1223     * multiple overridings, subclasses should generally invoke
1224 dl 1.5 * <tt>super.beforeExecute</tt> at the end of this method.
1225 tim 1.1 *
1226 dl 1.2 * @param t the thread that will run task r.
1227     * @param r the task that will be executed.
1228 tim 1.1 */
1229 dl 1.2 protected void beforeExecute(Thread t, Runnable r) { }
1230 tim 1.1
1231     /**
1232 dl 1.2 * Method invoked upon completion of execution of the given
1233     * Runnable. If non-null, the Throwable is the uncaught exception
1234 dl 1.5 * that caused execution to terminate abruptly. Note: To properly
1235     * nest multiple overridings, subclasses should generally invoke
1236     * <tt>super.afterExecute</tt> at the beginning of this method.
1237 tim 1.1 *
1238 dl 1.2 * @param r the runnable that has completed.
1239     * @param t the exception that cause termination, or null if
1240     * execution completed normally.
1241 tim 1.1 */
1242 dl 1.2 protected void afterExecute(Runnable r, Throwable t) { }
1243 tim 1.1
1244 dl 1.2 /**
1245     * Method invoked when the Executor has terminated. Default
1246 dl 1.17 * implementation does nothing. Note: To properly nest multiple
1247     * overridings, subclasses should generally invoke
1248     * <tt>super.terminated</tt> within this method.
1249 dl 1.2 */
1250     protected void terminated() { }
1251 tim 1.1
1252     /**
1253 dl 1.21 * A handler for rejected tasks that runs the rejected task
1254     * directly in the calling thread of the <tt>execute</tt> method,
1255     * unless the executor has been shut down, in which case the task
1256     * is discarded.
1257 tim 1.1 */
1258 dl 1.2 public static class CallerRunsPolicy implements RejectedExecutionHandler {
1259 tim 1.1
1260     /**
1261     * Constructs a <tt>CallerRunsPolicy</tt>.
1262     */
1263     public CallerRunsPolicy() { }
1264    
1265 dl 1.2 public void rejectedExecution(Runnable r, ThreadPoolExecutor e) {
1266     if (!e.isShutdown()) {
1267 tim 1.1 r.run();
1268     }
1269     }
1270     }
1271    
1272     /**
1273 dl 1.21 * A handler for rejected tasks that throws a
1274 dl 1.8 * <tt>RejectedExecutionException</tt>.
1275 tim 1.1 */
1276 dl 1.2 public static class AbortPolicy implements RejectedExecutionHandler {
1277 tim 1.1
1278     /**
1279     * Constructs a <tt>AbortPolicy</tt>.
1280     */
1281     public AbortPolicy() { }
1282    
1283 dl 1.2 public void rejectedExecution(Runnable r, ThreadPoolExecutor e) {
1284     throw new RejectedExecutionException();
1285 tim 1.1 }
1286     }
1287    
1288     /**
1289 dl 1.21 * A handler for rejected tasks that silently discards the
1290     * rejected task.
1291 tim 1.1 */
1292 dl 1.2 public static class DiscardPolicy implements RejectedExecutionHandler {
1293 tim 1.1
1294     /**
1295     * Constructs <tt>DiscardPolicy</tt>.
1296     */
1297     public DiscardPolicy() { }
1298    
1299 dl 1.2 public void rejectedExecution(Runnable r, ThreadPoolExecutor e) {
1300 tim 1.1 }
1301     }
1302    
1303     /**
1304 dl 1.21 * A handler for rejected tasks that discards the oldest unhandled
1305     * request and then retries <tt>execute</tt>, unless the executor
1306     * is shut down, in which case the task is discarded.
1307 tim 1.1 */
1308 dl 1.2 public static class DiscardOldestPolicy implements RejectedExecutionHandler {
1309 tim 1.1 /**
1310 dl 1.2 * Constructs a <tt>DiscardOldestPolicy</tt> for the given executor.
1311 tim 1.1 */
1312     public DiscardOldestPolicy() { }
1313    
1314 dl 1.2 public void rejectedExecution(Runnable r, ThreadPoolExecutor e) {
1315     if (!e.isShutdown()) {
1316     e.getQueue().poll();
1317     e.execute(r);
1318 tim 1.1 }
1319     }
1320     }
1321     }