提交 32c540e9 编写于 作者: D dl

7023006: Reduce unnecessary thread activity in ForkJoinPool

Reviewed-by: chegar, dholmes
上级 41df01e6
...@@ -40,6 +40,7 @@ import java.util.Arrays; ...@@ -40,6 +40,7 @@ import java.util.Arrays;
import java.util.Collection; import java.util.Collection;
import java.util.Collections; import java.util.Collections;
import java.util.List; import java.util.List;
import java.util.Random;
import java.util.concurrent.AbstractExecutorService; import java.util.concurrent.AbstractExecutorService;
import java.util.concurrent.Callable; import java.util.concurrent.Callable;
import java.util.concurrent.ExecutorService; import java.util.concurrent.ExecutorService;
...@@ -51,6 +52,7 @@ import java.util.concurrent.TimeoutException; ...@@ -51,6 +52,7 @@ import java.util.concurrent.TimeoutException;
import java.util.concurrent.atomic.AtomicInteger; import java.util.concurrent.atomic.AtomicInteger;
import java.util.concurrent.locks.LockSupport; import java.util.concurrent.locks.LockSupport;
import java.util.concurrent.locks.ReentrantLock; import java.util.concurrent.locks.ReentrantLock;
import java.util.concurrent.locks.Condition;
/** /**
* An {@link ExecutorService} for running {@link ForkJoinTask}s. * An {@link ExecutorService} for running {@link ForkJoinTask}s.
...@@ -158,239 +160,208 @@ public class ForkJoinPool extends AbstractExecutorService { ...@@ -158,239 +160,208 @@ public class ForkJoinPool extends AbstractExecutorService {
* set of worker threads: Submissions from non-FJ threads enter * set of worker threads: Submissions from non-FJ threads enter
* into a submission queue. Workers take these tasks and typically * into a submission queue. Workers take these tasks and typically
* split them into subtasks that may be stolen by other workers. * split them into subtasks that may be stolen by other workers.
* The main work-stealing mechanics implemented in class * Preference rules give first priority to processing tasks from
* ForkJoinWorkerThread give first priority to processing tasks * their own queues (LIFO or FIFO, depending on mode), then to
* from their own queues (LIFO or FIFO, depending on mode), then * randomized FIFO steals of tasks in other worker queues, and
* to randomized FIFO steals of tasks in other worker queues, and * lastly to new submissions.
* lastly to new submissions. These mechanics do not consider *
* affinities, loads, cache localities, etc, so rarely provide the * The main throughput advantages of work-stealing stem from
* best possible performance on a given machine, but portably * decentralized control -- workers mostly take tasks from
* provide good throughput by averaging over these factors. * themselves or each other. We cannot negate this in the
* (Further, even if we did try to use such information, we do not * implementation of other management responsibilities. The main
* usually have a basis for exploiting it. For example, some sets * tactic for avoiding bottlenecks is packing nearly all
* of tasks profit from cache affinities, but others are harmed by * essentially atomic control state into a single 64bit volatile
* cache pollution effects.) * variable ("ctl"). This variable is read on the order of 10-100
* * times as often as it is modified (always via CAS). (There is
* Beyond work-stealing support and essential bookkeeping, the * some additional control state, for example variable "shutdown"
* main responsibility of this framework is to take actions when * for which we can cope with uncoordinated updates.) This
* one worker is waiting to join a task stolen (or always held by) * streamlines synchronization and control at the expense of messy
* another. Because we are multiplexing many tasks on to a pool * constructions needed to repack status bits upon updates.
* of workers, we can't just let them block (as in Thread.join). * Updates tend not to contend with each other except during
* We also cannot just reassign the joiner's run-time stack with * bursts while submitted tasks begin or end. In some cases when
* another and replace it later, which would be a form of * they do contend, threads can instead do something else
* "continuation", that even if possible is not necessarily a good * (usually, scan for tasks) until contention subsides.
* idea. Given that the creation costs of most threads on most *
* systems mainly surrounds setting up runtime stacks, thread * To enable packing, we restrict maximum parallelism to (1<<15)-1
* creation and switching is usually not much more expensive than * (which is far in excess of normal operating range) to allow
* stack creation and switching, and is more flexible). Instead we * ids, counts, and their negations (used for thresholding) to fit
* into 16bit fields.
*
* Recording Workers. Workers are recorded in the "workers" array
* that is created upon pool construction and expanded if (rarely)
* necessary. This is an array as opposed to some other data
* structure to support index-based random steals by workers.
* Updates to the array recording new workers and unrecording
* terminated ones are protected from each other by a seqLock
* (scanGuard) but the array is otherwise concurrently readable,
* and accessed directly by workers. To simplify index-based
* operations, the array size is always a power of two, and all
* readers must tolerate null slots. To avoid flailing during
* start-up, the array is presized to hold twice #parallelism
* workers (which is unlikely to need further resizing during
* execution). But to avoid dealing with so many null slots,
* variable scanGuard includes a mask for the nearest power of two
* that contains all current workers. All worker thread creation
* is on-demand, triggered by task submissions, replacement of
* terminated workers, and/or compensation for blocked
* workers. However, all other support code is set up to work with
* other policies. To ensure that we do not hold on to worker
* references that would prevent GC, ALL accesses to workers are
* via indices into the workers array (which is one source of some
* of the messy code constructions here). In essence, the workers
* array serves as a weak reference mechanism. Thus for example
* the wait queue field of ctl stores worker indices, not worker
* references. Access to the workers in associated methods (for
* example signalWork) must both index-check and null-check the
* IDs. All such accesses ignore bad IDs by returning out early
* from what they are doing, since this can only be associated
* with termination, in which case it is OK to give up.
*
* All uses of the workers array, as well as queue arrays, check
* that the array is non-null (even if previously non-null). This
* allows nulling during termination, which is currently not
* necessary, but remains an option for resource-revocation-based
* shutdown schemes.
*
* Wait Queuing. Unlike HPC work-stealing frameworks, we cannot
* let workers spin indefinitely scanning for tasks when none can
* be found immediately, and we cannot start/resume workers unless
* there appear to be tasks available. On the other hand, we must
* quickly prod them into action when new tasks are submitted or
* generated. We park/unpark workers after placing in an event
* wait queue when they cannot find work. This "queue" is actually
* a simple Treiber stack, headed by the "id" field of ctl, plus a
* 15bit counter value to both wake up waiters (by advancing their
* count) and avoid ABA effects. Successors are held in worker
* field "nextWait". Queuing deals with several intrinsic races,
* mainly that a task-producing thread can miss seeing (and
* signalling) another thread that gave up looking for work but
* has not yet entered the wait queue. We solve this by requiring
* a full sweep of all workers both before (in scan()) and after
* (in tryAwaitWork()) a newly waiting worker is added to the wait
* queue. During a rescan, the worker might release some other
* queued worker rather than itself, which has the same net
* effect. Because enqueued workers may actually be rescanning
* rather than waiting, we set and clear the "parked" field of
* ForkJoinWorkerThread to reduce unnecessary calls to unpark.
* (Use of the parked field requires a secondary recheck to avoid
* missed signals.)
*
* Signalling. We create or wake up workers only when there
* appears to be at least one task they might be able to find and
* execute. When a submission is added or another worker adds a
* task to a queue that previously had two or fewer tasks, they
* signal waiting workers (or trigger creation of new ones if
* fewer than the given parallelism level -- see signalWork).
* These primary signals are buttressed by signals during rescans
* as well as those performed when a worker steals a task and
* notices that there are more tasks too; together these cover the
* signals needed in cases when more than two tasks are pushed
* but untaken.
*
* Trimming workers. To release resources after periods of lack of
* use, a worker starting to wait when the pool is quiescent will
* time out and terminate if the pool has remained quiescent for
* SHRINK_RATE nanosecs. This will slowly propagate, eventually
* terminating all workers after long periods of non-use.
*
* Submissions. External submissions are maintained in an
* array-based queue that is structured identically to
* ForkJoinWorkerThread queues except for the use of
* submissionLock in method addSubmission. Unlike the case for
* worker queues, multiple external threads can add new
* submissions, so adding requires a lock.
*
* Compensation. Beyond work-stealing support and lifecycle
* control, the main responsibility of this framework is to take
* actions when one worker is waiting to join a task stolen (or
* always held by) another. Because we are multiplexing many
* tasks on to a pool of workers, we can't just let them block (as
* in Thread.join). We also cannot just reassign the joiner's
* run-time stack with another and replace it later, which would
* be a form of "continuation", that even if possible is not
* necessarily a good idea since we sometimes need both an
* unblocked task and its continuation to progress. Instead we
* combine two tactics: * combine two tactics:
* *
* Helping: Arranging for the joiner to execute some task that it * Helping: Arranging for the joiner to execute some task that it
* would be running if the steal had not occurred. Method * would be running if the steal had not occurred. Method
* ForkJoinWorkerThread.helpJoinTask tracks joining->stealing * ForkJoinWorkerThread.joinTask tracks joining->stealing
* links to try to find such a task. * links to try to find such a task.
* *
* Compensating: Unless there are already enough live threads, * Compensating: Unless there are already enough live threads,
* method helpMaintainParallelism() may create or * method tryPreBlock() may create or re-activate a spare
* re-activate a spare thread to compensate for blocked * thread to compensate for blocked joiners until they
* joiners until they unblock. * unblock.
*
* It is impossible to keep exactly the target (parallelism)
* number of threads running at any given time. Determining
* existence of conservatively safe helping targets, the
* availability of already-created spares, and the apparent need
* to create new spares are all racy and require heuristic
* guidance, so we rely on multiple retries of each. Compensation
* occurs in slow-motion. It is triggered only upon timeouts of
* Object.wait used for joins. This reduces poor decisions that
* would otherwise be made when threads are waiting for others
* that are stalled because of unrelated activities such as
* garbage collection.
* *
* The ManagedBlocker extension API can't use helping so relies * The ManagedBlocker extension API can't use helping so relies
* only on compensation in method awaitBlocker. * only on compensation in method awaitBlocker.
* *
* The main throughput advantages of work-stealing stem from * It is impossible to keep exactly the target parallelism number
* decentralized control -- workers mostly steal tasks from each * of threads running at any given time. Determining the
* other. We do not want to negate this by creating bottlenecks * existence of conservatively safe helping targets, the
* implementing other management responsibilities. So we use a * availability of already-created spares, and the apparent need
* collection of techniques that avoid, reduce, or cope well with * to create new spares are all racy and require heuristic
* contention. These entail several instances of bit-packing into * guidance, so we rely on multiple retries of each. Currently,
* CASable fields to maintain only the minimally required * in keeping with on-demand signalling policy, we compensate only
* atomicity. To enable such packing, we restrict maximum * if blocking would leave less than one active (non-waiting,
* parallelism to (1<<15)-1 (enabling twice this (to accommodate * non-blocked) worker. Additionally, to avoid some false alarms
* unbalanced increments and decrements) to fit into a 16 bit * due to GC, lagging counters, system activity, etc, compensated
* field, which is far in excess of normal operating range. Even * blocking for joins is only attempted after rechecks stabilize
* though updates to some of these bookkeeping fields do sometimes * (retries are interspersed with Thread.yield, for good
* contend with each other, they don't normally cache-contend with * citizenship). The variable blockedCount, incremented before
* updates to others enough to warrant memory padding or * blocking and decremented after, is sometimes needed to
* isolation. So they are all held as fields of ForkJoinPool * distinguish cases of waiting for work vs blocking on joins or
* objects. The main capabilities are as follows: * other managed sync. Both cases are equivalent for most pool
* * control, so we can update non-atomically. (Additionally,
* 1. Creating and removing workers. Workers are recorded in the * contention on blockedCount alleviates some contention on ctl).
* "workers" array. This is an array as opposed to some other data *
* structure to support index-based random steals by workers. * Shutdown and Termination. A call to shutdownNow atomically sets
* Updates to the array recording new workers and unrecording * the ctl stop bit and then (non-atomically) sets each workers
* terminated ones are protected from each other by a lock * "terminate" status, cancels all unprocessed tasks, and wakes up
* (workerLock) but the array is otherwise concurrently readable, * all waiting workers. Detecting whether termination should
* and accessed directly by workers. To simplify index-based * commence after a non-abrupt shutdown() call requires more work
* operations, the array size is always a power of two, and all * and bookkeeping. We need consensus about quiesence (i.e., that
* readers must tolerate null slots. Currently, all worker thread * there is no more work) which is reflected in active counts so
* creation is on-demand, triggered by task submissions, * long as there are no current blockers, as well as possible
* replacement of terminated workers, and/or compensation for * re-evaluations during independent changes in blocking or
* blocked workers. However, all other support code is set up to * quiescing workers.
* work with other policies. *
* * Style notes: There is a lot of representation-level coupling
* To ensure that we do not hold on to worker references that
* would prevent GC, ALL accesses to workers are via indices into
* the workers array (which is one source of some of the unusual
* code constructions here). In essence, the workers array serves
* as a WeakReference mechanism. Thus for example the event queue
* stores worker indices, not worker references. Access to the
* workers in associated methods (for example releaseEventWaiters)
* must both index-check and null-check the IDs. All such accesses
* ignore bad IDs by returning out early from what they are doing,
* since this can only be associated with shutdown, in which case
* it is OK to give up. On termination, we just clobber these
* data structures without trying to use them.
*
* 2. Bookkeeping for dynamically adding and removing workers. We
* aim to approximately maintain the given level of parallelism.
* When some workers are known to be blocked (on joins or via
* ManagedBlocker), we may create or resume others to take their
* place until they unblock (see below). Implementing this
* requires counts of the number of "running" threads (i.e., those
* that are neither blocked nor artificially suspended) as well as
* the total number. These two values are packed into one field,
* "workerCounts" because we need accurate snapshots when deciding
* to create, resume or suspend. Note however that the
* correspondence of these counts to reality is not guaranteed. In
* particular updates for unblocked threads may lag until they
* actually wake up.
*
* 3. Maintaining global run state. The run state of the pool
* consists of a runLevel (SHUTDOWN, TERMINATING, etc) similar to
* those in other Executor implementations, as well as a count of
* "active" workers -- those that are, or soon will be, or
* recently were executing tasks. The runLevel and active count
* are packed together in order to correctly trigger shutdown and
* termination. Without care, active counts can be subject to very
* high contention. We substantially reduce this contention by
* relaxing update rules. A worker must claim active status
* prospectively, by activating if it sees that a submitted or
* stealable task exists (it may find after activating that the
* task no longer exists). It stays active while processing this
* task (if it exists) and any other local subtasks it produces,
* until it cannot find any other tasks. It then tries
* inactivating (see method preStep), but upon update contention
* instead scans for more tasks, later retrying inactivation if it
* doesn't find any.
*
* 4. Managing idle workers waiting for tasks. We cannot let
* workers spin indefinitely scanning for tasks when none are
* available. On the other hand, we must quickly prod them into
* action when new tasks are submitted or generated. We
* park/unpark these idle workers using an event-count scheme.
* Field eventCount is incremented upon events that may enable
* workers that previously could not find a task to now find one:
* Submission of a new task to the pool, or another worker pushing
* a task onto a previously empty queue. (We also use this
* mechanism for configuration and termination actions that
* require wakeups of idle workers). Each worker maintains its
* last known event count, and blocks when a scan for work did not
* find a task AND its lastEventCount matches the current
* eventCount. Waiting idle workers are recorded in a variant of
* Treiber stack headed by field eventWaiters which, when nonzero,
* encodes the thread index and count awaited for by the worker
* thread most recently calling eventSync. This thread in turn has
* a record (field nextEventWaiter) for the next waiting worker.
* In addition to allowing simpler decisions about need for
* wakeup, the event count bits in eventWaiters serve the role of
* tags to avoid ABA errors in Treiber stacks. Upon any wakeup,
* released threads also try to release at most two others. The
* net effect is a tree-like diffusion of signals, where released
* threads (and possibly others) help with unparks. To further
* reduce contention effects a bit, failed CASes to increment
* field eventCount are tolerated without retries in signalWork.
* Conceptually they are merged into the same event, which is OK
* when their only purpose is to enable workers to scan for work.
*
* 5. Managing suspension of extra workers. When a worker notices
* (usually upon timeout of a wait()) that there are too few
* running threads, we may create a new thread to maintain
* parallelism level, or at least avoid starvation. Usually, extra
* threads are needed for only very short periods, yet join
* dependencies are such that we sometimes need them in
* bursts. Rather than create new threads each time this happens,
* we suspend no-longer-needed extra ones as "spares". For most
* purposes, we don't distinguish "extra" spare threads from
* normal "core" threads: On each call to preStep (the only point
* at which we can do this) a worker checks to see if there are
* now too many running workers, and if so, suspends itself.
* Method helpMaintainParallelism looks for suspended threads to
* resume before considering creating a new replacement. The
* spares themselves are encoded on another variant of a Treiber
* Stack, headed at field "spareWaiters". Note that the use of
* spares is intrinsically racy. One thread may become a spare at
* about the same time as another is needlessly being created. We
* counteract this and related slop in part by requiring resumed
* spares to immediately recheck (in preStep) to see whether they
* should re-suspend.
*
* 6. Killing off unneeded workers. A timeout mechanism is used to
* shed unused workers: The oldest (first) event queue waiter uses
* a timed rather than hard wait. When this wait times out without
* a normal wakeup, it tries to shutdown any one (for convenience
* the newest) other spare or event waiter via
* tryShutdownUnusedWorker. This eventually reduces the number of
* worker threads to a minimum of one after a long enough period
* without use.
*
* 7. Deciding when to create new workers. The main dynamic
* control in this class is deciding when to create extra threads
* in method helpMaintainParallelism. We would like to keep
* exactly #parallelism threads running, which is an impossible
* task. We always need to create one when the number of running
* threads would become zero and all workers are busy. Beyond
* this, we must rely on heuristics that work well in the
* presence of transient phenomena such as GC stalls, dynamic
* compilation, and wake-up lags. These transients are extremely
* common -- we are normally trying to fully saturate the CPUs on
* a machine, so almost any activity other than running tasks
* impedes accuracy. Our main defense is to allow parallelism to
* lapse for a while during joins, and use a timeout to see if,
* after the resulting settling, there is still a need for
* additional workers. This also better copes with the fact that
* some of the methods in this class tend to never become compiled
* (but are interpreted), so some components of the entire set of
* controls might execute 100 times faster than others. And
* similarly for cases where the apparent lack of work is just due
* to GC stalls and other transient system activity.
*
* Beware that there is a lot of representation-level coupling
* among classes ForkJoinPool, ForkJoinWorkerThread, and * among classes ForkJoinPool, ForkJoinWorkerThread, and
* ForkJoinTask. For example, direct access to "workers" array by * ForkJoinTask. Most fields of ForkJoinWorkerThread maintain
* data structures managed by ForkJoinPool, so are directly
* accessed. Conversely we allow access to "workers" array by
* workers, and direct access to ForkJoinTask.status by both * workers, and direct access to ForkJoinTask.status by both
* ForkJoinPool and ForkJoinWorkerThread. There is little point * ForkJoinPool and ForkJoinWorkerThread. There is little point
* trying to reduce this, since any associated future changes in * trying to reduce this, since any associated future changes in
* representations will need to be accompanied by algorithmic * representations will need to be accompanied by algorithmic
* changes anyway. * changes anyway. All together, these low-level implementation
* * choices produce as much as a factor of 4 performance
* Style notes: There are lots of inline assignments (of form * improvement compared to naive implementations, and enable the
* "while ((local = field) != 0)") which are usually the simplest * processing of billions of tasks per second, at the expense of
* way to ensure the required read orderings (which are sometimes * some ugliness.
* critical). Also several occurrences of the unusual "do {} *
* while (!cas...)" which is the simplest way to force an update of * Methods signalWork() and scan() are the main bottlenecks so are
* a CAS'ed variable. There are also other coding oddities that * especially heavily micro-optimized/mangled. There are lots of
* help some methods perform reasonably even when interpreted (not * inline assignments (of form "while ((local = field) != 0)")
* compiled), at the expense of some messy constructions that * which are usually the simplest way to ensure the required read
* reduce byte code counts. * orderings (which are sometimes critical). This leads to a
* * "C"-like style of listing declarations of these locals at the
* The order of declarations in this file is: (1) statics (2) * heads of methods or blocks. There are several occurrences of
* fields (along with constants used when unpacking some of them) * the unusual "do {} while (!cas...)" which is the simplest way
* (3) internal control methods (4) callbacks and other support * to force an update of a CAS'ed variable. There are also other
* for ForkJoinTask and ForkJoinWorkerThread classes, (5) exported * coding oddities that help some methods perform reasonably even
* methods (plus a few little helpers). * when interpreted (not compiled).
*
* The order of declarations in this file is: (1) declarations of
* statics (2) fields (along with constants used when unpacking
* some of them), listed in an order that tends to reduce
* contention among them a bit under most JVMs. (3) internal
* control methods (4) callbacks and other support for
* ForkJoinTask and ForkJoinWorkerThread classes, (5) exported
* methods (plus a few little helpers). (6) static block
* initializing all statics in a minimally dependent order.
*/ */
/** /**
...@@ -425,15 +396,13 @@ public class ForkJoinPool extends AbstractExecutorService { ...@@ -425,15 +396,13 @@ public class ForkJoinPool extends AbstractExecutorService {
* overridden in ForkJoinPool constructors. * overridden in ForkJoinPool constructors.
*/ */
public static final ForkJoinWorkerThreadFactory public static final ForkJoinWorkerThreadFactory
defaultForkJoinWorkerThreadFactory = defaultForkJoinWorkerThreadFactory;
new DefaultForkJoinWorkerThreadFactory();
/** /**
* Permission required for callers of methods that may start or * Permission required for callers of methods that may start or
* kill threads. * kill threads.
*/ */
private static final RuntimePermission modifyThreadPermission = private static final RuntimePermission modifyThreadPermission;
new RuntimePermission("modifyThread");
/** /**
* If there is a security manager, makes sure caller has * If there is a security manager, makes sure caller has
...@@ -448,63 +417,59 @@ public class ForkJoinPool extends AbstractExecutorService { ...@@ -448,63 +417,59 @@ public class ForkJoinPool extends AbstractExecutorService {
/** /**
* Generator for assigning sequence numbers as pool names. * Generator for assigning sequence numbers as pool names.
*/ */
private static final AtomicInteger poolNumberGenerator = private static final AtomicInteger poolNumberGenerator;
new AtomicInteger();
/** /**
* The time to block in a join (see awaitJoin) before checking if * Generator for initial random seeds for worker victim
* a new worker should be (re)started to maintain parallelism * selection. This is used only to create initial seeds. Random
* level. The value should be short enough to maintain global * steals use a cheaper xorshift generator per steal attempt. We
* responsiveness and progress but long enough to avoid * don't expect much contention on seedGenerator, so just use a
* counterproductive firings during GC stalls or unrelated system * plain Random.
* activity, and to not bog down systems with continual re-firings
* on GCs or legitimately long waits.
*/ */
private static final long JOIN_TIMEOUT_MILLIS = 250L; // 4 per second static final Random workerSeedGenerator;
/** /**
* The wakeup interval (in nanoseconds) for the oldest worker * Array holding all worker threads in the pool. Initialized upon
* waiting for an event to invoke tryShutdownUnusedWorker to * construction. Array size must be a power of two. Updates and
* shrink the number of workers. The exact value does not matter * replacements are protected by scanGuard, but the array is
* too much. It must be short enough to release resources during * always kept in a consistent enough state to be randomly
* sustained periods of idleness, but not so short that threads * accessed without locking by workers performing work-stealing,
* are continually re-created. * as well as other traversal-based methods in this class, so long
* as reads memory-acquire by first reading ctl. All readers must
* tolerate that some array slots may be null.
*/ */
private static final long SHRINK_RATE_NANOS = ForkJoinWorkerThread[] workers;
30L * 1000L * 1000L * 1000L; // 2 per minute
/** /**
* Absolute bound for parallelism level. Twice this number plus * Initial size for submission queue array. Must be a power of
* one (i.e., 0xfff) must fit into a 16bit field to enable * two. In many applications, these always stay small so we use a
* word-packing for some counts and indices. * small initial cap.
*/ */
private static final int MAX_WORKERS = 0x7fff; private static final int INITIAL_QUEUE_CAPACITY = 8;
/** /**
* Array holding all worker threads in the pool. Array size must * Maximum size for submission queue array. Must be a power of two
* be a power of two. Updates and replacements are protected by * less than or equal to 1 << (31 - width of array entry) to
* workerLock, but the array is always kept in a consistent enough * ensure lack of index wraparound, but is capped at a lower
* state to be randomly accessed without locking by workers * value to help users trap runaway computations.
* performing work-stealing, as well as other traversal-based
* methods in this class. All readers must tolerate that some
* array slots may be null.
*/ */
volatile ForkJoinWorkerThread[] workers; private static final int MAXIMUM_QUEUE_CAPACITY = 1 << 24; // 16M
/** /**
* Queue for external submissions. * Array serving as submission queue. Initialized upon construction.
*/ */
private final LinkedTransferQueue<ForkJoinTask<?>> submissionQueue; private ForkJoinTask<?>[] submissionQueue;
/** /**
* Lock protecting updates to workers array. * Lock protecting submissions array for addSubmission
*/ */
private final ReentrantLock workerLock; private final ReentrantLock submissionLock;
/** /**
* Latch released upon termination. * Condition for awaitTermination, using submissionLock for
* convenience.
*/ */
private final Phaser termination; private final Condition termination;
/** /**
* Creation factory for worker threads. * Creation factory for worker threads.
...@@ -512,87 +477,103 @@ public class ForkJoinPool extends AbstractExecutorService { ...@@ -512,87 +477,103 @@ public class ForkJoinPool extends AbstractExecutorService {
private final ForkJoinWorkerThreadFactory factory; private final ForkJoinWorkerThreadFactory factory;
/** /**
* Sum of per-thread steal counts, updated only when threads are * The uncaught exception handler used when any worker abruptly
* idle or terminating. * terminates.
*/ */
private volatile long stealCount; final Thread.UncaughtExceptionHandler ueh;
/** /**
* Encoded record of top of Treiber stack of threads waiting for * Prefix for assigning names to worker threads
* events. The top 32 bits contain the count being waited for. The
* bottom 16 bits contains one plus the pool index of waiting
* worker thread. (Bits 16-31 are unused.)
*/ */
private volatile long eventWaiters; private final String workerNamePrefix;
private static final int EVENT_COUNT_SHIFT = 32;
private static final int WAITER_ID_MASK = (1 << 16) - 1;
/** /**
* A counter for events that may wake up worker threads: * Sum of per-thread steal counts, updated only when threads are
* - Submission of a new task to the pool * idle or terminating.
* - A worker pushing a task on an empty queue
* - termination
*/ */
private volatile int eventCount; private volatile long stealCount;
/** /**
* Encoded record of top of Treiber stack of spare threads waiting * Main pool control -- a long packed with:
* for resumption. The top 16 bits contain an arbitrary count to * AC: Number of active running workers minus target parallelism (16 bits)
* avoid ABA effects. The bottom 16bits contains one plus the pool * TC: Number of total workers minus target parallelism (16bits)
* index of waiting worker thread. * ST: true if pool is terminating (1 bit)
*/ * EC: the wait count of top waiting thread (15 bits)
private volatile int spareWaiters; * ID: ~poolIndex of top of Treiber stack of waiting threads (16 bits)
*
private static final int SPARE_COUNT_SHIFT = 16; * When convenient, we can extract the upper 32 bits of counts and
private static final int SPARE_ID_MASK = (1 << 16) - 1; * the lower 32 bits of queue state, u = (int)(ctl >>> 32) and e =
* (int)ctl. The ec field is never accessed alone, but always
* together with id and st. The offsets of counts by the target
* parallelism and the positionings of fields makes it possible to
* perform the most common checks via sign tests of fields: When
* ac is negative, there are not enough active workers, when tc is
* negative, there are not enough total workers, when id is
* negative, there is at least one waiting worker, and when e is
* negative, the pool is terminating. To deal with these possibly
* negative fields, we use casts in and out of "short" and/or
* signed shifts to maintain signedness.
*/
volatile long ctl;
// bit positions/shifts for fields
private static final int AC_SHIFT = 48;
private static final int TC_SHIFT = 32;
private static final int ST_SHIFT = 31;
private static final int EC_SHIFT = 16;
// bounds
private static final int MAX_ID = 0x7fff; // max poolIndex
private static final int SMASK = 0xffff; // mask short bits
private static final int SHORT_SIGN = 1 << 15;
private static final int INT_SIGN = 1 << 31;
// masks
private static final long STOP_BIT = 0x0001L << ST_SHIFT;
private static final long AC_MASK = ((long)SMASK) << AC_SHIFT;
private static final long TC_MASK = ((long)SMASK) << TC_SHIFT;
// units for incrementing and decrementing
private static final long TC_UNIT = 1L << TC_SHIFT;
private static final long AC_UNIT = 1L << AC_SHIFT;
// masks and units for dealing with u = (int)(ctl >>> 32)
private static final int UAC_SHIFT = AC_SHIFT - 32;
private static final int UTC_SHIFT = TC_SHIFT - 32;
private static final int UAC_MASK = SMASK << UAC_SHIFT;
private static final int UTC_MASK = SMASK << UTC_SHIFT;
private static final int UAC_UNIT = 1 << UAC_SHIFT;
private static final int UTC_UNIT = 1 << UTC_SHIFT;
// masks and units for dealing with e = (int)ctl
private static final int E_MASK = 0x7fffffff; // no STOP_BIT
private static final int EC_UNIT = 1 << EC_SHIFT;
/** /**
* Lifecycle control. The low word contains the number of workers * The target parallelism level.
* that are (probably) executing tasks. This value is atomically
* incremented before a worker gets a task to run, and decremented
* when a worker has no tasks and cannot find any. Bits 16-18
* contain runLevel value. When all are zero, the pool is
* running. Level transitions are monotonic (running -> shutdown
* -> terminating -> terminated) so each transition adds a bit.
* These are bundled together to ensure consistent read for
* termination checks (i.e., that runLevel is at least SHUTDOWN
* and active threads is zero).
*
* Notes: Most direct CASes are dependent on these bitfield
* positions. Also, this field is non-private to enable direct
* performance-sensitive CASes in ForkJoinWorkerThread.
*/ */
volatile int runState; final int parallelism;
// Note: The order among run level values matters.
private static final int RUNLEVEL_SHIFT = 16;
private static final int SHUTDOWN = 1 << RUNLEVEL_SHIFT;
private static final int TERMINATING = 1 << (RUNLEVEL_SHIFT + 1);
private static final int TERMINATED = 1 << (RUNLEVEL_SHIFT + 2);
private static final int ACTIVE_COUNT_MASK = (1 << RUNLEVEL_SHIFT) - 1;
/** /**
* Holds number of total (i.e., created and not yet terminated) * Index (mod submission queue length) of next element to take
* and running (i.e., not blocked on joins or other managed sync) * from submission queue. Usage is identical to that for
* threads, packed together to ensure consistent snapshot when * per-worker queues -- see ForkJoinWorkerThread internal
* making decisions about creating and suspending spare * documentation.
* threads. Updated only by CAS. Note that adding a new worker
* requires incrementing both counts, since workers start off in
* running state.
*/ */
private volatile int workerCounts; volatile int queueBase;
private static final int TOTAL_COUNT_SHIFT = 16; /**
private static final int RUNNING_COUNT_MASK = (1 << TOTAL_COUNT_SHIFT) - 1; * Index (mod submission queue length) of next element to add
private static final int ONE_RUNNING = 1; * in submission queue. Usage is identical to that for
private static final int ONE_TOTAL = 1 << TOTAL_COUNT_SHIFT; * per-worker queues -- see ForkJoinWorkerThread internal
* documentation.
*/
int queueTop;
/** /**
* The target parallelism level. * True when shutdown() has been called.
* Accessed directly by ForkJoinWorkerThreads.
*/ */
final int parallelism; volatile boolean shutdown;
/** /**
* True if use local fifo, not default lifo, for local polling * True if use local fifo, not default lifo, for local polling
...@@ -601,557 +582,664 @@ public class ForkJoinPool extends AbstractExecutorService { ...@@ -601,557 +582,664 @@ public class ForkJoinPool extends AbstractExecutorService {
final boolean locallyFifo; final boolean locallyFifo;
/** /**
* The uncaught exception handler used when any worker abruptly * The number of threads in ForkJoinWorkerThreads.helpQuiescePool.
* terminates. * When non-zero, suppresses automatic shutdown when active
* counts become zero.
*/ */
private final Thread.UncaughtExceptionHandler ueh; volatile int quiescerCount;
/** /**
* Pool number, just for assigning useful names to worker threads * The number of threads blocked in join.
*/ */
private final int poolNumber; volatile int blockedCount;
// Utilities for CASing fields. Note that most of these /**
// are usually manually inlined by callers * Counter for worker Thread names (unrelated to their poolIndex)
*/
private volatile int nextWorkerNumber;
/** /**
* Increments running count part of workerCounts. * The index for the next created worker. Accessed under scanGuard.
*/ */
final void incrementRunningCount() { private int nextWorkerIndex;
int c;
do {} while (!UNSAFE.compareAndSwapInt(this, workerCountsOffset,
c = workerCounts,
c + ONE_RUNNING));
}
/** /**
* Tries to increment running count part of workerCounts. * SeqLock and index masking for updates to workers array. Locked
* when SG_UNIT is set. Unlocking clears bit by adding
* SG_UNIT. Staleness of read-only operations can be checked by
* comparing scanGuard to value before the reads. The low 16 bits
* (i.e, anding with SMASK) hold (the smallest power of two
* covering all worker indices, minus one, and is used to avoid
* dealing with large numbers of null slots when the workers array
* is overallocated.
*/ */
final boolean tryIncrementRunningCount() { volatile int scanGuard;
int c;
return UNSAFE.compareAndSwapInt(this, workerCountsOffset, private static final int SG_UNIT = 1 << 16;
c = workerCounts,
c + ONE_RUNNING);
}
/** /**
* Tries to decrement running count unless already zero. * The wakeup interval (in nanoseconds) for a worker waiting for a
* task when the pool is quiescent to instead try to shrink the
* number of workers. The exact value does not matter too
* much. It must be short enough to release resources during
* sustained periods of idleness, but not so short that threads
* are continually re-created.
*/ */
final boolean tryDecrementRunningCount() { private static final long SHRINK_RATE =
int wc = workerCounts; 4L * 1000L * 1000L * 1000L; // 4 seconds
if ((wc & RUNNING_COUNT_MASK) == 0)
return false;
return UNSAFE.compareAndSwapInt(this, workerCountsOffset,
wc, wc - ONE_RUNNING);
}
/** /**
* Forces decrement of encoded workerCounts, awaiting nonzero if * Top-level loop for worker threads: On each step: if the
* (rarely) necessary when other count updates lag. * previous step swept through all queues and found no tasks, or
* there are excess threads, then possibly blocks. Otherwise,
* scans for and, if found, executes a task. Returns when pool
* and/or worker terminate.
* *
* @param dr -- either zero or ONE_RUNNING * @param w the worker
* @param dt -- either zero or ONE_TOTAL
*/ */
private void decrementWorkerCounts(int dr, int dt) { final void work(ForkJoinWorkerThread w) {
for (;;) { boolean swept = false; // true on empty scans
int wc = workerCounts; long c;
if ((wc & RUNNING_COUNT_MASK) - dr < 0 || while (!w.terminate && (int)(c = ctl) >= 0) {
(wc >>> TOTAL_COUNT_SHIFT) - dt < 0) { int a; // active count
if ((runState & TERMINATED) != 0) if (!swept && (a = (int)(c >> AC_SHIFT)) <= 0)
return; // lagging termination on a backout swept = scan(w, a);
Thread.yield(); else if (tryAwaitWork(w, c))
} swept = false;
if (UNSAFE.compareAndSwapInt(this, workerCountsOffset,
wc, wc - (dr + dt)))
return;
} }
} }
// Signalling
/** /**
* Tries decrementing active count; fails on contention. * Wakes up or creates a worker.
* Called when workers cannot find tasks to run.
*/ */
final boolean tryDecrementActiveCount() { final void signalWork() {
int c; /*
return UNSAFE.compareAndSwapInt(this, runStateOffset, * The while condition is true if: (there is are too few total
c = runState, c - 1); * workers OR there is at least one waiter) AND (there are too
* few active workers OR the pool is terminating). The value
* of e distinguishes the remaining cases: zero (no waiters)
* for create, negative if terminating (in which case do
* nothing), else release a waiter. The secondary checks for
* release (non-null array etc) can fail if the pool begins
* terminating after the test, and don't impose any added cost
* because JVMs must perform null and bounds checks anyway.
*/
long c; int e, u;
while ((((e = (int)(c = ctl)) | (u = (int)(c >>> 32))) &
(INT_SIGN|SHORT_SIGN)) == (INT_SIGN|SHORT_SIGN) && e >= 0) {
if (e > 0) { // release a waiting worker
int i; ForkJoinWorkerThread w; ForkJoinWorkerThread[] ws;
if ((ws = workers) == null ||
(i = ~e & SMASK) >= ws.length ||
(w = ws[i]) == null)
break;
long nc = (((long)(w.nextWait & E_MASK)) |
((long)(u + UAC_UNIT) << 32));
if (w.eventCount == e &&
UNSAFE.compareAndSwapLong(this, ctlOffset, c, nc)) {
w.eventCount = (e + EC_UNIT) & E_MASK;
if (w.parked)
UNSAFE.unpark(w);
break;
}
}
else if (UNSAFE.compareAndSwapLong
(this, ctlOffset, c,
(long)(((u + UTC_UNIT) & UTC_MASK) |
((u + UAC_UNIT) & UAC_MASK)) << 32)) {
addWorker();
break;
}
}
} }
/** /**
* Advances to at least the given level. Returns true if not * Variant of signalWork to help release waiters on rescans.
* already in at least the given level. * Tries once to release a waiter if active count < 0.
*
* @return false if failed due to contention, else true
*/ */
private boolean advanceRunLevel(int level) { private boolean tryReleaseWaiter() {
for (;;) { long c; int e, i; ForkJoinWorkerThread w; ForkJoinWorkerThread[] ws;
int s = runState; if ((e = (int)(c = ctl)) > 0 &&
if ((s & level) != 0) (int)(c >> AC_SHIFT) < 0 &&
(ws = workers) != null &&
(i = ~e & SMASK) < ws.length &&
(w = ws[i]) != null) {
long nc = ((long)(w.nextWait & E_MASK) |
((c + AC_UNIT) & (AC_MASK|TC_MASK)));
if (w.eventCount != e ||
!UNSAFE.compareAndSwapLong(this, ctlOffset, c, nc))
return false; return false;
if (UNSAFE.compareAndSwapInt(this, runStateOffset, s, s | level)) w.eventCount = (e + EC_UNIT) & E_MASK;
return true; if (w.parked)
UNSAFE.unpark(w);
} }
return true;
} }
// workers array maintenance // Scanning for tasks
/** /**
* Records and returns a workers array index for new worker. * Scans for and, if found, executes one task. Scans start at a
* random index of workers array, and randomly select the first
* (2*#workers)-1 probes, and then, if all empty, resort to 2
* circular sweeps, which is necessary to check quiescence. and
* taking a submission only if no stealable tasks were found. The
* steal code inside the loop is a specialized form of
* ForkJoinWorkerThread.deqTask, followed bookkeeping to support
* helpJoinTask and signal propagation. The code for submission
* queues is almost identical. On each steal, the worker completes
* not only the task, but also all local tasks that this task may
* have generated. On detecting staleness or contention when
* trying to take a task, this method returns without finishing
* sweep, which allows global state rechecks before retry.
*
* @param w the worker
* @param a the number of active workers
* @return true if swept all queues without finding a task
*/ */
private int recordWorker(ForkJoinWorkerThread w) { private boolean scan(ForkJoinWorkerThread w, int a) {
// Try using slot totalCount-1. If not available, scan and/or resize int g = scanGuard; // mask 0 avoids useless scans if only one active
int k = (workerCounts >>> TOTAL_COUNT_SHIFT) - 1; int m = (parallelism == 1 - a && blockedCount == 0) ? 0 : g & SMASK;
final ReentrantLock lock = this.workerLock;
lock.lock();
try {
ForkJoinWorkerThread[] ws = workers; ForkJoinWorkerThread[] ws = workers;
int n = ws.length; if (ws == null || ws.length <= m) // staleness check
if (k < 0 || k >= n || ws[k] != null) { return false;
for (k = 0; k < n && ws[k] != null; ++k) for (int r = w.seed, k = r, j = -(m + m); j <= m + m; ++j) {
; ForkJoinTask<?> t; ForkJoinTask<?>[] q; int b, i;
if (k == n) ForkJoinWorkerThread v = ws[k & m];
ws = workers = Arrays.copyOf(ws, n << 1); if (v != null && (b = v.queueBase) != v.queueTop &&
(q = v.queue) != null && (i = (q.length - 1) & b) >= 0) {
long u = (i << ASHIFT) + ABASE;
if ((t = q[i]) != null && v.queueBase == b &&
UNSAFE.compareAndSwapObject(q, u, t, null)) {
int d = (v.queueBase = b + 1) - v.queueTop;
v.stealHint = w.poolIndex;
if (d != 0)
signalWork(); // propagate if nonempty
w.execTask(t);
}
r ^= r << 13; r ^= r >>> 17; w.seed = r ^ (r << 5);
return false; // store next seed
}
else if (j < 0) { // xorshift
r ^= r << 13; r ^= r >>> 17; k = r ^= r << 5;
} }
ws[k] = w; else
int c = eventCount; // advance event count to ensure visibility ++k;
UNSAFE.compareAndSwapInt(this, eventCountOffset, c, c+1);
} finally {
lock.unlock();
} }
return k; if (scanGuard != g) // staleness check
return false;
else { // try to take submission
ForkJoinTask<?> t; ForkJoinTask<?>[] q; int b, i;
if ((b = queueBase) != queueTop &&
(q = submissionQueue) != null &&
(i = (q.length - 1) & b) >= 0) {
long u = (i << ASHIFT) + ABASE;
if ((t = q[i]) != null && queueBase == b &&
UNSAFE.compareAndSwapObject(q, u, t, null)) {
queueBase = b + 1;
w.execTask(t);
} }
return false;
/** }
* Nulls out record of worker in workers array. return true; // all queues empty
*/ }
private void forgetWorker(ForkJoinWorkerThread w) { }
int idx = w.poolIndex;
// Locking helps method recordWorker avoid unnecessary expansion /**
final ReentrantLock lock = this.workerLock; * Tries to enqueue worker w in wait queue and await change in
lock.lock(); * worker's eventCount. If the pool is quiescent, possibly
try { * terminates worker upon exit. Otherwise, before blocking,
* rescans queues to avoid missed signals. Upon finding work,
* releases at least one worker (which may be the current
* worker). Rescans restart upon detected staleness or failure to
* release due to contention. Note the unusual conventions about
* Thread.interrupt here and elsewhere: Because interrupts are
* used solely to alert threads to check termination, which is
* checked here anyway, we clear status (using Thread.interrupted)
* before any call to park, so that park does not immediately
* return due to status being set via some other unrelated call to
* interrupt in user code.
*
* @param w the calling worker
* @param c the ctl value on entry
* @return true if waited or another thread was released upon enq
*/
private boolean tryAwaitWork(ForkJoinWorkerThread w, long c) {
int v = w.eventCount;
w.nextWait = (int)c; // w's successor record
long nc = (long)(v & E_MASK) | ((c - AC_UNIT) & (AC_MASK|TC_MASK));
if (ctl != c || !UNSAFE.compareAndSwapLong(this, ctlOffset, c, nc)) {
long d = ctl; // return true if lost to a deq, to force scan
return (int)d != (int)c && ((d - c) & AC_MASK) >= 0L;
}
for (int sc = w.stealCount; sc != 0;) { // accumulate stealCount
long s = stealCount;
if (UNSAFE.compareAndSwapLong(this, stealCountOffset, s, s + sc))
sc = w.stealCount = 0;
else if (w.eventCount != v)
return true; // update next time
}
if (parallelism + (int)(nc >> AC_SHIFT) == 0 &&
blockedCount == 0 && quiescerCount == 0)
idleAwaitWork(w, nc, c, v); // quiescent
for (boolean rescanned = false;;) {
if (w.eventCount != v)
return true;
if (!rescanned) {
int g = scanGuard, m = g & SMASK;
ForkJoinWorkerThread[] ws = workers; ForkJoinWorkerThread[] ws = workers;
if (idx >= 0 && idx < ws.length && ws[idx] == w) // verify if (ws != null && m < ws.length) {
ws[idx] = null; rescanned = true;
} finally { for (int i = 0; i <= m; ++i) {
lock.unlock(); ForkJoinWorkerThread u = ws[i];
if (u != null) {
if (u.queueBase != u.queueTop &&
!tryReleaseWaiter())
rescanned = false; // contended
if (w.eventCount != v)
return true;
} }
} }
/**
* Final callback from terminating worker. Removes record of
* worker from array, and adjusts counts. If pool is shutting
* down, tries to complete termination.
*
* @param w the worker
*/
final void workerTerminated(ForkJoinWorkerThread w) {
forgetWorker(w);
decrementWorkerCounts(w.isTrimmed() ? 0 : ONE_RUNNING, ONE_TOTAL);
while (w.stealCount != 0) // collect final count
tryAccumulateStealCount(w);
tryTerminate(false);
} }
if (scanGuard != g || // stale
// Waiting for and signalling events (queueBase != queueTop && !tryReleaseWaiter()))
rescanned = false;
/** if (!rescanned)
* Releases workers blocked on a count not equal to current count. Thread.yield(); // reduce contention
* Normally called after precheck that eventWaiters isn't zero to else
* avoid wasted array checks. Gives up upon a change in count or Thread.interrupted(); // clear before park
* upon releasing four workers, letting others take over.
*/
private void releaseEventWaiters() {
ForkJoinWorkerThread[] ws = workers;
int n = ws.length;
long h = eventWaiters;
int ec = eventCount;
int releases = 4;
ForkJoinWorkerThread w; int id;
while ((id = (((int)h) & WAITER_ID_MASK) - 1) >= 0 &&
(int)(h >>> EVENT_COUNT_SHIFT) != ec &&
id < n && (w = ws[id]) != null) {
if (UNSAFE.compareAndSwapLong(this, eventWaitersOffset,
h, w.nextWaiter)) {
LockSupport.unpark(w);
if (--releases == 0)
break;
} }
if (eventCount != ec) else {
break; w.parked = true; // must recheck
h = eventWaiters; if (w.eventCount != v) {
w.parked = false;
return true;
}
LockSupport.park(this);
rescanned = w.parked = false;
} }
} }
/**
* Tries to advance eventCount and releases waiters. Called only
* from workers.
*/
final void signalWork() {
int c; // try to increment event count -- CAS failure OK
UNSAFE.compareAndSwapInt(this, eventCountOffset, c = eventCount, c+1);
if (eventWaiters != 0L)
releaseEventWaiters();
} }
/** /**
* Adds the given worker to event queue and blocks until * If inactivating worker w has caused pool to become
* terminating or event count advances from the given value * quiescent, check for pool termination, and wait for event
* for up to SHRINK_RATE nanosecs (rescans are unnecessary in
* this case because quiescence reflects consensus about lack
* of work). On timeout, if ctl has not changed, terminate the
* worker. Upon its termination (see deregisterWorker), it may
* wake up another worker to possibly repeat this process.
* *
* @param w the calling worker thread * @param w the calling worker
* @param ec the count * @param currentCtl the ctl value after enqueuing w
* @param prevCtl the ctl value if w terminated
* @param v the eventCount w awaits change
*/ */
private void eventSync(ForkJoinWorkerThread w, int ec) { private void idleAwaitWork(ForkJoinWorkerThread w, long currentCtl,
long nh = (((long)ec) << EVENT_COUNT_SHIFT) | ((long)(w.poolIndex+1)); long prevCtl, int v) {
long h; if (w.eventCount == v) {
while ((runState < SHUTDOWN || !tryTerminate(false)) && if (shutdown)
(((int)(h = eventWaiters) & WAITER_ID_MASK) == 0 || tryTerminate(false);
(int)(h >>> EVENT_COUNT_SHIFT) == ec) && ForkJoinTask.helpExpungeStaleExceptions(); // help clean weak refs
eventCount == ec) { while (ctl == currentCtl) {
if (UNSAFE.compareAndSwapLong(this, eventWaitersOffset, long startTime = System.nanoTime();
w.nextWaiter = h, nh)) { w.parked = true;
awaitEvent(w, ec); if (w.eventCount == v) // must recheck
LockSupport.parkNanos(this, SHRINK_RATE);
w.parked = false;
if (w.eventCount != v)
break; break;
else if (System.nanoTime() - startTime < SHRINK_RATE)
Thread.interrupted(); // spurious wakeup
else if (UNSAFE.compareAndSwapLong(this, ctlOffset,
currentCtl, prevCtl)) {
w.terminate = true; // restore previous
w.eventCount = ((int)currentCtl + EC_UNIT) & E_MASK;
break;
}
} }
} }
} }
// Submissions
/** /**
* Blocks the given worker (that has already been entered as an * Enqueues the given task in the submissionQueue. Same idea as
* event waiter) until terminating or event count advances from * ForkJoinWorkerThread.pushTask except for use of submissionLock.
* the given value. The oldest (first) waiter uses a timed wait to
* occasionally one-by-one shrink the number of workers (to a
* minimum of one) if the pool has not been used for extended
* periods.
* *
* @param w the calling worker thread * @param t the task
* @param ec the count
*/ */
private void awaitEvent(ForkJoinWorkerThread w, int ec) { private void addSubmission(ForkJoinTask<?> t) {
while (eventCount == ec) { final ReentrantLock lock = this.submissionLock;
if (tryAccumulateStealCount(w)) { // transfer while idle lock.lock();
boolean untimed = (w.nextWaiter != 0L || try {
(workerCounts & RUNNING_COUNT_MASK) <= 1); ForkJoinTask<?>[] q; int s, m;
long startTime = untimed ? 0 : System.nanoTime(); if ((q = submissionQueue) != null) { // ignore if queue removed
Thread.interrupted(); // clear/ignore interrupt long u = (((s = queueTop) & (m = q.length-1)) << ASHIFT)+ABASE;
if (w.isTerminating() || eventCount != ec) UNSAFE.putOrderedObject(q, u, t);
break; // recheck after clear queueTop = s + 1;
if (untimed) if (s - queueBase == m)
LockSupport.park(w); growSubmissionQueue();
else {
LockSupport.parkNanos(w, SHRINK_RATE_NANOS);
if (eventCount != ec || w.isTerminating())
break;
if (System.nanoTime() - startTime >= SHRINK_RATE_NANOS)
tryShutdownUnusedWorker(ec);
}
} }
} finally {
lock.unlock();
} }
signalWork();
} }
// Maintaining parallelism // (pollSubmission is defined below with exported methods)
/** /**
* Pushes worker onto the spare stack. * Creates or doubles submissionQueue array.
* Basically identical to ForkJoinWorkerThread version.
*/ */
final void pushSpare(ForkJoinWorkerThread w) { private void growSubmissionQueue() {
int ns = (++w.spareCount << SPARE_COUNT_SHIFT) | (w.poolIndex + 1); ForkJoinTask<?>[] oldQ = submissionQueue;
do {} while (!UNSAFE.compareAndSwapInt(this, spareWaitersOffset, int size = oldQ != null ? oldQ.length << 1 : INITIAL_QUEUE_CAPACITY;
w.nextSpare = spareWaiters,ns)); if (size > MAXIMUM_QUEUE_CAPACITY)
throw new RejectedExecutionException("Queue capacity exceeded");
if (size < INITIAL_QUEUE_CAPACITY)
size = INITIAL_QUEUE_CAPACITY;
ForkJoinTask<?>[] q = submissionQueue = new ForkJoinTask<?>[size];
int mask = size - 1;
int top = queueTop;
int oldMask;
if (oldQ != null && (oldMask = oldQ.length - 1) >= 0) {
for (int b = queueBase; b != top; ++b) {
long u = ((b & oldMask) << ASHIFT) + ABASE;
Object x = UNSAFE.getObjectVolatile(oldQ, u);
if (x != null && UNSAFE.compareAndSwapObject(oldQ, u, x, null))
UNSAFE.putObjectVolatile
(q, ((b & mask) << ASHIFT) + ABASE, x);
}
} }
}
// Blocking support
/** /**
* Tries (once) to resume a spare if the number of running * Tries to increment blockedCount, decrement active count
* threads is less than target. * (sometimes implicitly) and possibly release or create a
* compensating worker in preparation for blocking. Fails
* on contention or termination.
*
* @return true if the caller can block, else should recheck and retry
*/ */
private void tryResumeSpare() { private boolean tryPreBlock() {
int sw, id; int b = blockedCount;
ForkJoinWorkerThread[] ws = workers; if (UNSAFE.compareAndSwapInt(this, blockedCountOffset, b, b + 1)) {
int n = ws.length;
ForkJoinWorkerThread w;
if ((sw = spareWaiters) != 0 &&
(id = (sw & SPARE_ID_MASK) - 1) >= 0 &&
id < n && (w = ws[id]) != null &&
(runState >= TERMINATING ||
(workerCounts & RUNNING_COUNT_MASK) < parallelism) &&
spareWaiters == sw &&
UNSAFE.compareAndSwapInt(this, spareWaitersOffset,
sw, w.nextSpare)) {
int c; // increment running count before resume
do {} while (!UNSAFE.compareAndSwapInt
(this, workerCountsOffset,
c = workerCounts, c + ONE_RUNNING));
if (w.tryUnsuspend())
LockSupport.unpark(w);
else // back out if w was shutdown
decrementWorkerCounts(ONE_RUNNING, 0);
}
}
/**
* Tries to increase the number of running workers if below target
* parallelism: If a spare exists tries to resume it via
* tryResumeSpare. Otherwise, if not enough total workers or all
* existing workers are busy, adds a new worker. In all cases also
* helps wake up releasable workers waiting for work.
*/
private void helpMaintainParallelism() {
int pc = parallelism; int pc = parallelism;
int wc, rs, tc; do {
while (((wc = workerCounts) & RUNNING_COUNT_MASK) < pc && ForkJoinWorkerThread[] ws; ForkJoinWorkerThread w;
(rs = runState) < TERMINATING) { int e, ac, tc, rc, i;
if (spareWaiters != 0) long c = ctl;
tryResumeSpare(); int u = (int)(c >>> 32);
else if ((tc = wc >>> TOTAL_COUNT_SHIFT) >= MAX_WORKERS || if ((e = (int)c) < 0) {
(tc >= pc && (rs & ACTIVE_COUNT_MASK) != tc)) // skip -- terminating
break; // enough total }
else if (runState == rs && workerCounts == wc && else if ((ac = (u >> UAC_SHIFT)) <= 0 && e != 0 &&
UNSAFE.compareAndSwapInt(this, workerCountsOffset, wc, (ws = workers) != null &&
wc + (ONE_RUNNING|ONE_TOTAL))) { (i = ~e & SMASK) < ws.length &&
ForkJoinWorkerThread w = null; (w = ws[i]) != null) {
Throwable fail = null; long nc = ((long)(w.nextWait & E_MASK) |
try { (c & (AC_MASK|TC_MASK)));
w = factory.newThread(this); if (w.eventCount == e &&
} catch (Throwable ex) { UNSAFE.compareAndSwapLong(this, ctlOffset, c, nc)) {
fail = ex; w.eventCount = (e + EC_UNIT) & E_MASK;
} if (w.parked)
if (w == null) { // null or exceptional factory return UNSAFE.unpark(w);
decrementWorkerCounts(ONE_RUNNING, ONE_TOTAL); return true; // release an idle worker
tryTerminate(false); // handle failure during shutdown }
// If originating from an external caller, }
// propagate exception, else ignore else if ((tc = (short)(u >>> UTC_SHIFT)) >= 0 && ac + pc > 1) {
if (fail != null && runState < TERMINATING && long nc = ((c - AC_UNIT) & AC_MASK) | (c & ~AC_MASK);
!(Thread.currentThread() instanceof if (UNSAFE.compareAndSwapLong(this, ctlOffset, c, nc))
ForkJoinWorkerThread)) return true; // no compensation needed
UNSAFE.throwException(fail); }
break; else if (tc + pc < MAX_ID) {
} long nc = ((c + TC_UNIT) & TC_MASK) | (c & ~TC_MASK);
w.start(recordWorker(w), ueh); if (UNSAFE.compareAndSwapLong(this, ctlOffset, c, nc)) {
if ((workerCounts >>> TOTAL_COUNT_SHIFT) >= pc) addWorker();
break; // add at most one unless total below target return true; // create a replacement
}
}
// try to back out on any failure and let caller retry
} while (!UNSAFE.compareAndSwapInt(this, blockedCountOffset,
b = blockedCount, b - 1));
} }
return false;
} }
if (eventWaiters != 0L)
releaseEventWaiters(); /**
* Decrements blockedCount and increments active count
*/
private void postBlock() {
long c;
do {} while (!UNSAFE.compareAndSwapLong(this, ctlOffset, // no mask
c = ctl, c + AC_UNIT));
int b;
do {} while(!UNSAFE.compareAndSwapInt(this, blockedCountOffset,
b = blockedCount, b - 1));
} }
/** /**
* Callback from the oldest waiter in awaitEvent waking up after a * Possibly blocks waiting for the given task to complete, or
* period of non-use. If all workers are idle, tries (once) to * cancels the task if terminating. Fails to wait if contended.
* shutdown an event waiter or a spare, if one exists. Note that
* we don't need CAS or locks here because the method is called
* only from one thread occasionally waking (and even misfires are
* OK). Note that until the shutdown worker fully terminates,
* workerCounts will overestimate total count, which is tolerable.
* *
* @param ec the event count waited on by caller (to abort * @param joinMe the task
* attempt if count has since changed).
*/ */
private void tryShutdownUnusedWorker(int ec) { final void tryAwaitJoin(ForkJoinTask<?> joinMe) {
if (runState == 0 && eventCount == ec) { // only trigger if all idle int s;
ForkJoinWorkerThread[] ws = workers; Thread.interrupted(); // clear interrupts before checking termination
int n = ws.length; if (joinMe.status >= 0) {
ForkJoinWorkerThread w = null; if (tryPreBlock()) {
boolean shutdown = false; joinMe.tryAwaitDone(0L);
int sw; postBlock();
long h;
if ((sw = spareWaiters) != 0) { // prefer killing spares
int id = (sw & SPARE_ID_MASK) - 1;
if (id >= 0 && id < n && (w = ws[id]) != null &&
UNSAFE.compareAndSwapInt(this, spareWaitersOffset,
sw, w.nextSpare))
shutdown = true;
}
else if ((h = eventWaiters) != 0L) {
long nh;
int id = (((int)h) & WAITER_ID_MASK) - 1;
if (id >= 0 && id < n && (w = ws[id]) != null &&
(nh = w.nextWaiter) != 0L && // keep at least one worker
UNSAFE.compareAndSwapLong(this, eventWaitersOffset, h, nh))
shutdown = true;
}
if (w != null && shutdown) {
w.shutdown();
LockSupport.unpark(w);
} }
else if ((ctl & STOP_BIT) != 0L)
joinMe.cancelIgnoringExceptions();
} }
releaseEventWaiters(); // in case of interference
} }
/** /**
* Callback from workers invoked upon each top-level action (i.e., * Possibly blocks the given worker waiting for joinMe to
* stealing a task or taking a submission and running it). * complete or timeout
* Performs one or more of the following:
*
* 1. If the worker is active and either did not run a task
* or there are too many workers, try to set its active status
* to inactive and update activeCount. On contention, we may
* try again in this or a subsequent call.
*
* 2. If not enough total workers, help create some.
*
* 3. If there are too many running workers, suspend this worker
* (first forcing inactive if necessary). If it is not needed,
* it may be shutdown while suspended (via
* tryShutdownUnusedWorker). Otherwise, upon resume it
* rechecks running thread count and need for event sync.
*
* 4. If worker did not run a task, await the next task event via
* eventSync if necessary (first forcing inactivation), upon
* which the worker may be shutdown via
* tryShutdownUnusedWorker. Otherwise, help release any
* existing event waiters that are now releasable,
* *
* @param w the worker * @param joinMe the task
* @param ran true if worker ran a task since last call to this method * @param millis the wait time for underlying Object.wait
*/ */
final void preStep(ForkJoinWorkerThread w, boolean ran) { final void timedAwaitJoin(ForkJoinTask<?> joinMe, long nanos) {
int wec = w.lastEventCount; while (joinMe.status >= 0) {
boolean active = w.active; Thread.interrupted();
boolean inactivate = false; if ((ctl & STOP_BIT) != 0L) {
int pc = parallelism; joinMe.cancelIgnoringExceptions();
while (w.runState == 0) {
int rs = runState;
if (rs >= TERMINATING) { // propagate shutdown
w.shutdown();
break; break;
} }
if ((inactivate || (active && (rs & ACTIVE_COUNT_MASK) >= pc)) && if (tryPreBlock()) {
UNSAFE.compareAndSwapInt(this, runStateOffset, rs, --rs)) { long last = System.nanoTime();
inactivate = active = w.active = false; while (joinMe.status >= 0) {
if (rs == SHUTDOWN) { // all inactive and shut down long millis = TimeUnit.NANOSECONDS.toMillis(nanos);
tryTerminate(false); if (millis <= 0)
continue;
}
}
int wc = workerCounts; // try to suspend as spare
if ((wc & RUNNING_COUNT_MASK) > pc) {
if (!(inactivate |= active) && // must inactivate to suspend
workerCounts == wc &&
UNSAFE.compareAndSwapInt(this, workerCountsOffset,
wc, wc - ONE_RUNNING))
w.suspendAsSpare();
}
else if ((wc >>> TOTAL_COUNT_SHIFT) < pc)
helpMaintainParallelism(); // not enough workers
else if (ran)
break; break;
else { joinMe.tryAwaitDone(millis);
long h = eventWaiters; if (joinMe.status < 0)
int ec = eventCount;
if (h != 0L && (int)(h >>> EVENT_COUNT_SHIFT) != ec)
releaseEventWaiters(); // release others before waiting
else if (ec != wec) {
w.lastEventCount = ec; // no need to wait
break; break;
if ((ctl & STOP_BIT) != 0L) {
joinMe.cancelIgnoringExceptions();
break;
}
long now = System.nanoTime();
nanos -= now - last;
last = now;
} }
else if (!(inactivate |= active)) postBlock();
eventSync(w, wec); // must inactivate before sync break;
} }
} }
} }
/** /**
* Helps and/or blocks awaiting join of the given task. * If necessary, compensates for blocker, and blocks
* See above for explanation.
*
* @param joinMe the task to join
* @param worker the current worker thread
* @param timed true if wait should time out
* @param nanos timeout value if timed
*/ */
final void awaitJoin(ForkJoinTask<?> joinMe, ForkJoinWorkerThread worker, private void awaitBlocker(ManagedBlocker blocker)
boolean timed, long nanos) { throws InterruptedException {
long startTime = timed ? System.nanoTime() : 0L; while (!blocker.isReleasable()) {
int retries = 2 + (parallelism >> 2); // #helpJoins before blocking if (tryPreBlock()) {
boolean running = true; // false when count decremented try {
while (joinMe.status >= 0) { do {} while (!blocker.isReleasable() && !blocker.block());
if (runState >= TERMINATING) { } finally {
joinMe.cancelIgnoringExceptions(); postBlock();
break;
} }
running = worker.helpJoinTask(joinMe, running);
if (joinMe.status < 0)
break; break;
if (retries > 0) {
--retries;
continue;
} }
int wc = workerCounts;
if ((wc & RUNNING_COUNT_MASK) != 0) {
if (running) {
if (!UNSAFE.compareAndSwapInt(this, workerCountsOffset,
wc, wc - ONE_RUNNING))
continue;
running = false;
}
long h = eventWaiters;
if (h != 0L && (int)(h >>> EVENT_COUNT_SHIFT) != eventCount)
releaseEventWaiters();
if ((workerCounts & RUNNING_COUNT_MASK) != 0) {
long ms; int ns;
if (!timed) {
ms = JOIN_TIMEOUT_MILLIS;
ns = 0;
}
else { // at most JOIN_TIMEOUT_MILLIS per wait
long nt = nanos - (System.nanoTime() - startTime);
if (nt <= 0L)
break;
ms = nt / 1000000;
if (ms > JOIN_TIMEOUT_MILLIS) {
ms = JOIN_TIMEOUT_MILLIS;
ns = 0;
} }
else
ns = (int) (nt % 1000000);
}
joinMe.internalAwaitDone(ms, ns);
} }
if (joinMe.status < 0)
break; // Creating, registering and deregistring workers
/**
* Tries to create and start a worker; minimally rolls back counts
* on failure.
*/
private void addWorker() {
Throwable ex = null;
ForkJoinWorkerThread t = null;
try {
t = factory.newThread(this);
} catch (Throwable e) {
ex = e;
}
if (t == null) { // null or exceptional factory return
long c; // adjust counts
do {} while (!UNSAFE.compareAndSwapLong
(this, ctlOffset, c = ctl,
(((c - AC_UNIT) & AC_MASK) |
((c - TC_UNIT) & TC_MASK) |
(c & ~(AC_MASK|TC_MASK)))));
// Propagate exception if originating from an external caller
if (!tryTerminate(false) && ex != null &&
!(Thread.currentThread() instanceof ForkJoinWorkerThread))
UNSAFE.throwException(ex);
} }
helpMaintainParallelism(); else
t.start();
} }
if (!running) {
int c; /**
do {} while (!UNSAFE.compareAndSwapInt * Callback from ForkJoinWorkerThread constructor to assign a
(this, workerCountsOffset, * public name
c = workerCounts, c + ONE_RUNNING)); */
final String nextWorkerName() {
for (int n;;) {
if (UNSAFE.compareAndSwapInt(this, nextWorkerNumberOffset,
n = nextWorkerNumber, ++n))
return workerNamePrefix + n;
} }
} }
/** /**
* Same idea as awaitJoin, but no helping, retries, or timeouts. * Callback from ForkJoinWorkerThread constructor to
* determine its poolIndex and record in workers array.
*
* @param w the worker
* @return the worker's pool index
*/ */
final void awaitBlocker(ManagedBlocker blocker) final int registerWorker(ForkJoinWorkerThread w) {
throws InterruptedException { /*
while (!blocker.isReleasable()) { * In the typical case, a new worker acquires the lock, uses
int wc = workerCounts; * next available index and returns quickly. Since we should
if ((wc & RUNNING_COUNT_MASK) == 0) * not block callers (ultimately from signalWork or
helpMaintainParallelism(); * tryPreBlock) waiting for the lock needed to do this, we
else if (UNSAFE.compareAndSwapInt(this, workerCountsOffset, * instead help release other workers while waiting for the
wc, wc - ONE_RUNNING)) { * lock.
*/
for (int g;;) {
ForkJoinWorkerThread[] ws;
if (((g = scanGuard) & SG_UNIT) == 0 &&
UNSAFE.compareAndSwapInt(this, scanGuardOffset,
g, g | SG_UNIT)) {
int k = nextWorkerIndex;
try { try {
while (!blocker.isReleasable()) { if ((ws = workers) != null) { // ignore on shutdown
long h = eventWaiters; int n = ws.length;
if (h != 0L && if (k < 0 || k >= n || ws[k] != null) {
(int)(h >>> EVENT_COUNT_SHIFT) != eventCount) for (k = 0; k < n && ws[k] != null; ++k)
releaseEventWaiters(); ;
else if ((workerCounts & RUNNING_COUNT_MASK) == 0 && if (k == n)
runState < TERMINATING) ws = workers = Arrays.copyOf(ws, n << 1);
helpMaintainParallelism(); }
else if (blocker.block()) ws[k] = w;
break; nextWorkerIndex = k + 1;
int m = g & SMASK;
g = k >= m? ((m << 1) + 1) & SMASK : g + (SG_UNIT<<1);
} }
} finally { } finally {
int c; scanGuard = g;
do {} while (!UNSAFE.compareAndSwapInt }
(this, workerCountsOffset, return k;
c = workerCounts, c + ONE_RUNNING));
} }
else if ((ws = workers) != null) { // help release others
for (ForkJoinWorkerThread u : ws) {
if (u != null && u.queueBase != u.queueTop) {
if (tryReleaseWaiter())
break; break;
} }
} }
} }
}
}
/**
* Final callback from terminating worker. Removes record of
* worker from array, and adjusts counts. If pool is shutting
* down, tries to complete termination.
*
* @param w the worker
*/
final void deregisterWorker(ForkJoinWorkerThread w, Throwable ex) {
int idx = w.poolIndex;
int sc = w.stealCount;
int steps = 0;
// Remove from array, adjust worker counts and collect steal count.
// We can intermix failed removes or adjusts with steal updates
do {
long s, c;
int g;
if (steps == 0 && ((g = scanGuard) & SG_UNIT) == 0 &&
UNSAFE.compareAndSwapInt(this, scanGuardOffset,
g, g |= SG_UNIT)) {
ForkJoinWorkerThread[] ws = workers;
if (ws != null && idx >= 0 &&
idx < ws.length && ws[idx] == w)
ws[idx] = null; // verify
nextWorkerIndex = idx;
scanGuard = g + SG_UNIT;
steps = 1;
}
if (steps == 1 &&
UNSAFE.compareAndSwapLong(this, ctlOffset, c = ctl,
(((c - AC_UNIT) & AC_MASK) |
((c - TC_UNIT) & TC_MASK) |
(c & ~(AC_MASK|TC_MASK)))))
steps = 2;
if (sc != 0 &&
UNSAFE.compareAndSwapLong(this, stealCountOffset,
s = stealCount, s + sc))
sc = 0;
} while (steps != 2 || sc != 0);
if (!tryTerminate(false)) {
if (ex != null) // possibly replace if died abnormally
signalWork();
else
tryReleaseWaiter();
}
}
// Shutdown and termination
/** /**
* Possibly initiates and/or completes termination. * Possibly initiates and/or completes termination.
...@@ -1161,48 +1249,52 @@ public class ForkJoinPool extends AbstractExecutorService { ...@@ -1161,48 +1249,52 @@ public class ForkJoinPool extends AbstractExecutorService {
* @return true if now terminating or terminated * @return true if now terminating or terminated
*/ */
private boolean tryTerminate(boolean now) { private boolean tryTerminate(boolean now) {
if (now) long c;
advanceRunLevel(SHUTDOWN); // ensure at least SHUTDOWN while (((c = ctl) & STOP_BIT) == 0) {
else if (runState < SHUTDOWN || if (!now) {
!submissionQueue.isEmpty() || if ((int)(c >> AC_SHIFT) != -parallelism)
(runState & ACTIVE_COUNT_MASK) != 0)
return false; return false;
if (!shutdown || blockedCount != 0 || quiescerCount != 0 ||
if (advanceRunLevel(TERMINATING)) queueBase != queueTop) {
if (ctl == c) // staleness check
return false;
continue;
}
}
if (UNSAFE.compareAndSwapLong(this, ctlOffset, c, c | STOP_BIT))
startTerminating(); startTerminating();
}
// Finish now if all threads terminated; else in some subsequent call if ((short)(c >>> TC_SHIFT) == -parallelism) { // signal when 0 workers
if ((workerCounts >>> TOTAL_COUNT_SHIFT) == 0) { final ReentrantLock lock = this.submissionLock;
advanceRunLevel(TERMINATED); lock.lock();
termination.forceTermination(); try {
termination.signalAll();
} finally {
lock.unlock();
}
} }
return true; return true;
} }
/** /**
* Actions on transition to TERMINATING * Runs up to three passes through workers: (0) Setting
* * termination status for each worker, followed by wakeups up to
* Runs up to four passes through workers: (0) shutting down each * queued workers; (1) helping cancel tasks; (2) interrupting
* (without waking up if parked) to quickly spread notifications * lagging threads (likely in external tasks, but possibly also
* without unnecessary bouncing around event queues etc (1) wake * blocked in joins). Each pass repeats previous steps because of
* up and help cancel tasks (2) interrupt (3) mop up races with * potential lagging thread creation.
* interrupted workers
*/ */
private void startTerminating() { private void startTerminating() {
cancelSubmissions(); cancelSubmissions();
for (int passes = 0; passes < 4 && workerCounts != 0; ++passes) { for (int pass = 0; pass < 3; ++pass) {
int c; // advance event count ForkJoinWorkerThread[] ws = workers;
UNSAFE.compareAndSwapInt(this, eventCountOffset, if (ws != null) {
c = eventCount, c+1); for (ForkJoinWorkerThread w : ws) {
eventWaiters = 0L; // clobber lists
spareWaiters = 0;
for (ForkJoinWorkerThread w : workers) {
if (w != null) { if (w != null) {
w.shutdown(); w.terminate = true;
if (passes > 0 && !w.isTerminated()) { if (pass > 0) {
w.cancelTasks(); w.cancelTasks();
LockSupport.unpark(w); if (pass > 1 && !w.isInterrupted()) {
if (passes > 1 && !w.isInterrupted()) {
try { try {
w.interrupt(); w.interrupt();
} catch (SecurityException ignore) { } catch (SecurityException ignore) {
...@@ -1211,47 +1303,78 @@ public class ForkJoinPool extends AbstractExecutorService { ...@@ -1211,47 +1303,78 @@ public class ForkJoinPool extends AbstractExecutorService {
} }
} }
} }
terminateWaiters();
}
} }
} }
/** /**
* Clears out and cancels submissions, ignoring exceptions. * Polls and cancels all submissions. Called only during termination.
*/ */
private void cancelSubmissions() { private void cancelSubmissions() {
ForkJoinTask<?> task; while (queueBase != queueTop) {
while ((task = submissionQueue.poll()) != null) { ForkJoinTask<?> task = pollSubmission();
if (task != null) {
try { try {
task.cancel(false); task.cancel(false);
} catch (Throwable ignore) { } catch (Throwable ignore) {
} }
} }
} }
}
// misc support for ForkJoinWorkerThread
/** /**
* Returns pool number. * Tries to set the termination status of waiting workers, and
* then wakes them up (after which they will terminate).
*/ */
final int getPoolNumber() { private void terminateWaiters() {
return poolNumber; ForkJoinWorkerThread[] ws = workers;
if (ws != null) {
ForkJoinWorkerThread w; long c; int i, e;
int n = ws.length;
while ((i = ~(e = (int)(c = ctl)) & SMASK) < n &&
(w = ws[i]) != null && w.eventCount == (e & E_MASK)) {
if (UNSAFE.compareAndSwapLong(this, ctlOffset, c,
(long)(w.nextWait & E_MASK) |
((c + AC_UNIT) & AC_MASK) |
(c & (TC_MASK|STOP_BIT)))) {
w.terminate = true;
w.eventCount = e + EC_UNIT;
if (w.parked)
UNSAFE.unpark(w);
}
}
} }
}
// misc ForkJoinWorkerThread support
/** /**
* Tries to accumulate steal count from a worker, clearing * Increment or decrement quiescerCount. Needed only to prevent
* the worker's value if successful. * triggering shutdown if a worker is transiently inactive while
* checking quiescence.
* *
* @return true if worker steal count now zero * @param delta 1 for increment, -1 for decrement
*/ */
final boolean tryAccumulateStealCount(ForkJoinWorkerThread w) { final void addQuiescerCount(int delta) {
int sc = w.stealCount; int c;
long c = stealCount; do {} while(!UNSAFE.compareAndSwapInt(this, quiescerCountOffset,
// CAS even if zero, for fence effects c = quiescerCount, c + delta));
if (UNSAFE.compareAndSwapLong(this, stealCountOffset, c, c + sc)) {
if (sc != 0)
w.stealCount = 0;
return true;
} }
return sc == 0;
/**
* Directly increment or decrement active count without
* queuing. This method is used to transiently assert inactivation
* while checking quiescence.
*
* @param delta 1 for increment, -1 for decrement
*/
final void addActiveCount(int delta) {
long d = delta < 0 ? -AC_UNIT : AC_UNIT;
long c;
do {} while (!UNSAFE.compareAndSwapLong(this, ctlOffset, c = ctl,
((c + d) & AC_MASK) |
(c & ~AC_MASK)));
} }
/** /**
...@@ -1259,16 +1382,17 @@ public class ForkJoinPool extends AbstractExecutorService { ...@@ -1259,16 +1382,17 @@ public class ForkJoinPool extends AbstractExecutorService {
* active thread. * active thread.
*/ */
final int idlePerActive() { final int idlePerActive() {
int pc = parallelism; // use parallelism, not rc // Approximate at powers of two for small values, saturate past 4
int ac = runState; // no mask -- artificially boosts during shutdown int p = parallelism;
// Use exact results for small values, saturate past 4 int a = p + (int)(ctl >> AC_SHIFT);
return ((pc <= ac) ? 0 : return (a > (p >>>= 1) ? 0 :
(pc >>> 1 <= ac) ? 1 : a > (p >>>= 1) ? 1 :
(pc >>> 2 <= ac) ? 3 : a > (p >>>= 1) ? 2 :
pc >>> 3); a > (p >>>= 1) ? 4 :
8);
} }
// Public and protected methods // Exported methods
// Constructors // Constructors
...@@ -1337,49 +1461,42 @@ public class ForkJoinPool extends AbstractExecutorService { ...@@ -1337,49 +1461,42 @@ public class ForkJoinPool extends AbstractExecutorService {
checkPermission(); checkPermission();
if (factory == null) if (factory == null)
throw new NullPointerException(); throw new NullPointerException();
if (parallelism <= 0 || parallelism > MAX_WORKERS) if (parallelism <= 0 || parallelism > MAX_ID)
throw new IllegalArgumentException(); throw new IllegalArgumentException();
this.parallelism = parallelism; this.parallelism = parallelism;
this.factory = factory; this.factory = factory;
this.ueh = handler; this.ueh = handler;
this.locallyFifo = asyncMode; this.locallyFifo = asyncMode;
int arraySize = initialArraySizeFor(parallelism); long np = (long)(-parallelism); // offset ctl counts
this.workers = new ForkJoinWorkerThread[arraySize]; this.ctl = ((np << AC_SHIFT) & AC_MASK) | ((np << TC_SHIFT) & TC_MASK);
this.submissionQueue = new LinkedTransferQueue<ForkJoinTask<?>>(); this.submissionQueue = new ForkJoinTask<?>[INITIAL_QUEUE_CAPACITY];
this.workerLock = new ReentrantLock(); // initialize workers array with room for 2*parallelism if possible
this.termination = new Phaser(1); int n = parallelism << 1;
this.poolNumber = poolNumberGenerator.incrementAndGet(); if (n >= MAX_ID)
} n = MAX_ID;
else { // See Hackers Delight, sec 3.2, where n < (1 << 16)
/** n |= n >>> 1; n |= n >>> 2; n |= n >>> 4; n |= n >>> 8;
* Returns initial power of two size for workers array. }
* @param pc the initial parallelism level workers = new ForkJoinWorkerThread[n + 1];
*/ this.submissionLock = new ReentrantLock();
private static int initialArraySizeFor(int pc) { this.termination = submissionLock.newCondition();
// If possible, initially allocate enough space for one spare StringBuilder sb = new StringBuilder("ForkJoinPool-");
int size = pc < MAX_WORKERS ? pc + 1 : MAX_WORKERS; sb.append(poolNumberGenerator.incrementAndGet());
// See Hackers Delight, sec 3.2. We know MAX_WORKERS < (1 >>> 16) sb.append("-worker-");
size |= size >>> 1; this.workerNamePrefix = sb.toString();
size |= size >>> 2;
size |= size >>> 4;
size |= size >>> 8;
return size + 1;
} }
// Execution methods // Execution methods
/**
* Submits task and creates, starts, or resumes some workers if necessary
*/
private <T> void doSubmit(ForkJoinTask<T> task) {
submissionQueue.offer(task);
int c; // try to increment event count -- CAS failure OK
UNSAFE.compareAndSwapInt(this, eventCountOffset, c = eventCount, c+1);
helpMaintainParallelism();
}
/** /**
* Performs the given task, returning its result upon completion. * Performs the given task, returning its result upon completion.
* If the computation encounters an unchecked Exception or Error,
* it is rethrown as the outcome of this invocation. Rethrown
* exceptions behave in the same way as regular exceptions, but,
* when possible, contain stack traces (as displayed for example
* using {@code ex.printStackTrace()}) of both the current thread
* as well as the thread actually encountering the exception;
* minimally only the latter.
* *
* @param task the task * @param task the task
* @return the task's result * @return the task's result
...@@ -1388,16 +1505,16 @@ public class ForkJoinPool extends AbstractExecutorService { ...@@ -1388,16 +1505,16 @@ public class ForkJoinPool extends AbstractExecutorService {
* scheduled for execution * scheduled for execution
*/ */
public <T> T invoke(ForkJoinTask<T> task) { public <T> T invoke(ForkJoinTask<T> task) {
Thread t = Thread.currentThread();
if (task == null) if (task == null)
throw new NullPointerException(); throw new NullPointerException();
if (runState >= SHUTDOWN) if (shutdown)
throw new RejectedExecutionException(); throw new RejectedExecutionException();
Thread t = Thread.currentThread();
if ((t instanceof ForkJoinWorkerThread) && if ((t instanceof ForkJoinWorkerThread) &&
((ForkJoinWorkerThread)t).pool == this) ((ForkJoinWorkerThread)t).pool == this)
return task.invoke(); // bypass submit if in same pool return task.invoke(); // bypass submit if in same pool
else { else {
doSubmit(task); addSubmission(task);
return task.join(); return task.join();
} }
} }
...@@ -1407,14 +1524,15 @@ public class ForkJoinPool extends AbstractExecutorService { ...@@ -1407,14 +1524,15 @@ public class ForkJoinPool extends AbstractExecutorService {
* computation in the current pool, else submits as external task. * computation in the current pool, else submits as external task.
*/ */
private <T> void forkOrSubmit(ForkJoinTask<T> task) { private <T> void forkOrSubmit(ForkJoinTask<T> task) {
if (runState >= SHUTDOWN) ForkJoinWorkerThread w;
throw new RejectedExecutionException();
Thread t = Thread.currentThread(); Thread t = Thread.currentThread();
if (shutdown)
throw new RejectedExecutionException();
if ((t instanceof ForkJoinWorkerThread) && if ((t instanceof ForkJoinWorkerThread) &&
((ForkJoinWorkerThread)t).pool == this) (w = (ForkJoinWorkerThread)t).pool == this)
task.fork(); w.pushTask(task);
else else
doSubmit(task); addSubmission(task);
} }
/** /**
...@@ -1571,7 +1689,7 @@ public class ForkJoinPool extends AbstractExecutorService { ...@@ -1571,7 +1689,7 @@ public class ForkJoinPool extends AbstractExecutorService {
* @return the number of worker threads * @return the number of worker threads
*/ */
public int getPoolSize() { public int getPoolSize() {
return workerCounts >>> TOTAL_COUNT_SHIFT; return parallelism + (short)(ctl >>> TC_SHIFT);
} }
/** /**
...@@ -1593,7 +1711,8 @@ public class ForkJoinPool extends AbstractExecutorService { ...@@ -1593,7 +1711,8 @@ public class ForkJoinPool extends AbstractExecutorService {
* @return the number of worker threads * @return the number of worker threads
*/ */
public int getRunningThreadCount() { public int getRunningThreadCount() {
return workerCounts & RUNNING_COUNT_MASK; int r = parallelism + (int)(ctl >> AC_SHIFT);
return r <= 0? 0 : r; // suppress momentarily negative values
} }
/** /**
...@@ -1604,7 +1723,8 @@ public class ForkJoinPool extends AbstractExecutorService { ...@@ -1604,7 +1723,8 @@ public class ForkJoinPool extends AbstractExecutorService {
* @return the number of active threads * @return the number of active threads
*/ */
public int getActiveThreadCount() { public int getActiveThreadCount() {
return runState & ACTIVE_COUNT_MASK; int r = parallelism + (int)(ctl >> AC_SHIFT) + blockedCount;
return r <= 0? 0 : r; // suppress momentarily negative values
} }
/** /**
...@@ -1619,7 +1739,7 @@ public class ForkJoinPool extends AbstractExecutorService { ...@@ -1619,7 +1739,7 @@ public class ForkJoinPool extends AbstractExecutorService {
* @return {@code true} if all threads are currently idle * @return {@code true} if all threads are currently idle
*/ */
public boolean isQuiescent() { public boolean isQuiescent() {
return (runState & ACTIVE_COUNT_MASK) == 0; return parallelism + (int)(ctl >> AC_SHIFT) + blockedCount == 0;
} }
/** /**
...@@ -1649,21 +1769,25 @@ public class ForkJoinPool extends AbstractExecutorService { ...@@ -1649,21 +1769,25 @@ public class ForkJoinPool extends AbstractExecutorService {
*/ */
public long getQueuedTaskCount() { public long getQueuedTaskCount() {
long count = 0; long count = 0;
for (ForkJoinWorkerThread w : workers) ForkJoinWorkerThread[] ws;
if ((short)(ctl >>> TC_SHIFT) > -parallelism &&
(ws = workers) != null) {
for (ForkJoinWorkerThread w : ws)
if (w != null) if (w != null)
count += w.getQueueSize(); count -= w.queueBase - w.queueTop; // must read base first
}
return count; return count;
} }
/** /**
* Returns an estimate of the number of tasks submitted to this * Returns an estimate of the number of tasks submitted to this
* pool that have not yet begun executing. This method takes time * pool that have not yet begun executing. This method may take
* proportional to the number of submissions. * time proportional to the number of submissions.
* *
* @return the number of queued submissions * @return the number of queued submissions
*/ */
public int getQueuedSubmissionCount() { public int getQueuedSubmissionCount() {
return submissionQueue.size(); return -queueBase + queueTop;
} }
/** /**
...@@ -1673,7 +1797,7 @@ public class ForkJoinPool extends AbstractExecutorService { ...@@ -1673,7 +1797,7 @@ public class ForkJoinPool extends AbstractExecutorService {
* @return {@code true} if there are any queued submissions * @return {@code true} if there are any queued submissions
*/ */
public boolean hasQueuedSubmissions() { public boolean hasQueuedSubmissions() {
return !submissionQueue.isEmpty(); return queueBase != queueTop;
} }
/** /**
...@@ -1684,7 +1808,19 @@ public class ForkJoinPool extends AbstractExecutorService { ...@@ -1684,7 +1808,19 @@ public class ForkJoinPool extends AbstractExecutorService {
* @return the next submission, or {@code null} if none * @return the next submission, or {@code null} if none
*/ */
protected ForkJoinTask<?> pollSubmission() { protected ForkJoinTask<?> pollSubmission() {
return submissionQueue.poll(); ForkJoinTask<?> t; ForkJoinTask<?>[] q; int b, i;
while ((b = queueBase) != queueTop &&
(q = submissionQueue) != null &&
(i = (q.length - 1) & b) >= 0) {
long u = (i << ASHIFT) + ABASE;
if ((t = q[i]) != null &&
queueBase == b &&
UNSAFE.compareAndSwapObject(q, u, t, null)) {
queueBase = b + 1;
return t;
}
}
return null;
} }
/** /**
...@@ -1705,10 +1841,21 @@ public class ForkJoinPool extends AbstractExecutorService { ...@@ -1705,10 +1841,21 @@ public class ForkJoinPool extends AbstractExecutorService {
* @return the number of elements transferred * @return the number of elements transferred
*/ */
protected int drainTasksTo(Collection<? super ForkJoinTask<?>> c) { protected int drainTasksTo(Collection<? super ForkJoinTask<?>> c) {
int count = submissionQueue.drainTo(c); int count = 0;
for (ForkJoinWorkerThread w : workers) while (queueBase != queueTop) {
ForkJoinTask<?> t = pollSubmission();
if (t != null) {
c.add(t);
++count;
}
}
ForkJoinWorkerThread[] ws;
if ((short)(ctl >>> TC_SHIFT) > -parallelism &&
(ws = workers) != null) {
for (ForkJoinWorkerThread w : ws)
if (w != null) if (w != null)
count += w.drainTasksTo(c); count += w.drainTasksTo(c);
}
return count; return count;
} }
...@@ -1723,14 +1870,20 @@ public class ForkJoinPool extends AbstractExecutorService { ...@@ -1723,14 +1870,20 @@ public class ForkJoinPool extends AbstractExecutorService {
long st = getStealCount(); long st = getStealCount();
long qt = getQueuedTaskCount(); long qt = getQueuedTaskCount();
long qs = getQueuedSubmissionCount(); long qs = getQueuedSubmissionCount();
int wc = workerCounts;
int tc = wc >>> TOTAL_COUNT_SHIFT;
int rc = wc & RUNNING_COUNT_MASK;
int pc = parallelism; int pc = parallelism;
int rs = runState; long c = ctl;
int ac = rs & ACTIVE_COUNT_MASK; int tc = pc + (short)(c >>> TC_SHIFT);
int rc = pc + (int)(c >> AC_SHIFT);
if (rc < 0) // ignore transient negative
rc = 0;
int ac = rc + blockedCount;
String level;
if ((c & STOP_BIT) != 0)
level = (tc == 0)? "Terminated" : "Terminating";
else
level = shutdown? "Shutting down" : "Running";
return super.toString() + return super.toString() +
"[" + runLevelToString(rs) + "[" + level +
", parallelism = " + pc + ", parallelism = " + pc +
", size = " + tc + ", size = " + tc +
", active = " + ac + ", active = " + ac +
...@@ -1741,13 +1894,6 @@ public class ForkJoinPool extends AbstractExecutorService { ...@@ -1741,13 +1894,6 @@ public class ForkJoinPool extends AbstractExecutorService {
"]"; "]";
} }
private static String runLevelToString(int s) {
return ((s & TERMINATED) != 0 ? "Terminated" :
((s & TERMINATING) != 0 ? "Terminating" :
((s & SHUTDOWN) != 0 ? "Shutting down" :
"Running")));
}
/** /**
* Initiates an orderly shutdown in which previously submitted * Initiates an orderly shutdown in which previously submitted
* tasks are executed, but no new tasks will be accepted. * tasks are executed, but no new tasks will be accepted.
...@@ -1762,7 +1908,7 @@ public class ForkJoinPool extends AbstractExecutorService { ...@@ -1762,7 +1908,7 @@ public class ForkJoinPool extends AbstractExecutorService {
*/ */
public void shutdown() { public void shutdown() {
checkPermission(); checkPermission();
advanceRunLevel(SHUTDOWN); shutdown = true;
tryTerminate(false); tryTerminate(false);
} }
...@@ -1784,6 +1930,7 @@ public class ForkJoinPool extends AbstractExecutorService { ...@@ -1784,6 +1930,7 @@ public class ForkJoinPool extends AbstractExecutorService {
*/ */
public List<Runnable> shutdownNow() { public List<Runnable> shutdownNow() {
checkPermission(); checkPermission();
shutdown = true;
tryTerminate(true); tryTerminate(true);
return Collections.emptyList(); return Collections.emptyList();
} }
...@@ -1794,7 +1941,9 @@ public class ForkJoinPool extends AbstractExecutorService { ...@@ -1794,7 +1941,9 @@ public class ForkJoinPool extends AbstractExecutorService {
* @return {@code true} if all tasks have completed following shut down * @return {@code true} if all tasks have completed following shut down
*/ */
public boolean isTerminated() { public boolean isTerminated() {
return runState >= TERMINATED; long c = ctl;
return ((c & STOP_BIT) != 0L &&
(short)(c >>> TC_SHIFT) == -parallelism);
} }
/** /**
...@@ -1811,14 +1960,16 @@ public class ForkJoinPool extends AbstractExecutorService { ...@@ -1811,14 +1960,16 @@ public class ForkJoinPool extends AbstractExecutorService {
* @return {@code true} if terminating but not yet terminated * @return {@code true} if terminating but not yet terminated
*/ */
public boolean isTerminating() { public boolean isTerminating() {
return (runState & (TERMINATING|TERMINATED)) == TERMINATING; long c = ctl;
return ((c & STOP_BIT) != 0L &&
(short)(c >>> TC_SHIFT) != -parallelism);
} }
/** /**
* Returns true if terminating or terminated. Used by ForkJoinWorkerThread. * Returns true if terminating or terminated. Used by ForkJoinWorkerThread.
*/ */
final boolean isAtLeastTerminating() { final boolean isAtLeastTerminating() {
return runState >= TERMINATING; return (ctl & STOP_BIT) != 0L;
} }
/** /**
...@@ -1827,7 +1978,7 @@ public class ForkJoinPool extends AbstractExecutorService { ...@@ -1827,7 +1978,7 @@ public class ForkJoinPool extends AbstractExecutorService {
* @return {@code true} if this pool has been shut down * @return {@code true} if this pool has been shut down
*/ */
public boolean isShutdown() { public boolean isShutdown() {
return runState >= SHUTDOWN; return shutdown;
} }
/** /**
...@@ -1843,12 +1994,20 @@ public class ForkJoinPool extends AbstractExecutorService { ...@@ -1843,12 +1994,20 @@ public class ForkJoinPool extends AbstractExecutorService {
*/ */
public boolean awaitTermination(long timeout, TimeUnit unit) public boolean awaitTermination(long timeout, TimeUnit unit)
throws InterruptedException { throws InterruptedException {
long nanos = unit.toNanos(timeout);
final ReentrantLock lock = this.submissionLock;
lock.lock();
try { try {
termination.awaitAdvanceInterruptibly(0, timeout, unit); for (;;) {
} catch (TimeoutException ex) { if (isTerminated())
return true;
if (nanos <= 0)
return false; return false;
nanos = termination.awaitNanos(nanos);
}
} finally {
lock.unlock();
} }
return true;
} }
/** /**
...@@ -1859,13 +2018,15 @@ public class ForkJoinPool extends AbstractExecutorService { ...@@ -1859,13 +2018,15 @@ public class ForkJoinPool extends AbstractExecutorService {
* {@code isReleasable} must return {@code true} if blocking is * {@code isReleasable} must return {@code true} if blocking is
* not necessary. Method {@code block} blocks the current thread * not necessary. Method {@code block} blocks the current thread
* if necessary (perhaps internally invoking {@code isReleasable} * if necessary (perhaps internally invoking {@code isReleasable}
* before actually blocking). The unusual methods in this API * before actually blocking). These actions are performed by any
* accommodate synchronizers that may, but don't usually, block * thread invoking {@link ForkJoinPool#managedBlock}. The
* for long periods. Similarly, they allow more efficient internal * unusual methods in this API accommodate synchronizers that may,
* handling of cases in which additional workers may be, but * but don't usually, block for long periods. Similarly, they
* usually are not, needed to ensure sufficient parallelism. * allow more efficient internal handling of cases in which
* Toward this end, implementations of method {@code isReleasable} * additional workers may be, but usually are not, needed to
* must be amenable to repeated invocation. * ensure sufficient parallelism. Toward this end,
* implementations of method {@code isReleasable} must be amenable
* to repeated invocation.
* *
* <p>For example, here is a ManagedBlocker based on a * <p>For example, here is a ManagedBlocker based on a
* ReentrantLock: * ReentrantLock:
...@@ -1967,29 +2128,47 @@ public class ForkJoinPool extends AbstractExecutorService { ...@@ -1967,29 +2128,47 @@ public class ForkJoinPool extends AbstractExecutorService {
} }
// Unsafe mechanics // Unsafe mechanics
private static final sun.misc.Unsafe UNSAFE;
private static final sun.misc.Unsafe UNSAFE = sun.misc.Unsafe.getUnsafe(); private static final long ctlOffset;
private static final long workerCountsOffset = private static final long stealCountOffset;
objectFieldOffset("workerCounts", ForkJoinPool.class); private static final long blockedCountOffset;
private static final long runStateOffset = private static final long quiescerCountOffset;
objectFieldOffset("runState", ForkJoinPool.class); private static final long scanGuardOffset;
private static final long eventCountOffset = private static final long nextWorkerNumberOffset;
objectFieldOffset("eventCount", ForkJoinPool.class); private static final long ABASE;
private static final long eventWaitersOffset = private static final int ASHIFT;
objectFieldOffset("eventWaiters", ForkJoinPool.class);
private static final long stealCountOffset = static {
objectFieldOffset("stealCount", ForkJoinPool.class); poolNumberGenerator = new AtomicInteger();
private static final long spareWaitersOffset = workerSeedGenerator = new Random();
objectFieldOffset("spareWaiters", ForkJoinPool.class); modifyThreadPermission = new RuntimePermission("modifyThread");
defaultForkJoinWorkerThreadFactory =
private static long objectFieldOffset(String field, Class<?> klazz) { new DefaultForkJoinWorkerThreadFactory();
int s;
try { try {
return UNSAFE.objectFieldOffset(klazz.getDeclaredField(field)); UNSAFE = sun.misc.Unsafe.getUnsafe();
} catch (NoSuchFieldException e) { Class k = ForkJoinPool.class;
// Convert Exception to corresponding Error ctlOffset = UNSAFE.objectFieldOffset
NoSuchFieldError error = new NoSuchFieldError(field); (k.getDeclaredField("ctl"));
error.initCause(e); stealCountOffset = UNSAFE.objectFieldOffset
throw error; (k.getDeclaredField("stealCount"));
} blockedCountOffset = UNSAFE.objectFieldOffset
(k.getDeclaredField("blockedCount"));
quiescerCountOffset = UNSAFE.objectFieldOffset
(k.getDeclaredField("quiescerCount"));
scanGuardOffset = UNSAFE.objectFieldOffset
(k.getDeclaredField("scanGuard"));
nextWorkerNumberOffset = UNSAFE.objectFieldOffset
(k.getDeclaredField("nextWorkerNumber"));
Class a = ForkJoinTask[].class;
ABASE = UNSAFE.arrayBaseOffset(a);
s = UNSAFE.arrayIndexScale(a);
} catch (Exception e) {
throw new Error(e);
}
if ((s & (s-1)) != 0)
throw new Error("data type scale not a power of two");
ASHIFT = 31 - Integer.numberOfLeadingZeros(s);
} }
} }
...@@ -41,7 +41,8 @@ import java.util.Collections; ...@@ -41,7 +41,8 @@ import java.util.Collections;
import java.util.List; import java.util.List;
import java.util.RandomAccess; import java.util.RandomAccess;
import java.util.Map; import java.util.Map;
import java.util.WeakHashMap; import java.lang.ref.WeakReference;
import java.lang.ref.ReferenceQueue;
import java.util.concurrent.Callable; import java.util.concurrent.Callable;
import java.util.concurrent.CancellationException; import java.util.concurrent.CancellationException;
import java.util.concurrent.ExecutionException; import java.util.concurrent.ExecutionException;
...@@ -52,6 +53,8 @@ import java.util.concurrent.RejectedExecutionException; ...@@ -52,6 +53,8 @@ import java.util.concurrent.RejectedExecutionException;
import java.util.concurrent.RunnableFuture; import java.util.concurrent.RunnableFuture;
import java.util.concurrent.TimeUnit; import java.util.concurrent.TimeUnit;
import java.util.concurrent.TimeoutException; import java.util.concurrent.TimeoutException;
import java.util.concurrent.locks.ReentrantLock;
import java.lang.reflect.Constructor;
/** /**
* Abstract base class for tasks that run within a {@link ForkJoinPool}. * Abstract base class for tasks that run within a {@link ForkJoinPool}.
...@@ -95,7 +98,11 @@ import java.util.concurrent.TimeoutException; ...@@ -95,7 +98,11 @@ import java.util.concurrent.TimeoutException;
* rethrown to callers attempting to join them. These exceptions may * rethrown to callers attempting to join them. These exceptions may
* additionally include {@link RejectedExecutionException} stemming * additionally include {@link RejectedExecutionException} stemming
* from internal resource exhaustion, such as failure to allocate * from internal resource exhaustion, such as failure to allocate
* internal task queues. * internal task queues. Rethrown exceptions behave in the same way as
* regular exceptions, but, when possible, contain stack traces (as
* displayed for example using {@code ex.printStackTrace()}) of both
* the thread that initiated the computation as well as the thread
* actually encountering the exception; minimally only the latter.
* *
* <p>The primary method for awaiting completion and extracting * <p>The primary method for awaiting completion and extracting
* results of a task is {@link #join}, but there are several variants: * results of a task is {@link #join}, but there are several variants:
...@@ -192,8 +199,7 @@ public abstract class ForkJoinTask<V> implements Future<V>, Serializable { ...@@ -192,8 +199,7 @@ public abstract class ForkJoinTask<V> implements Future<V>, Serializable {
* status maintenance (2) execution and awaiting completion (3) * status maintenance (2) execution and awaiting completion (3)
* user-level methods that additionally report results. This is * user-level methods that additionally report results. This is
* sometimes hard to see because this file orders exported methods * sometimes hard to see because this file orders exported methods
* in a way that flows well in javadocs. In particular, most * in a way that flows well in javadocs.
* join mechanics are in method quietlyJoin, below.
*/ */
/* /*
...@@ -215,91 +221,67 @@ public abstract class ForkJoinTask<V> implements Future<V>, Serializable { ...@@ -215,91 +221,67 @@ public abstract class ForkJoinTask<V> implements Future<V>, Serializable {
/** The run status of this task */ /** The run status of this task */
volatile int status; // accessed directly by pool and workers volatile int status; // accessed directly by pool and workers
private static final int NORMAL = -1; private static final int NORMAL = -1;
private static final int CANCELLED = -2; private static final int CANCELLED = -2;
private static final int EXCEPTIONAL = -3; private static final int EXCEPTIONAL = -3;
private static final int SIGNAL = 1; private static final int SIGNAL = 1;
/**
* Table of exceptions thrown by tasks, to enable reporting by
* callers. Because exceptions are rare, we don't directly keep
* them with task objects, but instead use a weak ref table. Note
* that cancellation exceptions don't appear in the table, but are
* instead recorded as status values.
* TODO: Use ConcurrentReferenceHashMap
*/
static final Map<ForkJoinTask<?>, Throwable> exceptionMap =
Collections.synchronizedMap
(new WeakHashMap<ForkJoinTask<?>, Throwable>());
// Maintaining completion status
/** /**
* Marks completion and wakes up threads waiting to join this task, * Marks completion and wakes up threads waiting to join this task,
* also clearing signal request bits. * also clearing signal request bits.
* *
* @param completion one of NORMAL, CANCELLED, EXCEPTIONAL * @param completion one of NORMAL, CANCELLED, EXCEPTIONAL
* @return completion status on exit
*/ */
private void setCompletion(int completion) { private int setCompletion(int completion) {
int s; for (int s;;) {
while ((s = status) >= 0) { if ((s = status) < 0)
return s;
if (UNSAFE.compareAndSwapInt(this, statusOffset, s, completion)) { if (UNSAFE.compareAndSwapInt(this, statusOffset, s, completion)) {
if (s != 0) if (s != 0)
synchronized (this) { notifyAll(); } synchronized (this) { notifyAll(); }
break; return completion;
} }
} }
} }
/** /**
* Records exception and sets exceptional completion. * Tries to block a worker thread until completed or timed out.
* Uses Object.wait time argument conventions.
* May fail on contention or interrupt.
* *
* @return status on exit * @param millis if > 0, wait time.
*/ */
private void setExceptionalCompletion(Throwable rex) { final void tryAwaitDone(long millis) {
exceptionMap.put(this, rex); int s;
setCompletion(EXCEPTIONAL); try {
} if (((s = status) > 0 ||
(s == 0 &&
/** UNSAFE.compareAndSwapInt(this, statusOffset, 0, SIGNAL))) &&
* Blocks a worker thread until completed or timed out. Called status > 0) {
* only by pool.
*/
final void internalAwaitDone(long millis, int nanos) {
int s = status;
if ((s == 0 &&
UNSAFE.compareAndSwapInt(this, statusOffset, 0, SIGNAL)) ||
s > 0) {
try { // the odd construction reduces lock bias effects
synchronized (this) { synchronized (this) {
if (status > 0) if (status > 0)
wait(millis, nanos); wait(millis);
else
notifyAll();
} }
} catch (InterruptedException ie) {
cancelIfTerminating();
} }
} catch (InterruptedException ie) {
// caller must check termination
} }
} }
/** /**
* Blocks a non-worker-thread until completion. * Blocks a non-worker-thread until completion.
* @return status upon completion
*/ */
private void externalAwaitDone() { private int externalAwaitDone() {
if (status >= 0) { int s;
if ((s = status) >= 0) {
boolean interrupted = false; boolean interrupted = false;
synchronized (this) { synchronized (this) {
for (;;) { while ((s = status) >= 0) {
int s = status;
if (s == 0) if (s == 0)
UNSAFE.compareAndSwapInt(this, statusOffset, UNSAFE.compareAndSwapInt(this, statusOffset,
0, SIGNAL); 0, SIGNAL);
else if (s < 0) {
notifyAll();
break;
}
else { else {
try { try {
wait(); wait();
...@@ -312,54 +294,309 @@ public abstract class ForkJoinTask<V> implements Future<V>, Serializable { ...@@ -312,54 +294,309 @@ public abstract class ForkJoinTask<V> implements Future<V>, Serializable {
if (interrupted) if (interrupted)
Thread.currentThread().interrupt(); Thread.currentThread().interrupt();
} }
return s;
} }
/** /**
* Blocks a non-worker-thread until completion or interruption or timeout. * Blocks a non-worker-thread until completion or interruption or timeout.
*/ */
private void externalInterruptibleAwaitDone(boolean timed, long nanos) private int externalInterruptibleAwaitDone(long millis)
throws InterruptedException { throws InterruptedException {
int s;
if (Thread.interrupted()) if (Thread.interrupted())
throw new InterruptedException(); throw new InterruptedException();
if (status >= 0) { if ((s = status) >= 0) {
long startTime = timed ? System.nanoTime() : 0L;
synchronized (this) { synchronized (this) {
for (;;) { while ((s = status) >= 0) {
long nt;
int s = status;
if (s == 0) if (s == 0)
UNSAFE.compareAndSwapInt(this, statusOffset, UNSAFE.compareAndSwapInt(this, statusOffset,
0, SIGNAL); 0, SIGNAL);
else if (s < 0) { else {
notifyAll(); wait(millis);
if (millis > 0L)
break; break;
} }
else if (!timed)
wait();
else if ((nt = nanos - (System.nanoTime()-startTime)) > 0L)
wait(nt / 1000000, (int)(nt % 1000000));
else
break;
} }
} }
} }
return s;
} }
/** /**
* Unless done, calls exec and records status if completed, but * Primary execution method for stolen tasks. Unless done, calls
* doesn't wait for completion otherwise. Primary execution method * exec and records status if completed, but doesn't wait for
* for ForkJoinWorkerThread. * completion otherwise.
*/ */
final void quietlyExec() { final void doExec() {
if (status >= 0) {
boolean completed;
try { try {
if (status < 0 || !exec()) completed = exec();
return;
} catch (Throwable rex) { } catch (Throwable rex) {
setExceptionalCompletion(rex); setExceptionalCompletion(rex);
return; return;
} }
if (completed)
setCompletion(NORMAL); // must be outside try block setCompletion(NORMAL); // must be outside try block
} }
}
/**
* Primary mechanics for join, get, quietlyJoin.
* @return status upon completion
*/
private int doJoin() {
Thread t; ForkJoinWorkerThread w; int s; boolean completed;
if ((t = Thread.currentThread()) instanceof ForkJoinWorkerThread) {
if ((s = status) < 0)
return s;
if ((w = (ForkJoinWorkerThread)t).unpushTask(this)) {
try {
completed = exec();
} catch (Throwable rex) {
return setExceptionalCompletion(rex);
}
if (completed)
return setCompletion(NORMAL);
}
return w.joinTask(this);
}
else
return externalAwaitDone();
}
/**
* Primary mechanics for invoke, quietlyInvoke.
* @return status upon completion
*/
private int doInvoke() {
int s; boolean completed;
if ((s = status) < 0)
return s;
try {
completed = exec();
} catch (Throwable rex) {
return setExceptionalCompletion(rex);
}
if (completed)
return setCompletion(NORMAL);
else
return doJoin();
}
// Exception table support
/**
* Table of exceptions thrown by tasks, to enable reporting by
* callers. Because exceptions are rare, we don't directly keep
* them with task objects, but instead use a weak ref table. Note
* that cancellation exceptions don't appear in the table, but are
* instead recorded as status values.
*
* Note: These statics are initialized below in static block.
*/
private static final ExceptionNode[] exceptionTable;
private static final ReentrantLock exceptionTableLock;
private static final ReferenceQueue<Object> exceptionTableRefQueue;
/**
* Fixed capacity for exceptionTable.
*/
private static final int EXCEPTION_MAP_CAPACITY = 32;
/**
* Key-value nodes for exception table. The chained hash table
* uses identity comparisons, full locking, and weak references
* for keys. The table has a fixed capacity because it only
* maintains task exceptions long enough for joiners to access
* them, so should never become very large for sustained
* periods. However, since we do not know when the last joiner
* completes, we must use weak references and expunge them. We do
* so on each operation (hence full locking). Also, some thread in
* any ForkJoinPool will call helpExpungeStaleExceptions when its
* pool becomes isQuiescent.
*/
static final class ExceptionNode extends WeakReference<ForkJoinTask<?>>{
final Throwable ex;
ExceptionNode next;
final long thrower; // use id not ref to avoid weak cycles
ExceptionNode(ForkJoinTask<?> task, Throwable ex, ExceptionNode next) {
super(task, exceptionTableRefQueue);
this.ex = ex;
this.next = next;
this.thrower = Thread.currentThread().getId();
}
}
/**
* Records exception and sets exceptional completion.
*
* @return status on exit
*/
private int setExceptionalCompletion(Throwable ex) {
int h = System.identityHashCode(this);
final ReentrantLock lock = exceptionTableLock;
lock.lock();
try {
expungeStaleExceptions();
ExceptionNode[] t = exceptionTable;
int i = h & (t.length - 1);
for (ExceptionNode e = t[i]; ; e = e.next) {
if (e == null) {
t[i] = new ExceptionNode(this, ex, t[i]);
break;
}
if (e.get() == this) // already present
break;
}
} finally {
lock.unlock();
}
return setCompletion(EXCEPTIONAL);
}
/**
* Removes exception node and clears status
*/
private void clearExceptionalCompletion() {
int h = System.identityHashCode(this);
final ReentrantLock lock = exceptionTableLock;
lock.lock();
try {
ExceptionNode[] t = exceptionTable;
int i = h & (t.length - 1);
ExceptionNode e = t[i];
ExceptionNode pred = null;
while (e != null) {
ExceptionNode next = e.next;
if (e.get() == this) {
if (pred == null)
t[i] = next;
else
pred.next = next;
break;
}
pred = e;
e = next;
}
expungeStaleExceptions();
status = 0;
} finally {
lock.unlock();
}
}
/**
* Returns a rethrowable exception for the given task, if
* available. To provide accurate stack traces, if the exception
* was not thrown by the current thread, we try to create a new
* exception of the same type as the one thrown, but with the
* recorded exception as its cause. If there is no such
* constructor, we instead try to use a no-arg constructor,
* followed by initCause, to the same effect. If none of these
* apply, or any fail due to other exceptions, we return the
* recorded exception, which is still correct, although it may
* contain a misleading stack trace.
*
* @return the exception, or null if none
*/
private Throwable getThrowableException() {
if (status != EXCEPTIONAL)
return null;
int h = System.identityHashCode(this);
ExceptionNode e;
final ReentrantLock lock = exceptionTableLock;
lock.lock();
try {
expungeStaleExceptions();
ExceptionNode[] t = exceptionTable;
e = t[h & (t.length - 1)];
while (e != null && e.get() != this)
e = e.next;
} finally {
lock.unlock();
}
Throwable ex;
if (e == null || (ex = e.ex) == null)
return null;
if (e.thrower != Thread.currentThread().getId()) {
Class ec = ex.getClass();
try {
Constructor<?> noArgCtor = null;
Constructor<?>[] cs = ec.getConstructors();// public ctors only
for (int i = 0; i < cs.length; ++i) {
Constructor<?> c = cs[i];
Class<?>[] ps = c.getParameterTypes();
if (ps.length == 0)
noArgCtor = c;
else if (ps.length == 1 && ps[0] == Throwable.class)
return (Throwable)(c.newInstance(ex));
}
if (noArgCtor != null) {
Throwable wx = (Throwable)(noArgCtor.newInstance());
wx.initCause(ex);
return wx;
}
} catch (Exception ignore) {
}
}
return ex;
}
/**
* Poll stale refs and remove them. Call only while holding lock.
*/
private static void expungeStaleExceptions() {
for (Object x; (x = exceptionTableRefQueue.poll()) != null;) {
if (x instanceof ExceptionNode) {
ForkJoinTask<?> key = ((ExceptionNode)x).get();
ExceptionNode[] t = exceptionTable;
int i = System.identityHashCode(key) & (t.length - 1);
ExceptionNode e = t[i];
ExceptionNode pred = null;
while (e != null) {
ExceptionNode next = e.next;
if (e == x) {
if (pred == null)
t[i] = next;
else
pred.next = next;
break;
}
pred = e;
e = next;
}
}
}
}
/**
* If lock is available, poll stale refs and remove them.
* Called from ForkJoinPool when pools become quiescent.
*/
static final void helpExpungeStaleExceptions() {
final ReentrantLock lock = exceptionTableLock;
if (lock.tryLock()) {
try {
expungeStaleExceptions();
} finally {
lock.unlock();
}
}
}
/**
* Report the result of invoke or join; called only upon
* non-normal return of internal versions.
*/
private V reportResult() {
int s; Throwable ex;
if ((s = status) == CANCELLED)
throw new CancellationException();
if (s == EXCEPTIONAL && (ex = getThrowableException()) != null)
UNSAFE.throwException(ex);
return getRawResult();
}
// public methods // public methods
...@@ -399,10 +636,9 @@ public abstract class ForkJoinTask<V> implements Future<V>, Serializable { ...@@ -399,10 +636,9 @@ public abstract class ForkJoinTask<V> implements Future<V>, Serializable {
* @return the computed result * @return the computed result
*/ */
public final V join() { public final V join() {
quietlyJoin(); if (doJoin() != NORMAL)
Throwable ex; return reportResult();
if (status < NORMAL && (ex = getException()) != null) else
UNSAFE.throwException(ex);
return getRawResult(); return getRawResult();
} }
...@@ -415,10 +651,9 @@ public abstract class ForkJoinTask<V> implements Future<V>, Serializable { ...@@ -415,10 +651,9 @@ public abstract class ForkJoinTask<V> implements Future<V>, Serializable {
* @return the computed result * @return the computed result
*/ */
public final V invoke() { public final V invoke() {
quietlyInvoke(); if (doInvoke() != NORMAL)
Throwable ex; return reportResult();
if (status < NORMAL && (ex = getException()) != null) else
UNSAFE.throwException(ex);
return getRawResult(); return getRawResult();
} }
...@@ -483,24 +718,18 @@ public abstract class ForkJoinTask<V> implements Future<V>, Serializable { ...@@ -483,24 +718,18 @@ public abstract class ForkJoinTask<V> implements Future<V>, Serializable {
} }
else if (i != 0) else if (i != 0)
t.fork(); t.fork();
else { else if (t.doInvoke() < NORMAL && ex == null)
t.quietlyInvoke();
if (ex == null && t.status < NORMAL)
ex = t.getException(); ex = t.getException();
} }
}
for (int i = 1; i <= last; ++i) { for (int i = 1; i <= last; ++i) {
ForkJoinTask<?> t = tasks[i]; ForkJoinTask<?> t = tasks[i];
if (t != null) { if (t != null) {
if (ex != null) if (ex != null)
t.cancel(false); t.cancel(false);
else { else if (t.doJoin() < NORMAL && ex == null)
t.quietlyJoin();
if (ex == null && t.status < NORMAL)
ex = t.getException(); ex = t.getException();
} }
} }
}
if (ex != null) if (ex != null)
UNSAFE.throwException(ex); UNSAFE.throwException(ex);
} }
...@@ -546,24 +775,18 @@ public abstract class ForkJoinTask<V> implements Future<V>, Serializable { ...@@ -546,24 +775,18 @@ public abstract class ForkJoinTask<V> implements Future<V>, Serializable {
} }
else if (i != 0) else if (i != 0)
t.fork(); t.fork();
else { else if (t.doInvoke() < NORMAL && ex == null)
t.quietlyInvoke();
if (ex == null && t.status < NORMAL)
ex = t.getException(); ex = t.getException();
} }
}
for (int i = 1; i <= last; ++i) { for (int i = 1; i <= last; ++i) {
ForkJoinTask<?> t = ts.get(i); ForkJoinTask<?> t = ts.get(i);
if (t != null) { if (t != null) {
if (ex != null) if (ex != null)
t.cancel(false); t.cancel(false);
else { else if (t.doJoin() < NORMAL && ex == null)
t.quietlyJoin();
if (ex == null && t.status < NORMAL)
ex = t.getException(); ex = t.getException();
} }
} }
}
if (ex != null) if (ex != null)
UNSAFE.throwException(ex); UNSAFE.throwException(ex);
return tasks; return tasks;
...@@ -597,8 +820,7 @@ public abstract class ForkJoinTask<V> implements Future<V>, Serializable { ...@@ -597,8 +820,7 @@ public abstract class ForkJoinTask<V> implements Future<V>, Serializable {
* @return {@code true} if this task is now cancelled * @return {@code true} if this task is now cancelled
*/ */
public boolean cancel(boolean mayInterruptIfRunning) { public boolean cancel(boolean mayInterruptIfRunning) {
setCompletion(CANCELLED); return setCompletion(CANCELLED) == CANCELLED;
return status == CANCELLED;
} }
/** /**
...@@ -614,21 +836,6 @@ public abstract class ForkJoinTask<V> implements Future<V>, Serializable { ...@@ -614,21 +836,6 @@ public abstract class ForkJoinTask<V> implements Future<V>, Serializable {
} }
} }
/**
* Cancels if current thread is a terminating worker thread,
* ignoring any exceptions thrown by cancel.
*/
final void cancelIfTerminating() {
Thread t = Thread.currentThread();
if ((t instanceof ForkJoinWorkerThread) &&
((ForkJoinWorkerThread) t).isTerminating()) {
try {
cancel(false);
} catch (Throwable ignore) {
}
}
}
public final boolean isDone() { public final boolean isDone() {
return status < 0; return status < 0;
} }
...@@ -668,7 +875,7 @@ public abstract class ForkJoinTask<V> implements Future<V>, Serializable { ...@@ -668,7 +875,7 @@ public abstract class ForkJoinTask<V> implements Future<V>, Serializable {
int s = status; int s = status;
return ((s >= NORMAL) ? null : return ((s >= NORMAL) ? null :
(s == CANCELLED) ? new CancellationException() : (s == CANCELLED) ? new CancellationException() :
exceptionMap.get(this)); getThrowableException());
} }
/** /**
...@@ -726,19 +933,13 @@ public abstract class ForkJoinTask<V> implements Future<V>, Serializable { ...@@ -726,19 +933,13 @@ public abstract class ForkJoinTask<V> implements Future<V>, Serializable {
* member of a ForkJoinPool and was interrupted while waiting * member of a ForkJoinPool and was interrupted while waiting
*/ */
public final V get() throws InterruptedException, ExecutionException { public final V get() throws InterruptedException, ExecutionException {
Thread t = Thread.currentThread(); int s = (Thread.currentThread() instanceof ForkJoinWorkerThread) ?
if (t instanceof ForkJoinWorkerThread) doJoin() : externalInterruptibleAwaitDone(0L);
quietlyJoin();
else
externalInterruptibleAwaitDone(false, 0L);
int s = status;
if (s != NORMAL) {
Throwable ex; Throwable ex;
if (s == CANCELLED) if (s == CANCELLED)
throw new CancellationException(); throw new CancellationException();
if (s == EXCEPTIONAL && (ex = exceptionMap.get(this)) != null) if (s == EXCEPTIONAL && (ex = getThrowableException()) != null)
throw new ExecutionException(ex); throw new ExecutionException(ex);
}
return getRawResult(); return getRawResult();
} }
...@@ -758,20 +959,39 @@ public abstract class ForkJoinTask<V> implements Future<V>, Serializable { ...@@ -758,20 +959,39 @@ public abstract class ForkJoinTask<V> implements Future<V>, Serializable {
*/ */
public final V get(long timeout, TimeUnit unit) public final V get(long timeout, TimeUnit unit)
throws InterruptedException, ExecutionException, TimeoutException { throws InterruptedException, ExecutionException, TimeoutException {
long nanos = unit.toNanos(timeout);
Thread t = Thread.currentThread(); Thread t = Thread.currentThread();
if (t instanceof ForkJoinWorkerThread) if (t instanceof ForkJoinWorkerThread) {
((ForkJoinWorkerThread)t).joinTask(this, true, nanos); ForkJoinWorkerThread w = (ForkJoinWorkerThread) t;
else long nanos = unit.toNanos(timeout);
externalInterruptibleAwaitDone(true, nanos); if (status >= 0) {
boolean completed = false;
if (w.unpushTask(this)) {
try {
completed = exec();
} catch (Throwable rex) {
setExceptionalCompletion(rex);
}
}
if (completed)
setCompletion(NORMAL);
else if (status >= 0 && nanos > 0)
w.pool.timedAwaitJoin(this, nanos);
}
}
else {
long millis = unit.toMillis(timeout);
if (millis > 0)
externalInterruptibleAwaitDone(millis);
}
int s = status; int s = status;
if (s != NORMAL) { if (s != NORMAL) {
Throwable ex; Throwable ex;
if (s == CANCELLED) if (s == CANCELLED)
throw new CancellationException(); throw new CancellationException();
if (s == EXCEPTIONAL && (ex = exceptionMap.get(this)) != null) if (s != EXCEPTIONAL)
throw new ExecutionException(ex);
throw new TimeoutException(); throw new TimeoutException();
if ((ex = getThrowableException()) != null)
throw new ExecutionException(ex);
} }
return getRawResult(); return getRawResult();
} }
...@@ -783,28 +1003,7 @@ public abstract class ForkJoinTask<V> implements Future<V>, Serializable { ...@@ -783,28 +1003,7 @@ public abstract class ForkJoinTask<V> implements Future<V>, Serializable {
* known to have aborted. * known to have aborted.
*/ */
public final void quietlyJoin() { public final void quietlyJoin() {
Thread t; doJoin();
if ((t = Thread.currentThread()) instanceof ForkJoinWorkerThread) {
ForkJoinWorkerThread w = (ForkJoinWorkerThread) t;
if (status >= 0) {
if (w.unpushTask(this)) {
boolean completed;
try {
completed = exec();
} catch (Throwable rex) {
setExceptionalCompletion(rex);
return;
}
if (completed) {
setCompletion(NORMAL);
return;
}
}
w.joinTask(this, false, 0L);
}
}
else
externalAwaitDone();
} }
/** /**
...@@ -813,19 +1012,7 @@ public abstract class ForkJoinTask<V> implements Future<V>, Serializable { ...@@ -813,19 +1012,7 @@ public abstract class ForkJoinTask<V> implements Future<V>, Serializable {
* exception. * exception.
*/ */
public final void quietlyInvoke() { public final void quietlyInvoke() {
if (status >= 0) { doInvoke();
boolean completed;
try {
completed = exec();
} catch (Throwable rex) {
setExceptionalCompletion(rex);
return;
}
if (completed)
setCompletion(NORMAL);
else
quietlyJoin();
}
} }
/** /**
...@@ -864,7 +1051,8 @@ public abstract class ForkJoinTask<V> implements Future<V>, Serializable { ...@@ -864,7 +1051,8 @@ public abstract class ForkJoinTask<V> implements Future<V>, Serializable {
*/ */
public void reinitialize() { public void reinitialize() {
if (status == EXCEPTIONAL) if (status == EXCEPTIONAL)
exceptionMap.remove(this); clearExceptionalCompletion();
else
status = 0; status = 0;
} }
...@@ -1176,23 +1364,23 @@ public abstract class ForkJoinTask<V> implements Future<V>, Serializable { ...@@ -1176,23 +1364,23 @@ public abstract class ForkJoinTask<V> implements Future<V>, Serializable {
s.defaultReadObject(); s.defaultReadObject();
Object ex = s.readObject(); Object ex = s.readObject();
if (ex != null) if (ex != null)
setExceptionalCompletion((Throwable) ex); setExceptionalCompletion((Throwable)ex);
} }
// Unsafe mechanics // Unsafe mechanics
private static final sun.misc.Unsafe UNSAFE;
private static final sun.misc.Unsafe UNSAFE = sun.misc.Unsafe.getUnsafe(); private static final long statusOffset;
private static final long statusOffset = static {
objectFieldOffset("status", ForkJoinTask.class); exceptionTableLock = new ReentrantLock();
exceptionTableRefQueue = new ReferenceQueue<Object>();
private static long objectFieldOffset(String field, Class<?> klazz) { exceptionTable = new ExceptionNode[EXCEPTION_MAP_CAPACITY];
try { try {
return UNSAFE.objectFieldOffset(klazz.getDeclaredField(field)); UNSAFE = sun.misc.Unsafe.getUnsafe();
} catch (NoSuchFieldException e) { statusOffset = UNSAFE.objectFieldOffset
// Convert Exception to corresponding Error (ForkJoinTask.class.getDeclaredField("status"));
NoSuchFieldError error = new NoSuchFieldError(field); } catch (Exception e) {
error.initCause(e); throw new Error(e);
throw error;
} }
} }
} }
...@@ -35,9 +35,7 @@ ...@@ -35,9 +35,7 @@
package java.util.concurrent; package java.util.concurrent;
import java.util.Random;
import java.util.Collection; import java.util.Collection;
import java.util.concurrent.locks.LockSupport;
import java.util.concurrent.RejectedExecutionException; import java.util.concurrent.RejectedExecutionException;
/** /**
...@@ -84,33 +82,38 @@ public class ForkJoinWorkerThread extends Thread { ...@@ -84,33 +82,38 @@ public class ForkJoinWorkerThread extends Thread {
* a footprint as possible even in programs generating huge * a footprint as possible even in programs generating huge
* numbers of tasks. To accomplish this, we shift the CAS * numbers of tasks. To accomplish this, we shift the CAS
* arbitrating pop vs deq (steal) from being on the indices * arbitrating pop vs deq (steal) from being on the indices
* ("base" and "sp") to the slots themselves (mainly via method * ("queueBase" and "queueTop") to the slots themselves (mainly
* "casSlotNull()"). So, both a successful pop and deq mainly * via method "casSlotNull()"). So, both a successful pop and deq
* entail a CAS of a slot from non-null to null. Because we rely * mainly entail a CAS of a slot from non-null to null. Because
* on CASes of references, we do not need tag bits on base or sp. * we rely on CASes of references, we do not need tag bits on
* They are simple ints as used in any circular array-based queue * queueBase or queueTop. They are simple ints as used in any
* (see for example ArrayDeque). Updates to the indices must * circular array-based queue (see for example ArrayDeque).
* still be ordered in a way that guarantees that sp == base means * Updates to the indices must still be ordered in a way that
* the queue is empty, but otherwise may err on the side of * guarantees that queueTop == queueBase means the queue is empty,
* possibly making the queue appear nonempty when a push, pop, or * but otherwise may err on the side of possibly making the queue
* deq have not fully committed. Note that this means that the deq * appear nonempty when a push, pop, or deq have not fully
* operation, considered individually, is not wait-free. One thief * committed. Note that this means that the deq operation,
* cannot successfully continue until another in-progress one (or, * considered individually, is not wait-free. One thief cannot
* if previously empty, a push) completes. However, in the * successfully continue until another in-progress one (or, if
* previously empty, a push) completes. However, in the
* aggregate, we ensure at least probabilistic non-blockingness. * aggregate, we ensure at least probabilistic non-blockingness.
* If an attempted steal fails, a thief always chooses a different * If an attempted steal fails, a thief always chooses a different
* random victim target to try next. So, in order for one thief to * random victim target to try next. So, in order for one thief to
* progress, it suffices for any in-progress deq or new push on * progress, it suffices for any in-progress deq or new push on
* any empty queue to complete. One reason this works well here is * any empty queue to complete.
* that apparently-nonempty often means soon-to-be-stealable,
* which gives threads a chance to set activation status if
* necessary before stealing.
* *
* This approach also enables support for "async mode" where local * This approach also enables support for "async mode" where local
* task processing is in FIFO, not LIFO order; simply by using a * task processing is in FIFO, not LIFO order; simply by using a
* version of deq rather than pop when locallyFifo is true (as set * version of deq rather than pop when locallyFifo is true (as set
* by the ForkJoinPool). This allows use in message-passing * by the ForkJoinPool). This allows use in message-passing
* frameworks in which tasks are never joined. * frameworks in which tasks are never joined. However neither
* mode considers affinities, loads, cache localities, etc, so
* rarely provide the best possible performance on a given
* machine, but portably provide good throughput by averaging over
* these factors. (Further, even if we did try to use such
* information, we do not usually have a basis for exploiting
* it. For example, some sets of tasks profit from cache
* affinities, but others are harmed by cache pollution effects.)
* *
* When a worker would otherwise be blocked waiting to join a * When a worker would otherwise be blocked waiting to join a
* task, it first tries a form of linear helping: Each worker * task, it first tries a form of linear helping: Each worker
...@@ -137,29 +140,26 @@ public class ForkJoinWorkerThread extends Thread { ...@@ -137,29 +140,26 @@ public class ForkJoinWorkerThread extends Thread {
* miss links in the chain during long-lived tasks, GC stalls etc * miss links in the chain during long-lived tasks, GC stalls etc
* (which is OK since blocking in such cases is usually a good * (which is OK since blocking in such cases is usually a good
* idea). (4) We bound the number of attempts to find work (see * idea). (4) We bound the number of attempts to find work (see
* MAX_HELP_DEPTH) and fall back to suspending the worker and if * MAX_HELP) and fall back to suspending the worker and if
* necessary replacing it with a spare (see * necessary replacing it with another.
* ForkJoinPool.awaitJoin).
* *
* Efficient implementation of these algorithms currently relies * Efficient implementation of these algorithms currently relies
* on an uncomfortable amount of "Unsafe" mechanics. To maintain * on an uncomfortable amount of "Unsafe" mechanics. To maintain
* correct orderings, reads and writes of variable base require * correct orderings, reads and writes of variable queueBase
* volatile ordering. Variable sp does not require volatile * require volatile ordering. Variable queueTop need not be
* writes but still needs store-ordering, which we accomplish by * volatile because non-local reads always follow those of
* pre-incrementing sp before filling the slot with an ordered * queueBase. Similarly, because they are protected by volatile
* store. (Pre-incrementing also enables backouts used in * queueBase reads, reads of the queue array and its slots by
* joinTask.) Because they are protected by volatile base reads, * other threads do not need volatile load semantics, but writes
* reads of the queue array and its slots by other threads do not * (in push) require store order and CASes (in pop and deq)
* need volatile load semantics, but writes (in push) require * require (volatile) CAS semantics. (Michael, Saraswat, and
* store order and CASes (in pop and deq) require (volatile) CAS * Vechev's algorithm has similar properties, but without support
* semantics. (Michael, Saraswat, and Vechev's algorithm has * for nulling slots.) Since these combinations aren't supported
* similar properties, but without support for nulling slots.) * using ordinary volatiles, the only way to accomplish these
* Since these combinations aren't supported using ordinary * efficiently is to use direct Unsafe calls. (Using external
* volatiles, the only way to accomplish these efficiently is to * AtomicIntegers and AtomicReferenceArrays for the indices and
* use direct Unsafe calls. (Using external AtomicIntegers and * array is significantly slower because of memory locality and
* AtomicReferenceArrays for the indices and array is * indirection effects.)
* significantly slower because of memory locality and indirection
* effects.)
* *
* Further, performance on most platforms is very sensitive to * Further, performance on most platforms is very sensitive to
* placement and sizing of the (resizable) queue array. Even * placement and sizing of the (resizable) queue array. Even
...@@ -167,30 +167,13 @@ public class ForkJoinWorkerThread extends Thread { ...@@ -167,30 +167,13 @@ public class ForkJoinWorkerThread extends Thread {
* initial size must be large enough to counteract cache * initial size must be large enough to counteract cache
* contention effects across multiple queues (especially in the * contention effects across multiple queues (especially in the
* presence of GC cardmarking). Also, to improve thread-locality, * presence of GC cardmarking). Also, to improve thread-locality,
* queues are initialized after starting. All together, these * queues are initialized after starting.
* low-level implementation choices produce as much as a factor of
* 4 performance improvement compared to naive implementations,
* and enable the processing of billions of tasks per second,
* sometimes at the expense of ugliness.
*/ */
/** /**
* Generator for initial random seeds for random victim * Mask for pool indices encoded as shorts
* selection. This is used only to create initial seeds. Random
* steals use a cheaper xorshift generator per steal attempt. We
* expect only rare contention on seedGenerator, so just use a
* plain Random.
*/ */
private static final Random seedGenerator = new Random(); private static final int SMASK = 0xffff;
/**
* The maximum stolen->joining link depth allowed in helpJoinTask.
* Depths for legitimate chains are unbounded, but we use a fixed
* constant to avoid (otherwise unchecked) cycles and bound
* staleness of traversal parameters at the expense of sometimes
* blocking when we could be helping.
*/
private static final int MAX_HELP_DEPTH = 8;
/** /**
* Capacity of work-stealing queue array upon initialization. * Capacity of work-stealing queue array upon initialization.
...@@ -200,12 +183,19 @@ public class ForkJoinWorkerThread extends Thread { ...@@ -200,12 +183,19 @@ public class ForkJoinWorkerThread extends Thread {
private static final int INITIAL_QUEUE_CAPACITY = 1 << 13; private static final int INITIAL_QUEUE_CAPACITY = 1 << 13;
/** /**
* Maximum work-stealing queue array size. Must be less than or * Maximum size for queue array. Must be a power of two
* equal to 1 << (31 - width of array entry) to ensure lack of * less than or equal to 1 << (31 - width of array entry) to
* index wraparound. The value is set in the static block * ensure lack of index wraparound, but is capped at a lower
* at the end of this file after obtaining width. * value to help users trap runaway computations.
*/
private static final int MAXIMUM_QUEUE_CAPACITY = 1 << 24; // 16M
/**
* The work-stealing queue array. Size must be a power of two.
* Initialized when started (as oposed to when constructed), to
* improve memory locality.
*/ */
private static final int MAXIMUM_QUEUE_CAPACITY; ForkJoinTask<?>[] queue;
/** /**
* The pool this thread works in. Accessed directly by ForkJoinTask. * The pool this thread works in. Accessed directly by ForkJoinTask.
...@@ -213,25 +203,19 @@ public class ForkJoinWorkerThread extends Thread { ...@@ -213,25 +203,19 @@ public class ForkJoinWorkerThread extends Thread {
final ForkJoinPool pool; final ForkJoinPool pool;
/** /**
* The work-stealing queue array. Size must be a power of two. * Index (mod queue.length) of next queue slot to push to or pop
* Initialized in onStart, to improve memory locality. * from. It is written only by owner thread, and accessed by other
* threads only after reading (volatile) queueBase. Both queueTop
* and queueBase are allowed to wrap around on overflow, but
* (queueTop - queueBase) still estimates size.
*/ */
private ForkJoinTask<?>[] queue; int queueTop;
/** /**
* Index (mod queue.length) of least valid queue slot, which is * Index (mod queue.length) of least valid queue slot, which is
* always the next position to steal from if nonempty. * always the next position to steal from if nonempty.
*/ */
private volatile int base; volatile int queueBase;
/**
* Index (mod queue.length) of next queue slot to push to or pop
* from. It is written only by owner thread, and accessed by other
* threads only after reading (volatile) base. Both sp and base
* are allowed to wrap around on overflow, but (sp - base) still
* estimates size.
*/
private int sp;
/** /**
* The index of most recent stealer, used as a hint to avoid * The index of most recent stealer, used as a hint to avoid
...@@ -240,92 +224,68 @@ public class ForkJoinWorkerThread extends Thread { ...@@ -240,92 +224,68 @@ public class ForkJoinWorkerThread extends Thread {
* of them (usually the most current). Declared non-volatile, * of them (usually the most current). Declared non-volatile,
* relying on other prevailing sync to keep reasonably current. * relying on other prevailing sync to keep reasonably current.
*/ */
private int stealHint; int stealHint;
/**
* Run state of this worker. In addition to the usual run levels,
* tracks if this worker is suspended as a spare, and if it was
* killed (trimmed) while suspended. However, "active" status is
* maintained separately and modified only in conjunction with
* CASes of the pool's runState (which are currently sadly
* manually inlined for performance.) Accessed directly by pool
* to simplify checks for normal (zero) status.
*/
volatile int runState;
private static final int TERMINATING = 0x01;
private static final int TERMINATED = 0x02;
private static final int SUSPENDED = 0x04; // inactive spare
private static final int TRIMMED = 0x08; // killed while suspended
/** /**
* Number of steals. Directly accessed (and reset) by * Index of this worker in pool array. Set once by pool before
* pool.tryAccumulateStealCount when idle. * running, and accessed directly by pool to locate this worker in
* its workers array.
*/ */
int stealCount; final int poolIndex;
/** /**
* Seed for random number generator for choosing steal victims. * Encoded record for pool task waits. Usages are always
* Uses Marsaglia xorshift. Must be initialized as nonzero. * surrounded by volatile reads/writes
*/ */
private int seed; int nextWait;
/** /**
* Activity status. When true, this worker is considered active. * Complement of poolIndex, offset by count of entries of task
* Accessed directly by pool. Must be false upon construction. * waits. Accessed by ForkJoinPool to manage event waiters.
*/ */
boolean active; volatile int eventCount;
/** /**
* True if use local fifo, not default lifo, for local polling. * Seed for random number generator for choosing steal victims.
* Shadows value from ForkJoinPool. * Uses Marsaglia xorshift. Must be initialized as nonzero.
*/ */
private final boolean locallyFifo; int seed;
/** /**
* Index of this worker in pool array. Set once by pool before * Number of steals. Directly accessed (and reset) by pool when
* running, and accessed directly by pool to locate this worker in * idle.
* its workers array.
*/ */
int poolIndex; int stealCount;
/** /**
* The last pool event waited for. Accessed only by pool in * True if this worker should or did terminate
* callback methods invoked within this thread.
*/ */
int lastEventCount; volatile boolean terminate;
/** /**
* Encoded index and event count of next event waiter. Accessed * Set to true before LockSupport.park; false on return
* only by ForkJoinPool for managing event waiters.
*/ */
volatile long nextWaiter; volatile boolean parked;
/** /**
* Number of times this thread suspended as spare. Accessed only * True if use local fifo, not default lifo, for local polling.
* by pool. * Shadows value from ForkJoinPool.
*/ */
int spareCount; final boolean locallyFifo;
/** /**
* Encoded index and count of next spare waiter. Accessed only * The task most recently stolen from another worker (or
* by ForkJoinPool for managing spares. * submission queue). All uses are surrounded by enough volatile
* reads/writes to maintain as non-volatile.
*/ */
volatile int nextSpare; ForkJoinTask<?> currentSteal;
/** /**
* The task currently being joined, set only when actively trying * The task currently being joined, set only when actively trying
* to help other stealers in helpJoinTask. Written only by this * to help other stealers in helpJoinTask. All uses are surrounded
* thread, but read by others. * by enough volatile reads/writes to maintain as non-volatile.
*/ */
private volatile ForkJoinTask<?> currentJoin; ForkJoinTask<?> currentJoin;
/**
* The task most recently stolen from another worker (or
* submission queue). Written only by this thread, but read by
* others.
*/
private volatile ForkJoinTask<?> currentSteal;
/** /**
* Creates a ForkJoinWorkerThread operating in the given pool. * Creates a ForkJoinWorkerThread operating in the given pool.
...@@ -334,24 +294,19 @@ public class ForkJoinWorkerThread extends Thread { ...@@ -334,24 +294,19 @@ public class ForkJoinWorkerThread extends Thread {
* @throws NullPointerException if pool is null * @throws NullPointerException if pool is null
*/ */
protected ForkJoinWorkerThread(ForkJoinPool pool) { protected ForkJoinWorkerThread(ForkJoinPool pool) {
super(pool.nextWorkerName());
this.pool = pool; this.pool = pool;
this.locallyFifo = pool.locallyFifo; int k = pool.registerWorker(this);
setDaemon(true); poolIndex = k;
// To avoid exposing construction details to subclasses, eventCount = ~k & SMASK; // clear wait count
// remaining initialization is in start() and onStart() locallyFifo = pool.locallyFifo;
} Thread.UncaughtExceptionHandler ueh = pool.ueh;
/**
* Performs additional initialization and starts this thread.
*/
final void start(int poolIndex, UncaughtExceptionHandler ueh) {
this.poolIndex = poolIndex;
if (ueh != null) if (ueh != null)
setUncaughtExceptionHandler(ueh); setUncaughtExceptionHandler(ueh);
start(); setDaemon(true);
} }
// Public/protected methods // Public methods
/** /**
* Returns the pool hosting this thread. * Returns the pool hosting this thread.
...@@ -375,6 +330,25 @@ public class ForkJoinWorkerThread extends Thread { ...@@ -375,6 +330,25 @@ public class ForkJoinWorkerThread extends Thread {
return poolIndex; return poolIndex;
} }
// Randomization
/**
* Computes next value for random victim probes and backoffs.
* Scans don't require a very high quality generator, but also not
* a crummy one. Marsaglia xor-shift is cheap and works well
* enough. Note: This is manually inlined in FJP.scan() to avoid
* writes inside busy loops.
*/
private int nextSeed() {
int r = seed;
r ^= r << 13;
r ^= r >>> 17;
r ^= r << 5;
return seed = r;
}
// Run State management
/** /**
* Initializes internal state after construction but before * Initializes internal state after construction but before
* processing any tasks. If you override this method, you must * processing any tasks. If you override this method, you must
...@@ -385,15 +359,9 @@ public class ForkJoinWorkerThread extends Thread { ...@@ -385,15 +359,9 @@ public class ForkJoinWorkerThread extends Thread {
* processing tasks. * processing tasks.
*/ */
protected void onStart() { protected void onStart() {
int rs = seedGenerator.nextInt();
seed = (rs == 0) ? 1 : rs; // seed must be nonzero
// Allocate name string and arrays in this thread
String pid = Integer.toString(pool.getPoolNumber());
String wid = Integer.toString(poolIndex);
setName("ForkJoinPool-" + pid + "-worker-" + wid);
queue = new ForkJoinTask<?>[INITIAL_QUEUE_CAPACITY]; queue = new ForkJoinTask<?>[INITIAL_QUEUE_CAPACITY];
int r = pool.workerSeedGenerator.nextInt();
seed = (r == 0)? 1 : r; // must be nonzero
} }
/** /**
...@@ -406,16 +374,9 @@ public class ForkJoinWorkerThread extends Thread { ...@@ -406,16 +374,9 @@ public class ForkJoinWorkerThread extends Thread {
*/ */
protected void onTermination(Throwable exception) { protected void onTermination(Throwable exception) {
try { try {
ForkJoinPool p = pool; terminate = true;
if (active) {
int a; // inline p.tryDecrementActiveCount
active = false;
do {} while (!UNSAFE.compareAndSwapInt
(p, poolRunStateOffset, a = p.runState, a - 1));
}
cancelTasks(); cancelTasks();
setTerminated(); pool.deregisterWorker(this, exception);
p.workerTerminated(this);
} catch (Throwable ex) { // Shouldn't ever happen } catch (Throwable ex) { // Shouldn't ever happen
if (exception == null) // but if so, at least rethrown if (exception == null) // but if so, at least rethrown
exception = ex; exception = ex;
...@@ -434,7 +395,7 @@ public class ForkJoinWorkerThread extends Thread { ...@@ -434,7 +395,7 @@ public class ForkJoinWorkerThread extends Thread {
Throwable exception = null; Throwable exception = null;
try { try {
onStart(); onStart();
mainLoop(); pool.work(this);
} catch (Throwable ex) { } catch (Throwable ex) {
exception = ex; exception = ex;
} finally { } finally {
...@@ -442,81 +403,6 @@ public class ForkJoinWorkerThread extends Thread { ...@@ -442,81 +403,6 @@ public class ForkJoinWorkerThread extends Thread {
} }
} }
// helpers for run()
/**
* Finds and executes tasks, and checks status while running.
*/
private void mainLoop() {
boolean ran = false; // true if ran a task on last step
ForkJoinPool p = pool;
for (;;) {
p.preStep(this, ran);
if (runState != 0)
break;
ran = tryExecSteal() || tryExecSubmission();
}
}
/**
* Tries to steal a task and execute it.
*
* @return true if ran a task
*/
private boolean tryExecSteal() {
ForkJoinTask<?> t;
if ((t = scan()) != null) {
t.quietlyExec();
UNSAFE.putOrderedObject(this, currentStealOffset, null);
if (sp != base)
execLocalTasks();
return true;
}
return false;
}
/**
* If a submission exists, try to activate and run it.
*
* @return true if ran a task
*/
private boolean tryExecSubmission() {
ForkJoinPool p = pool;
// This loop is needed in case attempt to activate fails, in
// which case we only retry if there still appears to be a
// submission.
while (p.hasQueuedSubmissions()) {
ForkJoinTask<?> t; int a;
if (active || // inline p.tryIncrementActiveCount
(active = UNSAFE.compareAndSwapInt(p, poolRunStateOffset,
a = p.runState, a + 1))) {
if ((t = p.pollSubmission()) != null) {
UNSAFE.putOrderedObject(this, currentStealOffset, t);
t.quietlyExec();
UNSAFE.putOrderedObject(this, currentStealOffset, null);
if (sp != base)
execLocalTasks();
return true;
}
}
}
return false;
}
/**
* Runs local tasks until queue is empty or shut down. Call only
* while active.
*/
private void execLocalTasks() {
while (runState == 0) {
ForkJoinTask<?> t = locallyFifo ? locallyDeqTask() : popTask();
if (t != null)
t.quietlyExec();
else if (sp == base)
break;
}
}
/* /*
* Intrinsics-based atomic writes for queue slots. These are * Intrinsics-based atomic writes for queue slots. These are
* basically the same as methods in AtomicReferenceArray, but * basically the same as methods in AtomicReferenceArray, but
...@@ -528,10 +414,20 @@ public class ForkJoinWorkerThread extends Thread { ...@@ -528,10 +414,20 @@ public class ForkJoinWorkerThread extends Thread {
* because they are protected by other volatile reads and are * because they are protected by other volatile reads and are
* confirmed by CASes. * confirmed by CASes.
* *
* Most uses don't actually call these methods, but instead contain * Most uses don't actually call these methods, but instead
* inlined forms that enable more predictable optimization. We * contain inlined forms that enable more predictable
* don't define the version of write used in pushTask at all, but * optimization. We don't define the version of write used in
* instead inline there a store-fenced array slot write. * pushTask at all, but instead inline there a store-fenced array
* slot write.
*
* Also in most methods, as a performance (not correctness) issue,
* we'd like to encourage compilers not to arbitrarily postpone
* setting queueTop after writing slot. Currently there is no
* intrinsic for arranging this, but using Unsafe putOrderedInt
* may be a preferable strategy on some compilers even though its
* main effect is a pre-, not post- fence. To simplify possible
* changes, the option is left in comments next to the associated
* assignments.
*/ */
/** /**
...@@ -540,7 +436,7 @@ public class ForkJoinWorkerThread extends Thread { ...@@ -540,7 +436,7 @@ public class ForkJoinWorkerThread extends Thread {
*/ */
private static final boolean casSlotNull(ForkJoinTask<?>[] q, int i, private static final boolean casSlotNull(ForkJoinTask<?>[] q, int i,
ForkJoinTask<?> t) { ForkJoinTask<?> t) {
return UNSAFE.compareAndSwapObject(q, (i << qShift) + qBase, t, null); return UNSAFE.compareAndSwapObject(q, (i << ASHIFT) + ABASE, t, null);
} }
/** /**
...@@ -550,7 +446,7 @@ public class ForkJoinWorkerThread extends Thread { ...@@ -550,7 +446,7 @@ public class ForkJoinWorkerThread extends Thread {
*/ */
private static final void writeSlot(ForkJoinTask<?>[] q, int i, private static final void writeSlot(ForkJoinTask<?>[] q, int i,
ForkJoinTask<?> t) { ForkJoinTask<?> t) {
UNSAFE.putObjectVolatile(q, (i << qShift) + qBase, t); UNSAFE.putObjectVolatile(q, (i << ASHIFT) + ABASE, t);
} }
// queue methods // queue methods
...@@ -561,14 +457,43 @@ public class ForkJoinWorkerThread extends Thread { ...@@ -561,14 +457,43 @@ public class ForkJoinWorkerThread extends Thread {
* @param t the task. Caller must ensure non-null. * @param t the task. Caller must ensure non-null.
*/ */
final void pushTask(ForkJoinTask<?> t) { final void pushTask(ForkJoinTask<?> t) {
ForkJoinTask<?>[] q = queue; ForkJoinTask<?>[] q; int s, m;
int mask = q.length - 1; // implicit assert q != null if ((q = queue) != null) { // ignore if queue removed
int s = sp++; // ok to increment sp before slot write long u = (((s = queueTop) & (m = q.length - 1)) << ASHIFT) + ABASE;
UNSAFE.putOrderedObject(q, ((s & mask) << qShift) + qBase, t); UNSAFE.putOrderedObject(q, u, t);
if ((s -= base) == 0) queueTop = s + 1; // or use putOrderedInt
pool.signalWork(); // was empty if ((s -= queueBase) <= 2)
else if (s == mask) pool.signalWork();
growQueue(); // is full else if (s == m)
growQueue();
}
}
/**
* Creates or doubles queue array. Transfers elements by
* emulating steals (deqs) from old array and placing, oldest
* first, into new array.
*/
private void growQueue() {
ForkJoinTask<?>[] oldQ = queue;
int size = oldQ != null ? oldQ.length << 1 : INITIAL_QUEUE_CAPACITY;
if (size > MAXIMUM_QUEUE_CAPACITY)
throw new RejectedExecutionException("Queue capacity exceeded");
if (size < INITIAL_QUEUE_CAPACITY)
size = INITIAL_QUEUE_CAPACITY;
ForkJoinTask<?>[] q = queue = new ForkJoinTask<?>[size];
int mask = size - 1;
int top = queueTop;
int oldMask;
if (oldQ != null && (oldMask = oldQ.length - 1) >= 0) {
for (int b = queueBase; b != top; ++b) {
long u = ((b & oldMask) << ASHIFT) + ABASE;
Object x = UNSAFE.getObjectVolatile(oldQ, u);
if (x != null && UNSAFE.compareAndSwapObject(oldQ, u, x, null))
UNSAFE.putObjectVolatile
(q, ((b & mask) << ASHIFT) + ABASE, x);
}
}
} }
/** /**
...@@ -579,35 +504,34 @@ public class ForkJoinWorkerThread extends Thread { ...@@ -579,35 +504,34 @@ public class ForkJoinWorkerThread extends Thread {
* @return a task, or null if none or contended * @return a task, or null if none or contended
*/ */
final ForkJoinTask<?> deqTask() { final ForkJoinTask<?> deqTask() {
ForkJoinTask<?> t; ForkJoinTask<?> t; ForkJoinTask<?>[] q; int b, i;
ForkJoinTask<?>[] q; if (queueTop != (b = queueBase) &&
int b, i;
if (sp != (b = base) &&
(q = queue) != null && // must read q after b (q = queue) != null && // must read q after b
(t = q[i = (q.length - 1) & b]) != null && base == b && (i = (q.length - 1) & b) >= 0 &&
UNSAFE.compareAndSwapObject(q, (i << qShift) + qBase, t, null)) { (t = q[i]) != null && queueBase == b &&
base = b + 1; UNSAFE.compareAndSwapObject(q, (i << ASHIFT) + ABASE, t, null)) {
queueBase = b + 1;
return t; return t;
} }
return null; return null;
} }
/** /**
* Tries to take a task from the base of own queue. Assumes active * Tries to take a task from the base of own queue. Called only
* status. Called only by this thread. * by this thread.
* *
* @return a task, or null if none * @return a task, or null if none
*/ */
final ForkJoinTask<?> locallyDeqTask() { final ForkJoinTask<?> locallyDeqTask() {
ForkJoinTask<?> t; int m, b, i;
ForkJoinTask<?>[] q = queue; ForkJoinTask<?>[] q = queue;
if (q != null) { if (q != null && (m = q.length - 1) >= 0) {
ForkJoinTask<?> t; while (queueTop != (b = queueBase)) {
int b, i; if ((t = q[i = m & b]) != null &&
while (sp != (b = base)) { queueBase == b &&
if ((t = q[i = (q.length - 1) & b]) != null && base == b && UNSAFE.compareAndSwapObject(q, (i << ASHIFT) + ABASE,
UNSAFE.compareAndSwapObject(q, (i << qShift) + qBase,
t, null)) { t, null)) {
base = b + 1; queueBase = b + 1;
return t; return t;
} }
} }
...@@ -616,35 +540,21 @@ public class ForkJoinWorkerThread extends Thread { ...@@ -616,35 +540,21 @@ public class ForkJoinWorkerThread extends Thread {
} }
/** /**
* Returns a popped task, or null if empty. Assumes active status. * Returns a popped task, or null if empty.
* Called only by this thread. * Called only by this thread.
*/ */
private ForkJoinTask<?> popTask() { private ForkJoinTask<?> popTask() {
int m;
ForkJoinTask<?>[] q = queue; ForkJoinTask<?>[] q = queue;
if (q != null) { if (q != null && (m = q.length - 1) >= 0) {
int s; for (int s; (s = queueTop) != queueBase;) {
while ((s = sp) != base) { int i = m & --s;
int i = (q.length - 1) & --s; long u = (i << ASHIFT) + ABASE; // raw offset
long u = (i << qShift) + qBase; // raw offset
ForkJoinTask<?> t = q[i]; ForkJoinTask<?> t = q[i];
if (t == null) // lost to stealer if (t == null) // lost to stealer
break; break;
if (UNSAFE.compareAndSwapObject(q, u, t, null)) { if (UNSAFE.compareAndSwapObject(q, u, t, null)) {
/* queueTop = s; // or putOrderedInt
* Note: here and in related methods, as a
* performance (not correctness) issue, we'd like
* to encourage compiler not to arbitrarily
* postpone setting sp after successful CAS.
* Currently there is no intrinsic for arranging
* this, but using Unsafe putOrderedInt may be a
* preferable strategy on some compilers even
* though its main effect is a pre-, not post-
* fence. To simplify possible changes, the option
* is left in comments next to the associated
* assignments.
*/
sp = s; // putOrderedInt may encourage more timely write
// UNSAFE.putOrderedInt(this, spOffset, s);
return t; return t;
} }
} }
...@@ -654,18 +564,17 @@ public class ForkJoinWorkerThread extends Thread { ...@@ -654,18 +564,17 @@ public class ForkJoinWorkerThread extends Thread {
/** /**
* Specialized version of popTask to pop only if topmost element * Specialized version of popTask to pop only if topmost element
* is the given task. Called only by this thread while active. * is the given task. Called only by this thread.
* *
* @param t the task. Caller must ensure non-null. * @param t the task. Caller must ensure non-null.
*/ */
final boolean unpushTask(ForkJoinTask<?> t) { final boolean unpushTask(ForkJoinTask<?> t) {
ForkJoinTask<?>[] q;
int s; int s;
ForkJoinTask<?>[] q = queue; if ((q = queue) != null && (s = queueTop) != queueBase &&
if ((s = sp) != base && q != null &&
UNSAFE.compareAndSwapObject UNSAFE.compareAndSwapObject
(q, (((q.length - 1) & --s) << qShift) + qBase, t, null)) { (q, (((q.length - 1) & --s) << ASHIFT) + ABASE, t, null)) {
sp = s; // putOrderedInt may encourage more timely write queueTop = s; // or putOrderedInt
// UNSAFE.putOrderedInt(this, spOffset, s);
return true; return true;
} }
return false; return false;
...@@ -675,222 +584,30 @@ public class ForkJoinWorkerThread extends Thread { ...@@ -675,222 +584,30 @@ public class ForkJoinWorkerThread extends Thread {
* Returns next task, or null if empty or contended. * Returns next task, or null if empty or contended.
*/ */
final ForkJoinTask<?> peekTask() { final ForkJoinTask<?> peekTask() {
int m;
ForkJoinTask<?>[] q = queue; ForkJoinTask<?>[] q = queue;
if (q == null) if (q == null || (m = q.length - 1) < 0)
return null;
int mask = q.length - 1;
int i = locallyFifo ? base : (sp - 1);
return q[i & mask];
}
/**
* Doubles queue array size. Transfers elements by emulating
* steals (deqs) from old array and placing, oldest first, into
* new array.
*/
private void growQueue() {
ForkJoinTask<?>[] oldQ = queue;
int oldSize = oldQ.length;
int newSize = oldSize << 1;
if (newSize > MAXIMUM_QUEUE_CAPACITY)
throw new RejectedExecutionException("Queue capacity exceeded");
ForkJoinTask<?>[] newQ = queue = new ForkJoinTask<?>[newSize];
int b = base;
int bf = b + oldSize;
int oldMask = oldSize - 1;
int newMask = newSize - 1;
do {
int oldIndex = b & oldMask;
ForkJoinTask<?> t = oldQ[oldIndex];
if (t != null && !casSlotNull(oldQ, oldIndex, t))
t = null;
writeSlot(newQ, b & newMask, t);
} while (++b != bf);
pool.signalWork();
}
/**
* Computes next value for random victim probe in scan(). Scans
* don't require a very high quality generator, but also not a
* crummy one. Marsaglia xor-shift is cheap and works well enough.
* Note: This is manually inlined in scan().
*/
private static final int xorShift(int r) {
r ^= r << 13;
r ^= r >>> 17;
return r ^ (r << 5);
}
/**
* Tries to steal a task from another worker. Starts at a random
* index of workers array, and probes workers until finding one
* with non-empty queue or finding that all are empty. It
* randomly selects the first n probes. If these are empty, it
* resorts to a circular sweep, which is necessary to accurately
* set active status. (The circular sweep uses steps of
* approximately half the array size plus 1, to avoid bias
* stemming from leftmost packing of the array in ForkJoinPool.)
*
* This method must be both fast and quiet -- usually avoiding
* memory accesses that could disrupt cache sharing etc other than
* those needed to check for and take tasks (or to activate if not
* already active). This accounts for, among other things,
* updating random seed in place without storing it until exit.
*
* @return a task, or null if none found
*/
private ForkJoinTask<?> scan() {
ForkJoinPool p = pool;
ForkJoinWorkerThread[] ws; // worker array
int n; // upper bound of #workers
if ((ws = p.workers) != null && (n = ws.length) > 1) {
boolean canSteal = active; // shadow active status
int r = seed; // extract seed once
int mask = n - 1;
int j = -n; // loop counter
int k = r; // worker index, random if j < 0
for (;;) {
ForkJoinWorkerThread v = ws[k & mask];
r ^= r << 13; r ^= r >>> 17; r ^= r << 5; // inline xorshift
ForkJoinTask<?>[] q; ForkJoinTask<?> t; int b, a;
if (v != null && (b = v.base) != v.sp &&
(q = v.queue) != null) {
int i = (q.length - 1) & b;
long u = (i << qShift) + qBase; // raw offset
int pid = poolIndex;
if ((t = q[i]) != null) {
if (!canSteal && // inline p.tryIncrementActiveCount
UNSAFE.compareAndSwapInt(p, poolRunStateOffset,
a = p.runState, a + 1))
canSteal = active = true;
if (canSteal && v.base == b++ &&
UNSAFE.compareAndSwapObject(q, u, t, null)) {
v.base = b;
v.stealHint = pid;
UNSAFE.putOrderedObject(this,
currentStealOffset, t);
seed = r;
++stealCount;
return t;
}
}
j = -n;
k = r; // restart on contention
}
else if (++j <= 0)
k = r;
else if (j <= n)
k += (n >>> 1) | 1;
else
break;
}
}
return null; return null;
int i = locallyFifo ? queueBase : (queueTop - 1);
return q[i & m];
} }
// Run State management // Support methods for ForkJoinPool
// status check methods used mainly by ForkJoinPool
final boolean isRunning() { return runState == 0; }
final boolean isTerminated() { return (runState & TERMINATED) != 0; }
final boolean isSuspended() { return (runState & SUSPENDED) != 0; }
final boolean isTrimmed() { return (runState & TRIMMED) != 0; }
final boolean isTerminating() {
if ((runState & TERMINATING) != 0)
return true;
if (pool.isAtLeastTerminating()) { // propagate pool state
shutdown();
return true;
}
return false;
}
/** /**
* Sets state to TERMINATING. Does NOT unpark or interrupt * Runs the given task, plus any local tasks until queue is empty
* to wake up if currently blocked. Callers must do so if desired.
*/ */
final void shutdown() { final void execTask(ForkJoinTask<?> t) {
currentSteal = t;
for (;;) { for (;;) {
int s = runState; if (t != null)
if ((s & (TERMINATING|TERMINATED)) != 0) t.doExec();
break; if (queueTop == queueBase)
if ((s & SUSPENDED) != 0) { // kill and wakeup if suspended
if (UNSAFE.compareAndSwapInt(this, runStateOffset, s,
(s & ~SUSPENDED) |
(TRIMMED|TERMINATING)))
break;
}
else if (UNSAFE.compareAndSwapInt(this, runStateOffset, s,
s | TERMINATING))
break;
}
}
/**
* Sets state to TERMINATED. Called only by onTermination().
*/
private void setTerminated() {
int s;
do {} while (!UNSAFE.compareAndSwapInt(this, runStateOffset,
s = runState,
s | (TERMINATING|TERMINATED)));
}
/**
* If suspended, tries to set status to unsuspended.
* Does NOT wake up if blocked.
*
* @return true if successful
*/
final boolean tryUnsuspend() {
int s;
while (((s = runState) & SUSPENDED) != 0) {
if (UNSAFE.compareAndSwapInt(this, runStateOffset, s,
s & ~SUSPENDED))
return true;
}
return false;
}
/**
* Sets suspended status and blocks as spare until resumed
* or shutdown.
*/
final void suspendAsSpare() {
for (;;) { // set suspended unless terminating
int s = runState;
if ((s & TERMINATING) != 0) { // must kill
if (UNSAFE.compareAndSwapInt(this, runStateOffset, s,
s | (TRIMMED | TERMINATING)))
return;
}
else if (UNSAFE.compareAndSwapInt(this, runStateOffset, s,
s | SUSPENDED))
break;
}
ForkJoinPool p = pool;
p.pushSpare(this);
while ((runState & SUSPENDED) != 0) {
if (p.tryAccumulateStealCount(this)) {
interrupted(); // clear/ignore interrupts
if ((runState & SUSPENDED) == 0)
break; break;
LockSupport.park(this); t = locallyFifo ? locallyDeqTask() : popTask();
}
}
} }
++stealCount;
// Misc support methods for ForkJoinPool currentSteal = null;
/**
* Returns an estimate of the number of tasks in the queue. Also
* used by ForkJoinTask.
*/
final int getQueueSize() {
int n; // external calls must read base first
return (n = -base + sp) <= 0 ? 0 : n;
} }
/** /**
...@@ -899,17 +616,12 @@ public class ForkJoinWorkerThread extends Thread { ...@@ -899,17 +616,12 @@ public class ForkJoinWorkerThread extends Thread {
*/ */
final void cancelTasks() { final void cancelTasks() {
ForkJoinTask<?> cj = currentJoin; // try to cancel ongoing tasks ForkJoinTask<?> cj = currentJoin; // try to cancel ongoing tasks
if (cj != null && cj.status >= 0) { if (cj != null && cj.status >= 0)
cj.cancelIgnoringExceptions(); cj.cancelIgnoringExceptions();
try {
this.interrupt(); // awaken wait
} catch (SecurityException ignore) {
}
}
ForkJoinTask<?> cs = currentSteal; ForkJoinTask<?> cs = currentSteal;
if (cs != null && cs.status >= 0) if (cs != null && cs.status >= 0)
cs.cancelIgnoringExceptions(); cs.cancelIgnoringExceptions();
while (base != sp) { while (queueBase != queueTop) {
ForkJoinTask<?> t = deqTask(); ForkJoinTask<?> t = deqTask();
if (t != null) if (t != null)
t.cancelIgnoringExceptions(); t.cancelIgnoringExceptions();
...@@ -923,7 +635,7 @@ public class ForkJoinWorkerThread extends Thread { ...@@ -923,7 +635,7 @@ public class ForkJoinWorkerThread extends Thread {
*/ */
final int drainTasksTo(Collection<? super ForkJoinTask<?>> c) { final int drainTasksTo(Collection<? super ForkJoinTask<?>> c) {
int n = 0; int n = 0;
while (base != sp) { while (queueBase != queueTop) {
ForkJoinTask<?> t = deqTask(); ForkJoinTask<?> t = deqTask();
if (t != null) { if (t != null) {
c.add(t); c.add(t);
...@@ -935,22 +647,21 @@ public class ForkJoinWorkerThread extends Thread { ...@@ -935,22 +647,21 @@ public class ForkJoinWorkerThread extends Thread {
// Support methods for ForkJoinTask // Support methods for ForkJoinTask
/**
* Returns an estimate of the number of tasks in the queue.
*/
final int getQueueSize() {
return queueTop - queueBase;
}
/** /**
* Gets and removes a local task. * Gets and removes a local task.
* *
* @return a task, if available * @return a task, if available
*/ */
final ForkJoinTask<?> pollLocalTask() { final ForkJoinTask<?> pollLocalTask() {
ForkJoinPool p = pool;
while (sp != base) {
int a; // inline p.tryIncrementActiveCount
if (active ||
(active = UNSAFE.compareAndSwapInt(p, poolRunStateOffset,
a = p.runState, a + 1)))
return locallyFifo ? locallyDeqTask() : popTask(); return locallyFifo ? locallyDeqTask() : popTask();
} }
return null;
}
/** /**
* Gets and removes a local or stolen task. * Gets and removes a local or stolen task.
...@@ -958,172 +669,205 @@ public class ForkJoinWorkerThread extends Thread { ...@@ -958,172 +669,205 @@ public class ForkJoinWorkerThread extends Thread {
* @return a task, if available * @return a task, if available
*/ */
final ForkJoinTask<?> pollTask() { final ForkJoinTask<?> pollTask() {
ForkJoinWorkerThread[] ws;
ForkJoinTask<?> t = pollLocalTask(); ForkJoinTask<?> t = pollLocalTask();
if (t == null) { if (t != null || (ws = pool.workers) == null)
t = scan(); return t;
// cannot retain/track/help steal int n = ws.length; // cheap version of FJP.scan
UNSAFE.putOrderedObject(this, currentStealOffset, null); int steps = n << 1;
} int r = nextSeed();
int i = 0;
while (i < steps) {
ForkJoinWorkerThread w = ws[(i++ + r) & (n - 1)];
if (w != null && w.queueBase != w.queueTop && w.queue != null) {
if ((t = w.deqTask()) != null)
return t; return t;
i = 0;
} }
}
return null;
}
/**
* The maximum stolen->joining link depth allowed in helpJoinTask,
* as well as the maximum number of retries (allowing on average
* one staleness retry per level) per attempt to instead try
* compensation. Depths for legitimate chains are unbounded, but
* we use a fixed constant to avoid (otherwise unchecked) cycles
* and bound staleness of traversal parameters at the expense of
* sometimes blocking when we could be helping.
*/
private static final int MAX_HELP = 16;
/** /**
* Possibly runs some tasks and/or blocks, until task is done. * Possibly runs some tasks and/or blocks, until joinMe is done.
* *
* @param joinMe the task to join * @param joinMe the task to join
* @param timed true if use timed wait * @return completion status on exit
* @param nanos wait time if timed
*/ */
final void joinTask(ForkJoinTask<?> joinMe, boolean timed, long nanos) { final int joinTask(ForkJoinTask<?> joinMe) {
// currentJoin only written by this thread; only need ordered store
ForkJoinTask<?> prevJoin = currentJoin; ForkJoinTask<?> prevJoin = currentJoin;
UNSAFE.putOrderedObject(this, currentJoinOffset, joinMe); currentJoin = joinMe;
pool.awaitJoin(joinMe, this, timed, nanos); for (int s, retries = MAX_HELP;;) {
UNSAFE.putOrderedObject(this, currentJoinOffset, prevJoin); if ((s = joinMe.status) < 0) {
currentJoin = prevJoin;
return s;
}
if (retries > 0) {
if (queueTop != queueBase) {
if (!localHelpJoinTask(joinMe))
retries = 0; // cannot help
}
else if (retries == MAX_HELP >>> 1) {
--retries; // check uncommon case
if (tryDeqAndExec(joinMe) >= 0)
Thread.yield(); // for politeness
}
else
retries = helpJoinTask(joinMe)? MAX_HELP : retries - 1;
}
else {
retries = MAX_HELP; // restart if not done
pool.tryAwaitJoin(joinMe);
}
}
} }
/** /**
* Tries to locate and help perform tasks for a stealer of the * If present, pops and executes the given task, or any other
* given task, or in turn one of its stealers. Traces * cancelled task
* currentSteal->currentJoin links looking for a thread working on
* a descendant of the given task and with a non-empty queue to
* steal back and execute tasks from.
*
* The implementation is very branchy to cope with potential
* inconsistencies or loops encountering chains that are stale,
* unknown, or of length greater than MAX_HELP_DEPTH links. All
* of these cases are dealt with by just returning back to the
* caller, who is expected to retry if other join mechanisms also
* don't work out.
* *
* @param joinMe the task to join * @return false if any other non-cancelled task exists in local queue
* @param running if false, then must update pool count upon
* running a task
* @return value of running on exit
*/
final boolean helpJoinTask(ForkJoinTask<?> joinMe, boolean running) {
/*
* Initial checks to (1) abort if terminating; (2) clean out
* old cancelled tasks from local queue; (3) if joinMe is next
* task, run it; (4) omit scan if local queue nonempty (since
* it may contain non-descendents of joinMe).
*/ */
ForkJoinPool p = pool; private boolean localHelpJoinTask(ForkJoinTask<?> joinMe) {
for (;;) { int s, i; ForkJoinTask<?>[] q; ForkJoinTask<?> t;
ForkJoinTask<?>[] q; if ((s = queueTop) != queueBase && (q = queue) != null &&
int s; (i = (q.length - 1) & --s) >= 0 &&
if (joinMe.status < 0) (t = q[i]) != null) {
return running; if (t != joinMe && t.status >= 0)
else if ((runState & TERMINATING) != 0) { return false;
joinMe.cancelIgnoringExceptions(); if (UNSAFE.compareAndSwapObject
return running; (q, (i << ASHIFT) + ABASE, t, null)) {
} queueTop = s; // or putOrderedInt
else if ((s = sp) == base || (q = queue) == null) t.doExec();
break; // queue empty
else {
int i = (q.length - 1) & --s;
long u = (i << qShift) + qBase; // raw offset
ForkJoinTask<?> t = q[i];
if (t == null)
break; // lost to a stealer
else if (t != joinMe && t.status >= 0)
return running; // cannot safely help
else if ((running ||
(running = p.tryIncrementRunningCount())) &&
UNSAFE.compareAndSwapObject(q, u, t, null)) {
sp = s; // putOrderedInt may encourage more timely write
// UNSAFE.putOrderedInt(this, spOffset, s);
t.quietlyExec();
} }
} }
return true;
} }
int n; // worker array size /**
ForkJoinWorkerThread[] ws = p.workers; * Tries to locate and execute tasks for a stealer of the given
if (ws != null && (n = ws.length) > 1) { // need at least 2 workers * task, or in turn one of its stealers, Traces
* currentSteal->currentJoin links looking for a thread working on
* a descendant of the given task and with a non-empty queue to
* steal back and execute tasks from. The implementation is very
* branchy to cope with potential inconsistencies or loops
* encountering chains that are stale, unknown, or of length
* greater than MAX_HELP links. All of these cases are dealt with
* by just retrying by caller.
*
* @param joinMe the task to join
* @param canSteal true if local queue is empty
* @return true if ran a task
*/
private boolean helpJoinTask(ForkJoinTask<?> joinMe) {
boolean helped = false;
int m = pool.scanGuard & SMASK;
ForkJoinWorkerThread[] ws = pool.workers;
if (ws != null && ws.length > m && joinMe.status >= 0) {
int levels = MAX_HELP; // remaining chain length
ForkJoinTask<?> task = joinMe; // base of chain ForkJoinTask<?> task = joinMe; // base of chain
ForkJoinWorkerThread thread = this; // thread with stolen task outer:for (ForkJoinWorkerThread thread = this;;) {
outer:for (int d = 0; d < MAX_HELP_DEPTH; ++d) { // chain length
// Try to find v, the stealer of task, by first using hint // Try to find v, the stealer of task, by first using hint
ForkJoinWorkerThread v = ws[thread.stealHint & (n - 1)]; ForkJoinWorkerThread v = ws[thread.stealHint & m];
if (v == null || v.currentSteal != task) { if (v == null || v.currentSteal != task) {
for (int j = 0; ; ++j) { // search array for (int j = 0; ;) { // search array
if (j < n) { if ((v = ws[j]) != null && v.currentSteal == task) {
ForkJoinTask<?> vs;
if ((v = ws[j]) != null &&
(vs = v.currentSteal) != null) {
if (joinMe.status < 0)
break outer;
if (vs == task) {
if (task.status < 0)
break outer; // stale
thread.stealHint = j; thread.stealHint = j;
break; // save hint for next time break; // save hint for next time
} }
if (++j > m)
break outer; // can't find stealer
} }
} }
else
break outer; // no stealer
}
}
// Try to help v, using specialized form of deqTask // Try to help v, using specialized form of deqTask
for (;;) { for (;;) {
ForkJoinTask<?>[] q; int b, i;
if (joinMe.status < 0) if (joinMe.status < 0)
break outer; break outer;
int b = v.base; if ((b = v.queueBase) == v.queueTop ||
ForkJoinTask<?>[] q = v.queue; (q = v.queue) == null ||
if (b == v.sp || q == null) (i = (q.length-1) & b) < 0)
break; // empty break; // empty
int i = (q.length - 1) & b; long u = (i << ASHIFT) + ABASE;
long u = (i << qShift) + qBase;
ForkJoinTask<?> t = q[i]; ForkJoinTask<?> t = q[i];
if (task.status < 0) if (task.status < 0)
break outer; // stale break outer; // stale
if (t != null && if (t != null && v.queueBase == b &&
(running ||
(running = p.tryIncrementRunningCount())) &&
v.base == b++ &&
UNSAFE.compareAndSwapObject(q, u, t, null)) { UNSAFE.compareAndSwapObject(q, u, t, null)) {
if (t != joinMe && joinMe.status < 0) { v.queueBase = b + 1;
UNSAFE.putObjectVolatile(q, u, t); v.stealHint = poolIndex;
break outer; // joinMe cancelled; back out
}
v.base = b;
if (t.status >= 0) {
ForkJoinTask<?> ps = currentSteal; ForkJoinTask<?> ps = currentSteal;
int pid = poolIndex; currentSteal = t;
v.stealHint = pid; t.doExec();
UNSAFE.putOrderedObject(this, currentSteal = ps;
currentStealOffset, t); helped = true;
t.quietlyExec();
UNSAFE.putOrderedObject(this,
currentStealOffset, ps);
}
}
else if ((runState & TERMINATING) != 0) {
joinMe.cancelIgnoringExceptions();
break outer;
} }
} }
// Try to descend to find v's stealer // Try to descend to find v's stealer
ForkJoinTask<?> next = v.currentJoin; ForkJoinTask<?> next = v.currentJoin;
if (task.status < 0 || next == null || next == task || if (--levels > 0 && task.status >= 0 &&
joinMe.status < 0) next != null && next != task) {
break; // done, stale, dead-end, or cyclic
task = next; task = next;
thread = v; thread = v;
} }
else
break; // max levels, stale, dead-end, or cyclic
}
} }
return running; return helped;
} }
/** /**
* Implements ForkJoinTask.getSurplusQueuedTaskCount(). * Performs an uncommon case for joinTask: If task t is at base of
* Returns an estimate of the number of tasks, offset by a * some workers queue, steals and executes it.
* function of number of idle workers. *
* @param t the task
* @return t's status
*/
private int tryDeqAndExec(ForkJoinTask<?> t) {
int m = pool.scanGuard & SMASK;
ForkJoinWorkerThread[] ws = pool.workers;
if (ws != null && ws.length > m && t.status >= 0) {
for (int j = 0; j <= m; ++j) {
ForkJoinTask<?>[] q; int b, i;
ForkJoinWorkerThread v = ws[j];
if (v != null &&
(b = v.queueBase) != v.queueTop &&
(q = v.queue) != null &&
(i = (q.length - 1) & b) >= 0 &&
q[i] == t) {
long u = (i << ASHIFT) + ABASE;
if (v.queueBase == b &&
UNSAFE.compareAndSwapObject(q, u, t, null)) {
v.queueBase = b + 1;
v.stealHint = poolIndex;
ForkJoinTask<?> ps = currentSteal;
currentSteal = t;
t.doExec();
currentSteal = ps;
}
break;
}
}
}
return t.status;
}
/**
* Implements ForkJoinTask.getSurplusQueuedTaskCount(). Returns
* an estimate of the number of tasks, offset by a function of
* number of idle workers.
* *
* This method provides a cheap heuristic guide for task * This method provides a cheap heuristic guide for task
* partitioning when programmers, frameworks, tools, or languages * partitioning when programmers, frameworks, tools, or languages
...@@ -1159,82 +903,96 @@ public class ForkJoinWorkerThread extends Thread { ...@@ -1159,82 +903,96 @@ public class ForkJoinWorkerThread extends Thread {
* When all threads are active, it is on average OK to estimate * When all threads are active, it is on average OK to estimate
* surplus strictly locally. In steady-state, if one thread is * surplus strictly locally. In steady-state, if one thread is
* maintaining say 2 surplus tasks, then so are others. So we can * maintaining say 2 surplus tasks, then so are others. So we can
* just use estimated queue length (although note that (sp - base) * just use estimated queue length (although note that (queueTop -
* can be an overestimate because of stealers lagging increments * queueBase) can be an overestimate because of stealers lagging
* of base). However, this strategy alone leads to serious * increments of queueBase). However, this strategy alone leads
* mis-estimates in some non-steady-state conditions (ramp-up, * to serious mis-estimates in some non-steady-state conditions
* ramp-down, other stalls). We can detect many of these by * (ramp-up, ramp-down, other stalls). We can detect many of these
* further considering the number of "idle" threads, that are * by further considering the number of "idle" threads, that are
* known to have zero queued tasks, so compensate by a factor of * known to have zero queued tasks, so compensate by a factor of
* (#idle/#active) threads. * (#idle/#active) threads.
*/ */
final int getEstimatedSurplusTaskCount() { final int getEstimatedSurplusTaskCount() {
return sp - base - pool.idlePerActive(); return queueTop - queueBase - pool.idlePerActive();
} }
/** /**
* Runs tasks until {@code pool.isQuiescent()}. * Runs tasks until {@code pool.isQuiescent()}. We piggyback on
* pool's active count ctl maintenance, but rather than blocking
* when tasks cannot be found, we rescan until all others cannot
* find tasks either. The bracketing by pool quiescerCounts
* updates suppresses pool auto-shutdown mechanics that could
* otherwise prematurely terminate the pool because all threads
* appear to be inactive.
*/ */
final void helpQuiescePool() { final void helpQuiescePool() {
boolean active = true;
ForkJoinTask<?> ps = currentSteal; // to restore below ForkJoinTask<?> ps = currentSteal; // to restore below
ForkJoinPool p = pool;
p.addQuiescerCount(1);
for (;;) { for (;;) {
ForkJoinTask<?> t = pollLocalTask(); ForkJoinWorkerThread[] ws = p.workers;
if (t != null || (t = scan()) != null) ForkJoinWorkerThread v = null;
t.quietlyExec(); int n;
if (queueTop != queueBase)
v = this;
else if (ws != null && (n = ws.length) > 1) {
ForkJoinWorkerThread w;
int r = nextSeed(); // cheap version of FJP.scan
int steps = n << 1;
for (int i = 0; i < steps; ++i) {
if ((w = ws[(i + r) & (n - 1)]) != null &&
w.queueBase != w.queueTop) {
v = w;
break;
}
}
}
if (v != null) {
ForkJoinTask<?> t;
if (!active) {
active = true;
p.addActiveCount(1);
}
if ((t = (v != this) ? v.deqTask() :
locallyFifo? locallyDeqTask() : popTask()) != null) {
currentSteal = t;
t.doExec();
currentSteal = ps;
}
}
else { else {
ForkJoinPool p = pool;
int a; // to inline CASes
if (active) { if (active) {
if (!UNSAFE.compareAndSwapInt active = false;
(p, poolRunStateOffset, a = p.runState, a - 1)) p.addActiveCount(-1);
continue; // retry later
active = false; // inactivate
UNSAFE.putOrderedObject(this, currentStealOffset, ps);
} }
if (p.isQuiescent()) { if (p.isQuiescent()) {
active = true; // re-activate p.addActiveCount(1);
do {} while (!UNSAFE.compareAndSwapInt p.addQuiescerCount(-1);
(p, poolRunStateOffset, a = p.runState, a+1)); break;
return;
} }
} }
} }
} }
// Unsafe mechanics // Unsafe mechanics
private static final sun.misc.Unsafe UNSAFE;
private static final sun.misc.Unsafe UNSAFE = sun.misc.Unsafe.getUnsafe(); private static final long ABASE;
private static final long spOffset = private static final int ASHIFT;
objectFieldOffset("sp", ForkJoinWorkerThread.class);
private static final long runStateOffset =
objectFieldOffset("runState", ForkJoinWorkerThread.class);
private static final long currentJoinOffset =
objectFieldOffset("currentJoin", ForkJoinWorkerThread.class);
private static final long currentStealOffset =
objectFieldOffset("currentSteal", ForkJoinWorkerThread.class);
private static final long qBase =
UNSAFE.arrayBaseOffset(ForkJoinTask[].class);
private static final long poolRunStateOffset = // to inline CAS
objectFieldOffset("runState", ForkJoinPool.class);
private static final int qShift;
static { static {
int s = UNSAFE.arrayIndexScale(ForkJoinTask[].class); int s;
try {
UNSAFE = sun.misc.Unsafe.getUnsafe();
Class a = ForkJoinTask[].class;
ABASE = UNSAFE.arrayBaseOffset(a);
s = UNSAFE.arrayIndexScale(a);
} catch (Exception e) {
throw new Error(e);
}
if ((s & (s-1)) != 0) if ((s & (s-1)) != 0)
throw new Error("data type scale not a power of two"); throw new Error("data type scale not a power of two");
qShift = 31 - Integer.numberOfLeadingZeros(s); ASHIFT = 31 - Integer.numberOfLeadingZeros(s);
MAXIMUM_QUEUE_CAPACITY = 1 << (31 - qShift);
} }
private static long objectFieldOffset(String field, Class<?> klazz) {
try {
return UNSAFE.objectFieldOffset(klazz.getDeclaredField(field));
} catch (NoSuchFieldException e) {
// Convert Exception to corresponding Error
NoSuchFieldError error = new NoSuchFieldError(field);
error.initCause(e);
throw error;
}
}
} }
Markdown is supported
0% .
You are about to add 0 people to the discussion. Proceed with caution.
先完成此消息的编辑!
想要评论请 注册