提交 c0d43003 编写于 作者: D dl

8002356: Add ForkJoin common pool and CountedCompleter

Reviewed-by: chegar, mduigou
上级 4aef9ad6
...@@ -322,6 +322,7 @@ JAVA_JAVA_java = \ ...@@ -322,6 +322,7 @@ JAVA_JAVA_java = \
java/util/concurrent/CopyOnWriteArrayList.java \ java/util/concurrent/CopyOnWriteArrayList.java \
java/util/concurrent/CopyOnWriteArraySet.java \ java/util/concurrent/CopyOnWriteArraySet.java \
java/util/concurrent/CountDownLatch.java \ java/util/concurrent/CountDownLatch.java \
java/util/concurrent/CountedCompleter.java \
java/util/concurrent/CyclicBarrier.java \ java/util/concurrent/CyclicBarrier.java \
java/util/concurrent/DelayQueue.java \ java/util/concurrent/DelayQueue.java \
java/util/concurrent/Delayed.java \ java/util/concurrent/Delayed.java \
......
/*
* DO NOT ALTER OR REMOVE COPYRIGHT NOTICES OR THIS FILE HEADER.
*
* This code is free software; you can redistribute it and/or modify it
* under the terms of the GNU General Public License version 2 only, as
* published by the Free Software Foundation. Oracle designates this
* particular file as subject to the "Classpath" exception as provided
* by Oracle in the LICENSE file that accompanied this code.
*
* This code is distributed in the hope that it will be useful, but WITHOUT
* ANY WARRANTY; without even the implied warranty of MERCHANTABILITY or
* FITNESS FOR A PARTICULAR PURPOSE. See the GNU General Public License
* version 2 for more details (a copy is included in the LICENSE file that
* accompanied this code).
*
* You should have received a copy of the GNU General Public License version
* 2 along with this work; if not, write to the Free Software Foundation,
* Inc., 51 Franklin St, Fifth Floor, Boston, MA 02110-1301 USA.
*
* Please contact Oracle, 500 Oracle Parkway, Redwood Shores, CA 94065 USA
* or visit www.oracle.com if you need additional information or have any
* questions.
*/
/*
* This file is available under and governed by the GNU General Public
* License version 2 only, as published by the Free Software Foundation.
* However, the following notice accompanied the original version of this
* file:
*
* Written by Doug Lea with assistance from members of JCP JSR-166
* Expert Group and released to the public domain, as explained at
* http://creativecommons.org/publicdomain/zero/1.0/
*/
package java.util.concurrent;
/**
* A {@link ForkJoinTask} with a completion action performed when
* triggered and there are no remaining pending
* actions. CountedCompleters are in general more robust in the
* presence of subtask stalls and blockage than are other forms of
* ForkJoinTasks, but are less intuitive to program. Uses of
* CountedCompleter are similar to those of other completion based
* components (such as {@link java.nio.channels.CompletionHandler})
* except that multiple <em>pending</em> completions may be necessary
* to trigger the completion action {@link #onCompletion}, not just one.
* Unless initialized otherwise, the {@linkplain #getPendingCount pending
* count} starts at zero, but may be (atomically) changed using
* methods {@link #setPendingCount}, {@link #addToPendingCount}, and
* {@link #compareAndSetPendingCount}. Upon invocation of {@link
* #tryComplete}, if the pending action count is nonzero, it is
* decremented; otherwise, the completion action is performed, and if
* this completer itself has a completer, the process is continued
* with its completer. As is the case with related synchronization
* components such as {@link java.util.concurrent.Phaser Phaser} and
* {@link java.util.concurrent.Semaphore Semaphore}, these methods
* affect only internal counts; they do not establish any further
* internal bookkeeping. In particular, the identities of pending
* tasks are not maintained. As illustrated below, you can create
* subclasses that do record some or all pending tasks or their
* results when needed. As illustrated below, utility methods
* supporting customization of completion traversals are also
* provided. However, because CountedCompleters provide only basic
* synchronization mechanisms, it may be useful to create further
* abstract subclasses that maintain linkages, fields, and additional
* support methods appropriate for a set of related usages.
*
* <p>A concrete CountedCompleter class must define method {@link
* #compute}, that should in most cases (as illustrated below), invoke
* {@code tryComplete()} once before returning. The class may also
* optionally override method {@link #onCompletion} to perform an
* action upon normal completion, and method {@link
* #onExceptionalCompletion} to perform an action upon any exception.
*
* <p>CountedCompleters most often do not bear results, in which case
* they are normally declared as {@code CountedCompleter<Void>}, and
* will always return {@code null} as a result value. In other cases,
* you should override method {@link #getRawResult} to provide a
* result from {@code join(), invoke()}, and related methods. In
* general, this method should return the value of a field (or a
* function of one or more fields) of the CountedCompleter object that
* holds the result upon completion. Method {@link #setRawResult} by
* default plays no role in CountedCompleters. It is possible, but
* rarely applicable, to override this method to maintain other
* objects or fields holding result data.
*
* <p>A CountedCompleter that does not itself have a completer (i.e.,
* one for which {@link #getCompleter} returns {@code null}) can be
* used as a regular ForkJoinTask with this added functionality.
* However, any completer that in turn has another completer serves
* only as an internal helper for other computations, so its own task
* status (as reported in methods such as {@link ForkJoinTask#isDone})
* is arbitrary; this status changes only upon explicit invocations of
* {@link #complete}, {@link ForkJoinTask#cancel}, {@link
* ForkJoinTask#completeExceptionally} or upon exceptional completion
* of method {@code compute}. Upon any exceptional completion, the
* exception may be relayed to a task's completer (and its completer,
* and so on), if one exists and it has not otherwise already
* completed. Similarly, cancelling an internal CountedCompleter has
* only a local effect on that completer, so is not often useful.
*
* <p><b>Sample Usages.</b>
*
* <p><b>Parallel recursive decomposition.</b> CountedCompleters may
* be arranged in trees similar to those often used with {@link
* RecursiveAction}s, although the constructions involved in setting
* them up typically vary. Here, the completer of each task is its
* parent in the computation tree. Even though they entail a bit more
* bookkeeping, CountedCompleters may be better choices when applying
* a possibly time-consuming operation (that cannot be further
* subdivided) to each element of an array or collection; especially
* when the operation takes a significantly different amount of time
* to complete for some elements than others, either because of
* intrinsic variation (for example I/O) or auxiliary effects such as
* garbage collection. Because CountedCompleters provide their own
* continuations, other threads need not block waiting to perform
* them.
*
* <p>For example, here is an initial version of a class that uses
* divide-by-two recursive decomposition to divide work into single
* pieces (leaf tasks). Even when work is split into individual calls,
* tree-based techniques are usually preferable to directly forking
* leaf tasks, because they reduce inter-thread communication and
* improve load balancing. In the recursive case, the second of each
* pair of subtasks to finish triggers completion of its parent
* (because no result combination is performed, the default no-op
* implementation of method {@code onCompletion} is not overridden). A
* static utility method sets up the base task and invokes it
* (here, implicitly using the {@link ForkJoinPool#commonPool()}).
*
* <pre> {@code
* class MyOperation<E> { void apply(E e) { ... } }
*
* class ForEach<E> extends CountedCompleter<Void> {
*
* public static <E> void forEach(E[] array, MyOperation<E> op) {
* new ForEach<E>(null, array, op, 0, array.length).invoke();
* }
*
* final E[] array; final MyOperation<E> op; final int lo, hi;
* ForEach(CountedCompleter<?> p, E[] array, MyOperation<E> op, int lo, int hi) {
* super(p);
* this.array = array; this.op = op; this.lo = lo; this.hi = hi;
* }
*
* public void compute() { // version 1
* if (hi - lo >= 2) {
* int mid = (lo + hi) >>> 1;
* setPendingCount(2); // must set pending count before fork
* new ForEach(this, array, op, mid, hi).fork(); // right child
* new ForEach(this, array, op, lo, mid).fork(); // left child
* }
* else if (hi > lo)
* op.apply(array[lo]);
* tryComplete();
* }
* }}</pre>
*
* This design can be improved by noticing that in the recursive case,
* the task has nothing to do after forking its right task, so can
* directly invoke its left task before returning. (This is an analog
* of tail recursion removal.) Also, because the task returns upon
* executing its left task (rather than falling through to invoke
* {@code tryComplete}) the pending count is set to one:
*
* <pre> {@code
* class ForEach<E> ...
* public void compute() { // version 2
* if (hi - lo >= 2) {
* int mid = (lo + hi) >>> 1;
* setPendingCount(1); // only one pending
* new ForEach(this, array, op, mid, hi).fork(); // right child
* new ForEach(this, array, op, lo, mid).compute(); // direct invoke
* }
* else {
* if (hi > lo)
* op.apply(array[lo]);
* tryComplete();
* }
* }
* }</pre>
*
* As a further improvement, notice that the left task need not even
* exist. Instead of creating a new one, we can iterate using the
* original task, and add a pending count for each fork. Additionally,
* because no task in this tree implements an {@link #onCompletion}
* method, {@code tryComplete()} can be replaced with {@link
* #propagateCompletion}.
*
* <pre> {@code
* class ForEach<E> ...
* public void compute() { // version 3
* int l = lo, h = hi;
* while (h - l >= 2) {
* int mid = (l + h) >>> 1;
* addToPendingCount(1);
* new ForEach(this, array, op, mid, h).fork(); // right child
* h = mid;
* }
* if (h > l)
* op.apply(array[l]);
* propagateCompletion();
* }
* }</pre>
*
* Additional improvements of such classes might entail precomputing
* pending counts so that they can be established in constructors,
* specializing classes for leaf steps, subdividing by say, four,
* instead of two per iteration, and using an adaptive threshold
* instead of always subdividing down to single elements.
*
* <p><b>Searching.</b> A tree of CountedCompleters can search for a
* value or property in different parts of a data structure, and
* report a result in an {@link
* java.util.concurrent.atomic.AtomicReference AtomicReference} as
* soon as one is found. The others can poll the result to avoid
* unnecessary work. (You could additionally {@linkplain #cancel
* cancel} other tasks, but it is usually simpler and more efficient
* to just let them notice that the result is set and if so skip
* further processing.) Illustrating again with an array using full
* partitioning (again, in practice, leaf tasks will almost always
* process more than one element):
*
* <pre> {@code
* class Searcher<E> extends CountedCompleter<E> {
* final E[] array; final AtomicReference<E> result; final int lo, hi;
* Searcher(CountedCompleter<?> p, E[] array, AtomicReference<E> result, int lo, int hi) {
* super(p);
* this.array = array; this.result = result; this.lo = lo; this.hi = hi;
* }
* public E getRawResult() { return result.get(); }
* public void compute() { // similar to ForEach version 3
* int l = lo, h = hi;
* while (result.get() == null && h >= l) {
* if (h - l >= 2) {
* int mid = (l + h) >>> 1;
* addToPendingCount(1);
* new Searcher(this, array, result, mid, h).fork();
* h = mid;
* }
* else {
* E x = array[l];
* if (matches(x) && result.compareAndSet(null, x))
* quietlyCompleteRoot(); // root task is now joinable
* break;
* }
* }
* tryComplete(); // normally complete whether or not found
* }
* boolean matches(E e) { ... } // return true if found
*
* public static <E> E search(E[] array) {
* return new Searcher<E>(null, array, new AtomicReference<E>(), 0, array.length).invoke();
* }
*}}</pre>
*
* In this example, as well as others in which tasks have no other
* effects except to compareAndSet a common result, the trailing
* unconditional invocation of {@code tryComplete} could be made
* conditional ({@code if (result.get() == null) tryComplete();})
* because no further bookkeeping is required to manage completions
* once the root task completes.
*
* <p><b>Recording subtasks.</b> CountedCompleter tasks that combine
* results of multiple subtasks usually need to access these results
* in method {@link #onCompletion}. As illustrated in the following
* class (that performs a simplified form of map-reduce where mappings
* and reductions are all of type {@code E}), one way to do this in
* divide and conquer designs is to have each subtask record its
* sibling, so that it can be accessed in method {@code onCompletion}.
* This technique applies to reductions in which the order of
* combining left and right results does not matter; ordered
* reductions require explicit left/right designations. Variants of
* other streamlinings seen in the above examples may also apply.
*
* <pre> {@code
* class MyMapper<E> { E apply(E v) { ... } }
* class MyReducer<E> { E apply(E x, E y) { ... } }
* class MapReducer<E> extends CountedCompleter<E> {
* final E[] array; final MyMapper<E> mapper;
* final MyReducer<E> reducer; final int lo, hi;
* MapReducer<E> sibling;
* E result;
* MapReducer(CountedCompleter<?> p, E[] array, MyMapper<E> mapper,
* MyReducer<E> reducer, int lo, int hi) {
* super(p);
* this.array = array; this.mapper = mapper;
* this.reducer = reducer; this.lo = lo; this.hi = hi;
* }
* public void compute() {
* if (hi - lo >= 2) {
* int mid = (lo + hi) >>> 1;
* MapReducer<E> left = new MapReducer(this, array, mapper, reducer, lo, mid);
* MapReducer<E> right = new MapReducer(this, array, mapper, reducer, mid, hi);
* left.sibling = right;
* right.sibling = left;
* setPendingCount(1); // only right is pending
* right.fork();
* left.compute(); // directly execute left
* }
* else {
* if (hi > lo)
* result = mapper.apply(array[lo]);
* tryComplete();
* }
* }
* public void onCompletion(CountedCompleter<?> caller) {
* if (caller != this) {
* MapReducer<E> child = (MapReducer<E>)caller;
* MapReducer<E> sib = child.sibling;
* if (sib == null || sib.result == null)
* result = child.result;
* else
* result = reducer.apply(child.result, sib.result);
* }
* }
* public E getRawResult() { return result; }
*
* public static <E> E mapReduce(E[] array, MyMapper<E> mapper, MyReducer<E> reducer) {
* return new MapReducer<E>(null, array, mapper, reducer,
* 0, array.length).invoke();
* }
* }}</pre>
*
* Here, method {@code onCompletion} takes a form common to many
* completion designs that combine results. This callback-style method
* is triggered once per task, in either of the two different contexts
* in which the pending count is, or becomes, zero: (1) by a task
* itself, if its pending count is zero upon invocation of {@code
* tryComplete}, or (2) by any of its subtasks when they complete and
* decrement the pending count to zero. The {@code caller} argument
* distinguishes cases. Most often, when the caller is {@code this},
* no action is necessary. Otherwise the caller argument can be used
* (usually via a cast) to supply a value (and/or links to other
* values) to be combined. Assuming proper use of pending counts, the
* actions inside {@code onCompletion} occur (once) upon completion of
* a task and its subtasks. No additional synchronization is required
* within this method to ensure thread safety of accesses to fields of
* this task or other completed tasks.
*
* <p><b>Completion Traversals</b>. If using {@code onCompletion} to
* process completions is inapplicable or inconvenient, you can use
* methods {@link #firstComplete} and {@link #nextComplete} to create
* custom traversals. For example, to define a MapReducer that only
* splits out right-hand tasks in the form of the third ForEach
* example, the completions must cooperatively reduce along
* unexhausted subtask links, which can be done as follows:
*
* <pre> {@code
* class MapReducer<E> extends CountedCompleter<E> { // version 2
* final E[] array; final MyMapper<E> mapper;
* final MyReducer<E> reducer; final int lo, hi;
* MapReducer<E> forks, next; // record subtask forks in list
* E result;
* MapReducer(CountedCompleter<?> p, E[] array, MyMapper<E> mapper,
* MyReducer<E> reducer, int lo, int hi, MapReducer<E> next) {
* super(p);
* this.array = array; this.mapper = mapper;
* this.reducer = reducer; this.lo = lo; this.hi = hi;
* this.next = next;
* }
* public void compute() {
* int l = lo, h = hi;
* while (h - l >= 2) {
* int mid = (l + h) >>> 1;
* addToPendingCount(1);
* (forks = new MapReducer(this, array, mapper, reducer, mid, h, forks)).fork;
* h = mid;
* }
* if (h > l)
* result = mapper.apply(array[l]);
* // process completions by reducing along and advancing subtask links
* for (CountedCompleter<?> c = firstComplete(); c != null; c = c.nextComplete()) {
* for (MapReducer t = (MapReducer)c, s = t.forks; s != null; s = t.forks = s.next)
* t.result = reducer.apply(t.result, s.result);
* }
* }
* public E getRawResult() { return result; }
*
* public static <E> E mapReduce(E[] array, MyMapper<E> mapper, MyReducer<E> reducer) {
* return new MapReducer<E>(null, array, mapper, reducer,
* 0, array.length, null).invoke();
* }
* }}</pre>
*
* <p><b>Triggers.</b> Some CountedCompleters are themselves never
* forked, but instead serve as bits of plumbing in other designs;
* including those in which the completion of one of more async tasks
* triggers another async task. For example:
*
* <pre> {@code
* class HeaderBuilder extends CountedCompleter<...> { ... }
* class BodyBuilder extends CountedCompleter<...> { ... }
* class PacketSender extends CountedCompleter<...> {
* PacketSender(...) { super(null, 1); ... } // trigger on second completion
* public void compute() { } // never called
* public void onCompletion(CountedCompleter<?> caller) { sendPacket(); }
* }
* // sample use:
* PacketSender p = new PacketSender();
* new HeaderBuilder(p, ...).fork();
* new BodyBuilder(p, ...).fork();
* }</pre>
*
* @since 1.8
* @author Doug Lea
*/
public abstract class CountedCompleter<T> extends ForkJoinTask<T> {
private static final long serialVersionUID = 5232453752276485070L;
/** This task's completer, or null if none */
final CountedCompleter<?> completer;
/** The number of pending tasks until completion */
volatile int pending;
/**
* Creates a new CountedCompleter with the given completer
* and initial pending count.
*
* @param completer this task's completer, or {@code null} if none
* @param initialPendingCount the initial pending count
*/
protected CountedCompleter(CountedCompleter<?> completer,
int initialPendingCount) {
this.completer = completer;
this.pending = initialPendingCount;
}
/**
* Creates a new CountedCompleter with the given completer
* and an initial pending count of zero.
*
* @param completer this task's completer, or {@code null} if none
*/
protected CountedCompleter(CountedCompleter<?> completer) {
this.completer = completer;
}
/**
* Creates a new CountedCompleter with no completer
* and an initial pending count of zero.
*/
protected CountedCompleter() {
this.completer = null;
}
/**
* The main computation performed by this task.
*/
public abstract void compute();
/**
* Performs an action when method {@link #tryComplete} is invoked
* and the pending count is zero, or when the unconditional
* method {@link #complete} is invoked. By default, this method
* does nothing. You can distinguish cases by checking the
* identity of the given caller argument. If not equal to {@code
* this}, then it is typically a subtask that may contain results
* (and/or links to other results) to combine.
*
* @param caller the task invoking this method (which may
* be this task itself).
*/
public void onCompletion(CountedCompleter<?> caller) {
}
/**
* Performs an action when method {@link #completeExceptionally}
* is invoked or method {@link #compute} throws an exception, and
* this task has not otherwise already completed normally. On
* entry to this method, this task {@link
* ForkJoinTask#isCompletedAbnormally}. The return value of this
* method controls further propagation: If {@code true} and this
* task has a completer, then this completer is also completed
* exceptionally. The default implementation of this method does
* nothing except return {@code true}.
*
* @param ex the exception
* @param caller the task invoking this method (which may
* be this task itself).
* @return true if this exception should be propagated to this
* task's completer, if one exists.
*/
public boolean onExceptionalCompletion(Throwable ex, CountedCompleter<?> caller) {
return true;
}
/**
* Returns the completer established in this task's constructor,
* or {@code null} if none.
*
* @return the completer
*/
public final CountedCompleter<?> getCompleter() {
return completer;
}
/**
* Returns the current pending count.
*
* @return the current pending count
*/
public final int getPendingCount() {
return pending;
}
/**
* Sets the pending count to the given value.
*
* @param count the count
*/
public final void setPendingCount(int count) {
pending = count;
}
/**
* Adds (atomically) the given value to the pending count.
*
* @param delta the value to add
*/
public final void addToPendingCount(int delta) {
int c; // note: can replace with intrinsic in jdk8
do {} while (!U.compareAndSwapInt(this, PENDING, c = pending, c+delta));
}
/**
* Sets (atomically) the pending count to the given count only if
* it currently holds the given expected value.
*
* @param expected the expected value
* @param count the new value
* @return true if successful
*/
public final boolean compareAndSetPendingCount(int expected, int count) {
return U.compareAndSwapInt(this, PENDING, expected, count);
}
/**
* If the pending count is nonzero, (atomically) decrements it.
*
* @return the initial (undecremented) pending count holding on entry
* to this method
*/
public final int decrementPendingCountUnlessZero() {
int c;
do {} while ((c = pending) != 0 &&
!U.compareAndSwapInt(this, PENDING, c, c - 1));
return c;
}
/**
* Returns the root of the current computation; i.e., this
* task if it has no completer, else its completer's root.
*
* @return the root of the current computation
*/
public final CountedCompleter<?> getRoot() {
CountedCompleter<?> a = this, p;
while ((p = a.completer) != null)
a = p;
return a;
}
/**
* If the pending count is nonzero, decrements the count;
* otherwise invokes {@link #onCompletion} and then similarly
* tries to complete this task's completer, if one exists,
* else marks this task as complete.
*/
public final void tryComplete() {
CountedCompleter<?> a = this, s = a;
for (int c;;) {
if ((c = a.pending) == 0) {
a.onCompletion(s);
if ((a = (s = a).completer) == null) {
s.quietlyComplete();
return;
}
}
else if (U.compareAndSwapInt(a, PENDING, c, c - 1))
return;
}
}
/**
* Equivalent to {@link #tryComplete} but does not invoke {@link
* #onCompletion} along the completion path: If the pending count
* is nonzero, decrements the count; otherwise, similarly tries to
* complete this task's completer, if one exists, else marks this
* task as complete. This method may be useful in cases where
* {@code onCompletion} should not, or need not, be invoked for
* each completer in a computation.
*/
public final void propagateCompletion() {
CountedCompleter<?> a = this, s = a;
for (int c;;) {
if ((c = a.pending) == 0) {
if ((a = (s = a).completer) == null) {
s.quietlyComplete();
return;
}
}
else if (U.compareAndSwapInt(a, PENDING, c, c - 1))
return;
}
}
/**
* Regardless of pending count, invokes {@link #onCompletion},
* marks this task as complete and further triggers {@link
* #tryComplete} on this task's completer, if one exists. The
* given rawResult is used as an argument to {@link #setRawResult}
* before invoking {@link #onCompletion} or marking this task as
* complete; its value is meaningful only for classes overriding
* {@code setRawResult}.
*
* <p>This method may be useful when forcing completion as soon as
* any one (versus all) of several subtask results are obtained.
* However, in the common (and recommended) case in which {@code
* setRawResult} is not overridden, this effect can be obtained
* more simply using {@code quietlyCompleteRoot();}.
*
* @param rawResult the raw result
*/
public void complete(T rawResult) {
CountedCompleter<?> p;
setRawResult(rawResult);
onCompletion(this);
quietlyComplete();
if ((p = completer) != null)
p.tryComplete();
}
/**
* If this task's pending count is zero, returns this task;
* otherwise decrements its pending count and returns {@code
* null}. This method is designed to be used with {@link
* #nextComplete} in completion traversal loops.
*
* @return this task, if pending count was zero, else {@code null}
*/
public final CountedCompleter<?> firstComplete() {
for (int c;;) {
if ((c = pending) == 0)
return this;
else if (U.compareAndSwapInt(this, PENDING, c, c - 1))
return null;
}
}
/**
* If this task does not have a completer, invokes {@link
* ForkJoinTask#quietlyComplete} and returns {@code null}. Or, if
* this task's pending count is non-zero, decrements its pending
* count and returns {@code null}. Otherwise, returns the
* completer. This method can be used as part of a completion
* traversal loop for homogeneous task hierarchies:
*
* <pre> {@code
* for (CountedCompleter<?> c = firstComplete();
* c != null;
* c = c.nextComplete()) {
* // ... process c ...
* }}</pre>
*
* @return the completer, or {@code null} if none
*/
public final CountedCompleter<?> nextComplete() {
CountedCompleter<?> p;
if ((p = completer) != null)
return p.firstComplete();
else {
quietlyComplete();
return null;
}
}
/**
* Equivalent to {@code getRoot().quietlyComplete()}.
*/
public final void quietlyCompleteRoot() {
for (CountedCompleter<?> a = this, p;;) {
if ((p = a.completer) == null) {
a.quietlyComplete();
return;
}
a = p;
}
}
/**
* Supports ForkJoinTask exception propagation.
*/
void internalPropagateException(Throwable ex) {
CountedCompleter<?> a = this, s = a;
while (a.onExceptionalCompletion(ex, s) &&
(a = (s = a).completer) != null && a.status >= 0)
a.recordExceptionalCompletion(ex);
}
/**
* Implements execution conventions for CountedCompleters.
*/
protected final boolean exec() {
compute();
return false;
}
/**
* Returns the result of the computation. By default
* returns {@code null}, which is appropriate for {@code Void}
* actions, but in other cases should be overridden, almost
* always to return a field or function of a field that
* holds the result upon completion.
*
* @return the result of the computation
*/
public T getRawResult() { return null; }
/**
* A method that result-bearing CountedCompleters may optionally
* use to help maintain result data. By default, does nothing.
* Overrides are not recommended. However, if this method is
* overridden to update existing objects or fields, then it must
* in general be defined to be thread-safe.
*/
protected void setRawResult(T t) { }
// Unsafe mechanics
private static final sun.misc.Unsafe U;
private static final long PENDING;
static {
try {
U = sun.misc.Unsafe.getUnsafe();
PENDING = U.objectFieldOffset
(CountedCompleter.class.getDeclaredField("pending"));
} catch (Exception e) {
throw new Error(e);
}
}
}
...@@ -40,7 +40,6 @@ import java.util.Arrays; ...@@ -40,7 +40,6 @@ 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;
...@@ -48,11 +47,6 @@ import java.util.concurrent.Future; ...@@ -48,11 +47,6 @@ import java.util.concurrent.Future;
import java.util.concurrent.RejectedExecutionException; 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.atomic.AtomicInteger;
import java.util.concurrent.locks.LockSupport;
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.
...@@ -63,21 +57,31 @@ import java.util.concurrent.locks.Condition; ...@@ -63,21 +57,31 @@ import java.util.concurrent.locks.Condition;
* <p>A {@code ForkJoinPool} differs from other kinds of {@link * <p>A {@code ForkJoinPool} differs from other kinds of {@link
* ExecutorService} mainly by virtue of employing * ExecutorService} mainly by virtue of employing
* <em>work-stealing</em>: all threads in the pool attempt to find and * <em>work-stealing</em>: all threads in the pool attempt to find and
* execute subtasks created by other active tasks (eventually blocking * execute tasks submitted to the pool and/or created by other active
* waiting for work if none exist). This enables efficient processing * tasks (eventually blocking waiting for work if none exist). This
* when most tasks spawn other subtasks (as do most {@code * enables efficient processing when most tasks spawn other subtasks
* ForkJoinTask}s). When setting <em>asyncMode</em> to true in * (as do most {@code ForkJoinTask}s), as well as when many small
* constructors, {@code ForkJoinPool}s may also be appropriate for use * tasks are submitted to the pool from external clients. Especially
* with event-style tasks that are never joined. * when setting <em>asyncMode</em> to true in constructors, {@code
* ForkJoinPool}s may also be appropriate for use with event-style
* tasks that are never joined.
* *
* <p>A {@code ForkJoinPool} is constructed with a given target * <p>A static {@link #commonPool()} is available and appropriate for
* parallelism level; by default, equal to the number of available * most applications. The common pool is used by any ForkJoinTask that
* processors. The pool attempts to maintain enough active (or * is not explicitly submitted to a specified pool. Using the common
* available) threads by dynamically adding, suspending, or resuming * pool normally reduces resource usage (its threads are slowly
* internal worker threads, even if some tasks are stalled waiting to * reclaimed during periods of non-use, and reinstated upon subsequent
* join others. However, no such adjustments are guaranteed in the * use).
* face of blocked IO or other unmanaged synchronization. The nested *
* {@link ManagedBlocker} interface enables extension of the kinds of * <p>For applications that require separate or custom pools, a {@code
* ForkJoinPool} may be constructed with a given target parallelism
* level; by default, equal to the number of available processors. The
* pool attempts to maintain enough active (or available) threads by
* dynamically adding, suspending, or resuming internal worker
* threads, even if some tasks are stalled waiting to join
* others. However, no such adjustments are guaranteed in the face of
* blocked I/O or other unmanaged synchronization. The nested {@link
* ManagedBlocker} interface enables extension of the kinds of
* synchronization accommodated. * synchronization accommodated.
* *
* <p>In addition to execution and lifecycle control methods, this * <p>In addition to execution and lifecycle control methods, this
...@@ -87,16 +91,17 @@ import java.util.concurrent.locks.Condition; ...@@ -87,16 +91,17 @@ import java.util.concurrent.locks.Condition;
* {@link #toString} returns indications of pool state in a * {@link #toString} returns indications of pool state in a
* convenient form for informal monitoring. * convenient form for informal monitoring.
* *
* <p> As is the case with other ExecutorServices, there are three * <p>As is the case with other ExecutorServices, there are three
* main task execution methods summarized in the following * main task execution methods summarized in the following table.
* table. These are designed to be used by clients not already engaged * These are designed to be used primarily by clients not already
* in fork/join computations in the current pool. The main forms of * engaged in fork/join computations in the current pool. The main
* these methods accept instances of {@code ForkJoinTask}, but * forms of these methods accept instances of {@code ForkJoinTask},
* overloaded forms also allow mixed execution of plain {@code * but overloaded forms also allow mixed execution of plain {@code
* Runnable}- or {@code Callable}- based activities as well. However, * Runnable}- or {@code Callable}- based activities as well. However,
* tasks that are already executing in a pool should normally * tasks that are already executing in a pool should normally instead
* <em>NOT</em> use these pool execution methods, but instead use the * use the within-computation forms listed in the table unless using
* within-computation forms listed in the table. * async event-style tasks that are not usually joined, in which case
* there is little difference among choice of methods.
* *
* <table BORDER CELLPADDING=3 CELLSPACING=1> * <table BORDER CELLPADDING=3 CELLSPACING=1>
* <tr> * <tr>
...@@ -121,23 +126,16 @@ import java.util.concurrent.locks.Condition; ...@@ -121,23 +126,16 @@ import java.util.concurrent.locks.Condition;
* </tr> * </tr>
* </table> * </table>
* *
* <p><b>Sample Usage.</b> Normally a single {@code ForkJoinPool} is * <p>The common pool is by default constructed with default
* used for all parallel task execution in a program or subsystem. * parameters, but these may be controlled by setting three {@link
* Otherwise, use would not usually outweigh the construction and * System#getProperty system properties} with prefix {@code
* bookkeeping overhead of creating a large set of threads. For * java.util.concurrent.ForkJoinPool.common}: {@code parallelism} --
* example, a common pool could be used for the {@code SortTasks} * an integer greater than zero, {@code threadFactory} -- the class
* illustrated in {@link RecursiveAction}. Because {@code * name of a {@link ForkJoinWorkerThreadFactory}, and {@code
* ForkJoinPool} uses threads in {@linkplain java.lang.Thread#isDaemon * exceptionHandler} -- the class name of a {@link
* daemon} mode, there is typically no need to explicitly {@link * java.lang.Thread.UncaughtExceptionHandler
* #shutdown} such a pool upon program exit. * Thread.UncaughtExceptionHandler}. Upon any error in establishing
* * these settings, default parameters are used.
* <pre>
* static final ForkJoinPool mainPool = new ForkJoinPool();
* ...
* public void sort(long[] array) {
* mainPool.invoke(new SortTask(array, 0, array.length));
* }
* </pre>
* *
* <p><b>Implementation notes</b>: This implementation restricts the * <p><b>Implementation notes</b>: This implementation restricts the
* maximum number of running threads to 32767. Attempts to create * maximum number of running threads to 32767. Attempts to create
...@@ -156,213 +154,387 @@ public class ForkJoinPool extends AbstractExecutorService { ...@@ -156,213 +154,387 @@ public class ForkJoinPool extends AbstractExecutorService {
/* /*
* Implementation Overview * Implementation Overview
* *
* This class provides the central bookkeeping and control for a * This class and its nested classes provide the main
* set of worker threads: Submissions from non-FJ threads enter * functionality and control for a set of worker threads:
* into a submission queue. Workers take these tasks and typically * Submissions from non-FJ threads enter into submission queues.
* split them into subtasks that may be stolen by other workers. * Workers take these tasks and typically split them into subtasks
* Preference rules give first priority to processing tasks from * that may be stolen by other workers. Preference rules give
* their own queues (LIFO or FIFO, depending on mode), then to * first priority to processing tasks from their own queues (LIFO
* randomized FIFO steals of tasks in other worker queues, and * or FIFO, depending on mode), then to randomized FIFO steals of
* lastly to new submissions. * tasks in other queues.
*
* WorkQueues
* ==========
*
* Most operations occur within work-stealing queues (in nested
* class WorkQueue). These are special forms of Deques that
* support only three of the four possible end-operations -- push,
* pop, and poll (aka steal), under the further constraints that
* push and pop are called only from the owning thread (or, as
* extended here, under a lock), while poll may be called from
* other threads. (If you are unfamiliar with them, you probably
* want to read Herlihy and Shavit's book "The Art of
* Multiprocessor programming", chapter 16 describing these in
* more detail before proceeding.) The main work-stealing queue
* design is roughly similar to those in the papers "Dynamic
* Circular Work-Stealing Deque" by Chase and Lev, SPAA 2005
* (http://research.sun.com/scalable/pubs/index.html) and
* "Idempotent work stealing" by Michael, Saraswat, and Vechev,
* PPoPP 2009 (http://portal.acm.org/citation.cfm?id=1504186).
* The main differences ultimately stem from GC requirements that
* we null out taken slots as soon as we can, to maintain as small
* a footprint as possible even in programs generating huge
* numbers of tasks. To accomplish this, we shift the CAS
* arbitrating pop vs poll (steal) from being on the indices
* ("base" and "top") to the slots themselves. So, both a
* successful pop and poll mainly entail a CAS of a slot from
* non-null to null. Because we rely on CASes of references, we
* do not need tag bits on base or top. They are simple ints as
* used in any circular array-based queue (see for example
* ArrayDeque). Updates to the indices must still be ordered in a
* way that guarantees that top == base means the queue is empty,
* but otherwise may err on the side of possibly making the queue
* appear nonempty when a push, pop, or poll have not fully
* committed. Note that this means that the poll operation,
* considered individually, is not wait-free. One thief cannot
* 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.
* If an attempted steal fails, a thief always chooses a different
* random victim target to try next. So, in order for one thief to
* progress, it suffices for any in-progress poll or new push on
* any empty queue to complete. (This is why we normally use
* method pollAt and its variants that try once at the apparent
* base index, else consider alternative actions, rather than
* method poll.)
*
* This approach also enables support of a user mode in which local
* task processing is in FIFO, not LIFO order, simply by using
* poll rather than pop. This can be useful in message-passing
* 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.)
*
* WorkQueues are also used in a similar way for tasks submitted
* to the pool. We cannot mix these tasks in the same queues used
* for work-stealing (this would contaminate lifo/fifo
* processing). Instead, we randomly associate submission queues
* with submitting threads, using a form of hashing. The
* ThreadLocal Submitter class contains a value initially used as
* a hash code for choosing existing queues, but may be randomly
* repositioned upon contention with other submitters. In
* essence, submitters act like workers except that they are
* restricted to executing local tasks that they submitted (or in
* the case of CountedCompleters, others with the same root task).
* However, because most shared/external queue operations are more
* expensive than internal, and because, at steady state, external
* submitters will compete for CPU with workers, ForkJoinTask.join
* and related methods disable them from repeatedly helping to
* process tasks if all workers are active. Insertion of tasks in
* shared mode requires a lock (mainly to protect in the case of
* resizing) but we use only a simple spinlock (using bits in
* field qlock), because submitters encountering a busy queue move
* on to try or create other queues -- they block only when
* creating and registering new queues.
*
* Management
* ==========
* *
* The main throughput advantages of work-stealing stem from * The main throughput advantages of work-stealing stem from
* decentralized control -- workers mostly take tasks from * decentralized control -- workers mostly take tasks from
* themselves or each other. We cannot negate this in the * themselves or each other. We cannot negate this in the
* implementation of other management responsibilities. The main * implementation of other management responsibilities. The main
* tactic for avoiding bottlenecks is packing nearly all * tactic for avoiding bottlenecks is packing nearly all
* essentially atomic control state into a single 64bit volatile * essentially atomic control state into two volatile variables
* variable ("ctl"). This variable is read on the order of 10-100 * that are by far most often read (not written) as status and
* times as often as it is modified (always via CAS). (There is * consistency checks.
* some additional control state, for example variable "shutdown" *
* for which we can cope with uncoordinated updates.) This * Field "ctl" contains 64 bits holding all the information needed
* streamlines synchronization and control at the expense of messy * to atomically decide to add, inactivate, enqueue (on an event
* constructions needed to repack status bits upon updates. * queue), dequeue, and/or re-activate workers. To enable this
* Updates tend not to contend with each other except during * packing, we restrict maximum parallelism to (1<<15)-1 (which is
* bursts while submitted tasks begin or end. In some cases when * far in excess of normal operating range) to allow ids, counts,
* they do contend, threads can instead do something else * and their negations (used for thresholding) to fit into 16bit
* (usually, scan for tasks) until contention subsides. * fields.
* *
* To enable packing, we restrict maximum parallelism to (1<<15)-1 * Field "plock" is a form of sequence lock with a saturating
* (which is far in excess of normal operating range) to allow * shutdown bit (similarly for per-queue "qlocks"), mainly
* ids, counts, and their negations (used for thresholding) to fit * protecting updates to the workQueues array, as well as to
* into 16bit fields. * enable shutdown. When used as a lock, it is normally only very
* * briefly held, so is nearly always available after at most a
* Recording Workers. Workers are recorded in the "workers" array * brief spin, but we use a monitor-based backup strategy to
* that is created upon pool construction and expanded if (rarely) * block when needed.
* necessary. This is an array as opposed to some other data *
* structure to support index-based random steals by workers. * Recording WorkQueues. WorkQueues are recorded in the
* Updates to the array recording new workers and unrecording * "workQueues" array that is created upon first use and expanded
* terminated ones are protected from each other by a seqLock * if necessary. Updates to the array while recording new workers
* (scanGuard) but the array is otherwise concurrently readable, * and unrecording terminated ones are protected from each other
* and accessed directly by workers. To simplify index-based * by a lock but the array is otherwise concurrently readable, and
* operations, the array size is always a power of two, and all * accessed directly. To simplify index-based operations, the
* readers must tolerate null slots. To avoid flailing during * array size is always a power of two, and all readers must
* start-up, the array is presized to hold twice #parallelism * tolerate null slots. Worker queues are at odd indices. Shared
* workers (which is unlikely to need further resizing during * (submission) queues are at even indices, up to a maximum of 64
* execution). But to avoid dealing with so many null slots, * slots, to limit growth even if array needs to expand to add
* variable scanGuard includes a mask for the nearest power of two * more workers. Grouping them together in this way simplifies and
* that contains all current workers. All worker thread creation * speeds up task scanning.
* is on-demand, triggered by task submissions, replacement of *
* terminated workers, and/or compensation for blocked * All worker thread creation is on-demand, triggered by task
* workers. However, all other support code is set up to work with * submissions, replacement of terminated workers, and/or
* other policies. To ensure that we do not hold on to worker * compensation for blocked workers. However, all other support
* references that would prevent GC, ALL accesses to workers are * code is set up to work with other policies. To ensure that we
* via indices into the workers array (which is one source of some * do not hold on to worker references that would prevent GC, ALL
* of the messy code constructions here). In essence, the workers * accesses to workQueues are via indices into the workQueues
* array serves as a weak reference mechanism. Thus for example * array (which is one source of some of the messy code
* the wait queue field of ctl stores worker indices, not worker * constructions here). In essence, the workQueues array serves as
* references. Access to the workers in associated methods (for * a weak reference mechanism. Thus for example the wait queue
* example signalWork) must both index-check and null-check the * field of ctl stores indices, not references. Access to the
* IDs. All such accesses ignore bad IDs by returning out early * workQueues in associated methods (for example signalWork) must
* from what they are doing, since this can only be associated * both index-check and null-check the IDs. All such accesses
* with termination, in which case it is OK to give up. * ignore bad IDs by returning out early from what they are doing,
* * since this can only be associated with termination, in which
* All uses of the workers array, as well as queue arrays, check * case it is OK to give up. All uses of the workQueues array
* that the array is non-null (even if previously non-null). This * also check that it is non-null (even if previously
* allows nulling during termination, which is currently not * non-null). This allows nulling during termination, which is
* necessary, but remains an option for resource-revocation-based * currently not necessary, but remains an option for
* shutdown schemes. * resource-revocation-based shutdown schemes. It also helps
* * reduce JIT issuance of uncommon-trap code, which tends to
* Wait Queuing. Unlike HPC work-stealing frameworks, we cannot * unnecessarily complicate control flow in some methods.
*
* Event Queuing. Unlike HPC work-stealing frameworks, we cannot
* let workers spin indefinitely scanning for tasks when none can * let workers spin indefinitely scanning for tasks when none can
* be found immediately, and we cannot start/resume workers unless * be found immediately, and we cannot start/resume workers unless
* there appear to be tasks available. On the other hand, we must * there appear to be tasks available. On the other hand, we must
* quickly prod them into action when new tasks are submitted or * quickly prod them into action when new tasks are submitted or
* generated. We park/unpark workers after placing in an event * generated. In many usages, ramp-up time to activate workers is
* wait queue when they cannot find work. This "queue" is actually * the main limiting factor in overall performance (this is
* a simple Treiber stack, headed by the "id" field of ctl, plus a * compounded at program start-up by JIT compilation and
* 15bit counter value to both wake up waiters (by advancing their * allocation). So we try to streamline this as much as possible.
* count) and avoid ABA effects. Successors are held in worker * We park/unpark workers after placing in an event wait queue
* field "nextWait". Queuing deals with several intrinsic races, * when they cannot find work. This "queue" is actually a simple
* mainly that a task-producing thread can miss seeing (and * Treiber stack, headed by the "id" field of ctl, plus a 15bit
* counter value (that reflects the number of times a worker has
* been inactivated) to avoid ABA effects (we need only as many
* version numbers as worker threads). Successors are held in
* field WorkQueue.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 * signalling) another thread that gave up looking for work but
* has not yet entered the wait queue. We solve this by requiring * has not yet entered the wait queue. We solve this by requiring
* a full sweep of all workers both before (in scan()) and after * a full sweep of all workers (via repeated calls to method
* (in tryAwaitWork()) a newly waiting worker is added to the wait * scan()) both before and after a newly waiting worker is added
* queue. During a rescan, the worker might release some other * to the wait queue. During a rescan, the worker might release
* queued worker rather than itself, which has the same net * some other queued worker rather than itself, which has the same
* effect. Because enqueued workers may actually be rescanning * net effect. Because enqueued workers may actually be rescanning
* rather than waiting, we set and clear the "parked" field of * rather than waiting, we set and clear the "parker" field of
* ForkJoinWorkerThread to reduce unnecessary calls to unpark. * WorkQueues to reduce unnecessary calls to unpark. (This
* (Use of the parked field requires a secondary recheck to avoid * requires a secondary recheck to avoid missed signals.) Note
* missed signals.) * the unusual conventions about Thread.interrupts surrounding
* parking and other blocking: Because interrupts are used solely
* to alert threads to check termination, which is checked anyway
* upon blocking, 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.
* *
* Signalling. We create or wake up workers only when there * Signalling. We create or wake up workers only when there
* appears to be at least one task they might be able to find and * 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 * execute. However, many other threads may notice the same task
* task to a queue that previously had two or fewer tasks, they * and each signal to wake up a thread that might take it. So in
* signal waiting workers (or trigger creation of new ones if * general, pools will be over-signalled. When a submission is
* fewer than the given parallelism level -- see signalWork). * added or another worker adds a task to a queue that has fewer
* These primary signals are buttressed by signals during rescans * than two tasks, they signal waiting workers (or trigger
* as well as those performed when a worker steals a task and * creation of new ones if fewer than the given parallelism level
* notices that there are more tasks too; together these cover the * -- signalWork), and may leave a hint to the unparked worker to
* signals needed in cases when more than two tasks are pushed * help signal others upon wakeup). These primary signals are
* but untaken. * buttressed by others (see method helpSignal) whenever other
* threads scan for work or do not have a task to process. On
* most platforms, signalling (unpark) overhead time is noticeably
* long, and the time between signalling a thread and it actually
* making progress can be very noticeably long, so it is worth
* offloading these delays from critical paths as much as
* possible.
* *
* Trimming workers. To release resources after periods of lack of * Trimming workers. To release resources after periods of lack of
* use, a worker starting to wait when the pool is quiescent will * use, a worker starting to wait when the pool is quiescent will
* time out and terminate if the pool has remained quiescent for * time out and terminate if the pool has remained quiescent for a
* SHRINK_RATE nanosecs. This will slowly propagate, eventually * given period -- a short period if there are more threads than
* terminating all workers after long periods of non-use. * parallelism, longer as the number of threads decreases. This
* * will slowly propagate, eventually terminating all workers after
* Submissions. External submissions are maintained in an * periods of non-use.
* array-based queue that is structured identically to *
* ForkJoinWorkerThread queues except for the use of * Shutdown and Termination. A call to shutdownNow atomically sets
* submissionLock in method addSubmission. Unlike the case for * a plock bit and then (non-atomically) sets each worker's
* worker queues, multiple external threads can add new * qlock status, cancels all unprocessed tasks, and wakes up
* submissions, so adding requires a lock. * all waiting workers. Detecting whether termination should
* * commence after a non-abrupt shutdown() call requires more work
* Compensation. Beyond work-stealing support and lifecycle * and bookkeeping. We need consensus about quiescence (i.e., that
* control, the main responsibility of this framework is to take * there is no more work). The active count provides a primary
* actions when one worker is waiting to join a task stolen (or * indication but non-abrupt shutdown still requires a rechecking
* always held by) another. Because we are multiplexing many * scan for any workers that are inactive but not queued.
* 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 * Joining Tasks
* 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 * Any of several actions may be taken when one worker is waiting
* unblocked task and its continuation to progress. Instead we * to join a task stolen (or always held) by another. Because we
* combine two tactics: * 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:
* *
* 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.
* ForkJoinWorkerThread.joinTask tracks joining->stealing
* 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 tryPreBlock() may create or re-activate a spare * method tryCompensate() may create or re-activate a spare
* thread to compensate for blocked joiners until they * thread to compensate for blocked joiners until they unblock.
* unblock. *
* A third form (implemented in tryRemoveAndExec) amounts to
* helping a hypothetical compensator: If we can readily tell that
* a possible action of a compensator is to steal and execute the
* task being joined, the joining thread can do so directly,
* without the need for a compensation thread (although at the
* expense of larger run-time stacks, but the tradeoff is
* typically worthwhile).
* *
* 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 algorithm in tryHelpStealer entails a form of "linear"
* helping: Each worker records (in field currentSteal) the most
* recent task it stole from some other worker. Plus, it records
* (in field currentJoin) the task it is currently actively
* joining. Method tryHelpStealer uses these markers to try to
* find a worker to help (i.e., steal back a task from and execute
* it) that could hasten completion of the actively joined task.
* In essence, the joiner executes a task that would be on its own
* local deque had the to-be-joined task not been stolen. This may
* be seen as a conservative variant of the approach in Wagner &
* Calder "Leapfrogging: a portable technique for implementing
* efficient futures" SIGPLAN Notices, 1993
* (http://portal.acm.org/citation.cfm?id=155354). It differs in
* that: (1) We only maintain dependency links across workers upon
* steals, rather than use per-task bookkeeping. This sometimes
* requires a linear scan of workQueues array to locate stealers,
* but often doesn't because stealers leave hints (that may become
* stale/wrong) of where to locate them. It is only a hint
* because a worker might have had multiple steals and the hint
* records only one of them (usually the most current). Hinting
* isolates cost to when it is needed, rather than adding to
* per-task overhead. (2) It is "shallow", ignoring nesting and
* potentially cyclic mutual steals. (3) It is intentionally
* racy: field currentJoin is updated only while actively joining,
* which means that we miss links in the chain during long-lived
* tasks, GC stalls etc (which is OK since blocking in such cases
* is usually a good idea). (4) We bound the number of attempts
* to find work (see MAX_HELP) and fall back to suspending the
* worker and if necessary replacing it with another.
*
* Helping actions for CountedCompleters are much simpler: Method
* helpComplete can take and execute any task with the same root
* as the task being waited on. However, this still entails some
* traversal of completer chains, so is less efficient than using
* CountedCompleters without explicit joins.
*
* It is impossible to keep exactly the target parallelism number * It is impossible to keep exactly the target parallelism number
* of threads running at any given time. Determining the * of threads running at any given time. Determining the
* existence of conservatively safe helping targets, the * existence of conservatively safe helping targets, the
* availability of already-created spares, and the apparent need * availability of already-created spares, and the apparent need
* to create new spares are all racy and require heuristic * to create new spares are all racy, so we rely on multiple
* guidance, so we rely on multiple retries of each. Currently, * retries of each. Compensation in the apparent absence of
* in keeping with on-demand signalling policy, we compensate only * helping opportunities is challenging to control on JVMs, where
* if blocking would leave less than one active (non-waiting, * GC and other activities can stall progress of tasks that in
* non-blocked) worker. Additionally, to avoid some false alarms * turn stall out many other dependent tasks, without us being
* due to GC, lagging counters, system activity, etc, compensated * able to determine whether they will ever require compensation.
* blocking for joins is only attempted after rechecks stabilize * Even though work-stealing otherwise encounters little
* (retries are interspersed with Thread.yield, for good * degradation in the presence of more threads than cores,
* citizenship). The variable blockedCount, incremented before * aggressively adding new threads in such cases entails risk of
* blocking and decremented after, is sometimes needed to * unwanted positive feedback control loops in which more threads
* distinguish cases of waiting for work vs blocking on joins or * cause more dependent stalls (as well as delayed progress of
* other managed sync. Both cases are equivalent for most pool * unblocked threads to the point that we know they are available)
* control, so we can update non-atomically. (Additionally, * leading to more situations requiring more threads, and so
* contention on blockedCount alleviates some contention on ctl). * on. This aspect of control can be seen as an (analytically
* * intractable) game with an opponent that may choose the worst
* Shutdown and Termination. A call to shutdownNow atomically sets * (for us) active thread to stall at any time. We take several
* the ctl stop bit and then (non-atomically) sets each workers * precautions to bound losses (and thus bound gains), mainly in
* "terminate" status, cancels all unprocessed tasks, and wakes up * methods tryCompensate and awaitJoin.
* all waiting workers. Detecting whether termination should *
* commence after a non-abrupt shutdown() call requires more work * Common Pool
* and bookkeeping. We need consensus about quiesence (i.e., that * ===========
* there is no more work) which is reflected in active counts so *
* long as there are no current blockers, as well as possible * The static commonPool always exists after static
* re-evaluations during independent changes in blocking or * initialization. Since it (or any other created pool) need
* quiescing workers. * never be used, we minimize initial construction overhead and
* * footprint to the setup of about a dozen fields, with no nested
* Style notes: There is a lot of representation-level coupling * allocation. Most bootstrapping occurs within method
* among classes ForkJoinPool, ForkJoinWorkerThread, and * fullExternalPush during the first submission to the pool.
* ForkJoinTask. Most fields of ForkJoinWorkerThread maintain *
* data structures managed by ForkJoinPool, so are directly * When external threads submit to the common pool, they can
* accessed. Conversely we allow access to "workers" array by * perform some subtask processing (see externalHelpJoin and
* workers, and direct access to ForkJoinTask.status by both * related methods). We do not need to record whether these
* ForkJoinPool and ForkJoinWorkerThread. There is little point * submissions are to the common pool -- if not, externalHelpJoin
* returns quickly (at the most helping to signal some common pool
* workers). These submitters would otherwise be blocked waiting
* for completion, so the extra effort (with liberally sprinkled
* task status checks) in inapplicable cases amounts to an odd
* form of limited spin-wait before blocking in ForkJoinTask.join.
*
* Style notes
* ===========
*
* There is a lot of representation-level coupling among classes
* ForkJoinPool, ForkJoinWorkerThread, and ForkJoinTask. The
* fields of WorkQueue maintain data structures managed by
* ForkJoinPool, so are directly accessed. 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. All together, these low-level implementation * changes anyway. Several methods intrinsically sprawl because
* choices produce as much as a factor of 4 performance * they must accumulate sets of consistent reads of volatiles held
* improvement compared to naive implementations, and enable the * in local variables. Methods signalWork() and scan() are the
* processing of billions of tasks per second, at the expense of * main bottlenecks, so are especially heavily
* some ugliness. * micro-optimized/mangled. There are lots of inline assignments
* * (of form "while ((local = field) != 0)") which are usually the
* Methods signalWork() and scan() are the main bottlenecks so are * simplest way to ensure the required read orderings (which are
* especially heavily micro-optimized/mangled. There are lots of * sometimes critical). This leads to a "C"-like style of listing
* inline assignments (of form "while ((local = field) != 0)") * declarations of these locals at the heads of methods or blocks.
* which are usually the simplest way to ensure the required read * There are several occurrences of the unusual "do {} while
* orderings (which are sometimes critical). This leads to a * (!cas...)" which is the simplest way to force an update of a
* "C"-like style of listing declarations of these locals at the * CAS'ed variable. There are also other coding oddities (including
* heads of methods or blocks. There are several occurrences of * several unnecessary-looking hoisted null checks) that help
* the unusual "do {} while (!cas...)" which is the simplest way * some methods perform reasonably even when interpreted (not
* to force an update of a CAS'ed variable. There are also other * compiled).
* coding oddities that help some methods perform reasonably even *
* when interpreted (not compiled). * The order of declarations in this file is:
* * (1) Static utility functions
* The order of declarations in this file is: (1) declarations of * (2) Nested (static) classes
* statics (2) fields (along with constants used when unpacking * (3) Static fields
* some of them), listed in an order that tends to reduce * (4) Fields, along with constants used when unpacking some of them
* contention among them a bit under most JVMs. (3) internal * (5) Internal control methods
* control methods (4) callbacks and other support for * (6) Callbacks and other support for ForkJoinTask methods
* ForkJoinTask and ForkJoinWorkerThread classes, (5) exported * (7) Exported methods
* methods (plus a few little helpers). (6) static block * (8) Static block initializing statics in minimally dependent order
* initializing all statics in a minimally dependent order. */
// Static utilities
/**
* If there is a security manager, makes sure caller has
* permission to modify threads.
*/ */
private static void checkPermission() {
SecurityManager security = System.getSecurityManager();
if (security != null)
security.checkPermission(modifyThreadPermission);
}
// Nested classes
/** /**
* Factory for creating new {@link ForkJoinWorkerThread}s. * Factory for creating new {@link ForkJoinWorkerThread}s.
...@@ -384,122 +556,620 @@ public class ForkJoinPool extends AbstractExecutorService { ...@@ -384,122 +556,620 @@ public class ForkJoinPool extends AbstractExecutorService {
* Default ForkJoinWorkerThreadFactory implementation; creates a * Default ForkJoinWorkerThreadFactory implementation; creates a
* new ForkJoinWorkerThread. * new ForkJoinWorkerThread.
*/ */
static class DefaultForkJoinWorkerThreadFactory static final class DefaultForkJoinWorkerThreadFactory
implements ForkJoinWorkerThreadFactory { implements ForkJoinWorkerThreadFactory {
public ForkJoinWorkerThread newThread(ForkJoinPool pool) { public final ForkJoinWorkerThread newThread(ForkJoinPool pool) {
return new ForkJoinWorkerThread(pool); return new ForkJoinWorkerThread(pool);
} }
} }
/** /**
* Creates a new ForkJoinWorkerThread. This factory is used unless * Per-thread records for threads that submit to pools. Currently
* overridden in ForkJoinPool constructors. * holds only pseudo-random seed / index that is used to choose
*/ * submission queues in method externalPush. In the future, this may
public static final ForkJoinWorkerThreadFactory * also incorporate a means to implement different task rejection
defaultForkJoinWorkerThreadFactory; * and resubmission policies.
*
* Seeds for submitters and workers/workQueues work in basically
* the same way but are initialized and updated using slightly
* different mechanics. Both are initialized using the same
* approach as in class ThreadLocal, where successive values are
* unlikely to collide with previous values. Seeds are then
* randomly modified upon collisions using xorshifts, which
* requires a non-zero seed.
*/
static final class Submitter {
int seed;
Submitter(int s) { seed = s; }
}
/**
* Class for artificial tasks that are used to replace the target
* of local joins if they are removed from an interior queue slot
* in WorkQueue.tryRemoveAndExec. We don't need the proxy to
* actually do anything beyond having a unique identity.
*/
static final class EmptyTask extends ForkJoinTask<Void> {
private static final long serialVersionUID = -7721805057305804111L;
EmptyTask() { status = ForkJoinTask.NORMAL; } // force done
public final Void getRawResult() { return null; }
public final void setRawResult(Void x) {}
public final boolean exec() { return true; }
}
/**
* Queues supporting work-stealing as well as external task
* submission. See above for main rationale and algorithms.
* Implementation relies heavily on "Unsafe" intrinsics
* and selective use of "volatile":
*
* Field "base" is the index (mod array.length) of the least valid
* queue slot, which is always the next position to steal (poll)
* from if nonempty. Reads and writes require volatile orderings
* but not CAS, because updates are only performed after slot
* CASes.
*
* Field "top" is the index (mod array.length) of the next queue
* slot to push to or pop from. It is written only by owner thread
* for push, or under lock for external/shared push, and accessed
* by other threads only after reading (volatile) base. Both top
* and base are allowed to wrap around on overflow, but (top -
* base) (or more commonly -(base - top) to force volatile read of
* base before top) still estimates size. The lock ("qlock") is
* forced to -1 on termination, causing all further lock attempts
* to fail. (Note: we don't need CAS for termination state because
* upon pool shutdown, all shared-queues will stop being used
* anyway.) Nearly all lock bodies are set up so that exceptions
* within lock bodies are "impossible" (modulo JVM errors that
* would cause failure anyway.)
*
* The array slots are read and written using the emulation of
* volatiles/atomics provided by Unsafe. Insertions must in
* general use putOrderedObject as a form of releasing store to
* ensure that all writes to the task object are ordered before
* its publication in the queue. All removals entail a CAS to
* null. The array is always a power of two. To ensure safety of
* Unsafe array operations, all accesses perform explicit null
* checks and implicit bounds checks via power-of-two masking.
*
* In addition to basic queuing support, this class contains
* fields described elsewhere to control execution. It turns out
* to work better memory-layout-wise to include them in this class
* rather than a separate class.
*
* Performance on most platforms is very sensitive to placement of
* instances of both WorkQueues and their arrays -- we absolutely
* do not want multiple WorkQueue instances or multiple queue
* arrays sharing cache lines. (It would be best for queue objects
* and their arrays to share, but there is nothing available to
* help arrange that). Unfortunately, because they are recorded
* in a common array, WorkQueue instances are often moved to be
* adjacent by garbage collectors. To reduce impact, we use field
* padding that works OK on common platforms; this effectively
* trades off slightly slower average field access for the sake of
* avoiding really bad worst-case access. (Until better JVM
* support is in place, this padding is dependent on transient
* properties of JVM field layout rules.) We also take care in
* allocating, sizing and resizing the array. Non-shared queue
* arrays are initialized by workers before use. Others are
* allocated on first use.
*/
static final class WorkQueue {
/**
* Capacity of work-stealing queue array upon initialization.
* Must be a power of two; at least 4, but should be larger to
* reduce or eliminate cacheline sharing among queues.
* Currently, it is much larger, as a partial workaround for
* the fact that JVMs often place arrays in locations that
* share GC bookkeeping (especially cardmarks) such that
* per-write accesses encounter serious memory contention.
*/
static final int INITIAL_QUEUE_CAPACITY = 1 << 13;
/** /**
* Permission required for callers of methods that may start or * Maximum size for queue arrays. Must be a power of two less
* kill threads. * than or equal to 1 << (31 - width of array entry) to ensure
*/ * lack of wraparound of index calculations, but defined to a
private static final RuntimePermission modifyThreadPermission; * value a bit less than this to help users trap runaway
* programs before saturating systems.
*/
static final int MAXIMUM_QUEUE_CAPACITY = 1 << 26; // 64M
// Heuristic padding to ameliorate unfortunate memory placements
volatile long pad00, pad01, pad02, pad03, pad04, pad05, pad06;
int seed; // for random scanning; initialize nonzero
volatile int eventCount; // encoded inactivation count; < 0 if inactive
int nextWait; // encoded record of next event waiter
int hint; // steal or signal hint (index)
int poolIndex; // index of this queue in pool (or 0)
final int mode; // 0: lifo, > 0: fifo, < 0: shared
int nsteals; // number of steals
volatile int qlock; // 1: locked, -1: terminate; else 0
volatile int base; // index of next slot for poll
int top; // index of next slot for push
ForkJoinTask<?>[] array; // the elements (initially unallocated)
final ForkJoinPool pool; // the containing pool (may be null)
final ForkJoinWorkerThread owner; // owning thread or null if shared
volatile Thread parker; // == owner during call to park; else null
volatile ForkJoinTask<?> currentJoin; // task being joined in awaitJoin
ForkJoinTask<?> currentSteal; // current non-local task being executed
volatile Object pad10, pad11, pad12, pad13, pad14, pad15, pad16, pad17;
volatile Object pad18, pad19, pad1a, pad1b, pad1c, pad1d;
WorkQueue(ForkJoinPool pool, ForkJoinWorkerThread owner, int mode,
int seed) {
this.pool = pool;
this.owner = owner;
this.mode = mode;
this.seed = seed;
// Place indices in the center of array (that is not yet allocated)
base = top = INITIAL_QUEUE_CAPACITY >>> 1;
}
/** /**
* If there is a security manager, makes sure caller has * Returns the approximate number of tasks in the queue.
* permission to modify threads. */
*/ final int queueSize() {
private static void checkPermission() { int n = base - top; // non-owner callers must read base first
SecurityManager security = System.getSecurityManager(); return (n >= 0) ? 0 : -n; // ignore transient negative
if (security != null) }
security.checkPermission(modifyThreadPermission);
/**
* Provides a more accurate estimate of whether this queue has
* any tasks than does queueSize, by checking whether a
* near-empty queue has at least one unclaimed task.
*/
final boolean isEmpty() {
ForkJoinTask<?>[] a; int m, s;
int n = base - (s = top);
return (n >= 0 ||
(n == -1 &&
((a = array) == null ||
(m = a.length - 1) < 0 ||
U.getObject
(a, (long)((m & (s - 1)) << ASHIFT) + ABASE) == null)));
}
/**
* Pushes a task. Call only by owner in unshared queues. (The
* shared-queue version is embedded in method externalPush.)
*
* @param task the task. Caller must ensure non-null.
* @throw RejectedExecutionException if array cannot be resized
*/
final void push(ForkJoinTask<?> task) {
ForkJoinTask<?>[] a; ForkJoinPool p;
int s = top, m, n;
if ((a = array) != null) { // ignore if queue removed
int j = (((m = a.length - 1) & s) << ASHIFT) + ABASE;
U.putOrderedObject(a, j, task);
if ((n = (top = s + 1) - base) <= 2) {
if ((p = pool) != null)
p.signalWork(this);
}
else if (n >= m)
growArray();
}
}
/**
* Initializes or doubles the capacity of array. Call either
* by owner or with lock held -- it is OK for base, but not
* top, to move while resizings are in progress.
*/
final ForkJoinTask<?>[] growArray() {
ForkJoinTask<?>[] oldA = array;
int size = oldA != null ? oldA.length << 1 : INITIAL_QUEUE_CAPACITY;
if (size > MAXIMUM_QUEUE_CAPACITY)
throw new RejectedExecutionException("Queue capacity exceeded");
int oldMask, t, b;
ForkJoinTask<?>[] a = array = new ForkJoinTask<?>[size];
if (oldA != null && (oldMask = oldA.length - 1) >= 0 &&
(t = top) - (b = base) > 0) {
int mask = size - 1;
do {
ForkJoinTask<?> x;
int oldj = ((b & oldMask) << ASHIFT) + ABASE;
int j = ((b & mask) << ASHIFT) + ABASE;
x = (ForkJoinTask<?>)U.getObjectVolatile(oldA, oldj);
if (x != null &&
U.compareAndSwapObject(oldA, oldj, x, null))
U.putObjectVolatile(a, j, x);
} while (++b != t);
}
return a;
}
/**
* Takes next task, if one exists, in LIFO order. Call only
* by owner in unshared queues.
*/
final ForkJoinTask<?> pop() {
ForkJoinTask<?>[] a; ForkJoinTask<?> t; int m;
if ((a = array) != null && (m = a.length - 1) >= 0) {
for (int s; (s = top - 1) - base >= 0;) {
long j = ((m & s) << ASHIFT) + ABASE;
if ((t = (ForkJoinTask<?>)U.getObject(a, j)) == null)
break;
if (U.compareAndSwapObject(a, j, t, null)) {
top = s;
return t;
}
}
}
return null;
}
/**
* Takes a task in FIFO order if b is base of queue and a task
* can be claimed without contention. Specialized versions
* appear in ForkJoinPool methods scan and tryHelpStealer.
*/
final ForkJoinTask<?> pollAt(int b) {
ForkJoinTask<?> t; ForkJoinTask<?>[] a;
if ((a = array) != null) {
int j = (((a.length - 1) & b) << ASHIFT) + ABASE;
if ((t = (ForkJoinTask<?>)U.getObjectVolatile(a, j)) != null &&
base == b &&
U.compareAndSwapObject(a, j, t, null)) {
base = b + 1;
return t;
}
}
return null;
}
/**
* Takes next task, if one exists, in FIFO order.
*/
final ForkJoinTask<?> poll() {
ForkJoinTask<?>[] a; int b; ForkJoinTask<?> t;
while ((b = base) - top < 0 && (a = array) != null) {
int j = (((a.length - 1) & b) << ASHIFT) + ABASE;
t = (ForkJoinTask<?>)U.getObjectVolatile(a, j);
if (t != null) {
if (base == b &&
U.compareAndSwapObject(a, j, t, null)) {
base = b + 1;
return t;
}
}
else if (base == b) {
if (b + 1 == top)
break;
Thread.yield(); // wait for lagging update (very rare)
}
}
return null;
}
/**
* Takes next task, if one exists, in order specified by mode.
*/
final ForkJoinTask<?> nextLocalTask() {
return mode == 0 ? pop() : poll();
}
/**
* Returns next task, if one exists, in order specified by mode.
*/
final ForkJoinTask<?> peek() {
ForkJoinTask<?>[] a = array; int m;
if (a == null || (m = a.length - 1) < 0)
return null;
int i = mode == 0 ? top - 1 : base;
int j = ((i & m) << ASHIFT) + ABASE;
return (ForkJoinTask<?>)U.getObjectVolatile(a, j);
}
/**
* Pops the given task only if it is at the current top.
* (A shared version is available only via FJP.tryExternalUnpush)
*/
final boolean tryUnpush(ForkJoinTask<?> t) {
ForkJoinTask<?>[] a; int s;
if ((a = array) != null && (s = top) != base &&
U.compareAndSwapObject
(a, (((a.length - 1) & --s) << ASHIFT) + ABASE, t, null)) {
top = s;
return true;
}
return false;
}
/**
* Removes and cancels all known tasks, ignoring any exceptions.
*/
final void cancelAll() {
ForkJoinTask.cancelIgnoringExceptions(currentJoin);
ForkJoinTask.cancelIgnoringExceptions(currentSteal);
for (ForkJoinTask<?> t; (t = poll()) != null; )
ForkJoinTask.cancelIgnoringExceptions(t);
}
/**
* Computes next value for random probes. 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 its usages in ForkJoinPool to
* avoid writes inside busy scan loops.
*/
final int nextSeed() {
int r = seed;
r ^= r << 13;
r ^= r >>> 17;
return seed = r ^= r << 5;
}
// Specialized execution methods
/**
* Pops and runs tasks until empty.
*/
private void popAndExecAll() {
// A bit faster than repeated pop calls
ForkJoinTask<?>[] a; int m, s; long j; ForkJoinTask<?> t;
while ((a = array) != null && (m = a.length - 1) >= 0 &&
(s = top - 1) - base >= 0 &&
(t = ((ForkJoinTask<?>)
U.getObject(a, j = ((m & s) << ASHIFT) + ABASE)))
!= null) {
if (U.compareAndSwapObject(a, j, t, null)) {
top = s;
t.doExec();
}
}
}
/**
* Polls and runs tasks until empty.
*/
private void pollAndExecAll() {
for (ForkJoinTask<?> t; (t = poll()) != null;)
t.doExec();
}
/**
* If present, removes from queue and executes the given task,
* or any other cancelled task. Returns (true) on any CAS
* or consistency check failure so caller can retry.
*
* @return false if no progress can be made, else true;
*/
final boolean tryRemoveAndExec(ForkJoinTask<?> task) {
boolean stat = true, removed = false, empty = true;
ForkJoinTask<?>[] a; int m, s, b, n;
if ((a = array) != null && (m = a.length - 1) >= 0 &&
(n = (s = top) - (b = base)) > 0) {
for (ForkJoinTask<?> t;;) { // traverse from s to b
int j = ((--s & m) << ASHIFT) + ABASE;
t = (ForkJoinTask<?>)U.getObjectVolatile(a, j);
if (t == null) // inconsistent length
break;
else if (t == task) {
if (s + 1 == top) { // pop
if (!U.compareAndSwapObject(a, j, task, null))
break;
top = s;
removed = true;
}
else if (base == b) // replace with proxy
removed = U.compareAndSwapObject(a, j, task,
new EmptyTask());
break;
}
else if (t.status >= 0)
empty = false;
else if (s + 1 == top) { // pop and throw away
if (U.compareAndSwapObject(a, j, t, null))
top = s;
break;
}
if (--n == 0) {
if (!empty && base == b)
stat = false;
break;
}
}
}
if (removed)
task.doExec();
return stat;
}
/**
* Polls for and executes the given task or any other task in
* its CountedCompleter computation
*/
final boolean pollAndExecCC(ForkJoinTask<?> root) {
ForkJoinTask<?>[] a; int b; Object o;
outer: while ((b = base) - top < 0 && (a = array) != null) {
long j = (((a.length - 1) & b) << ASHIFT) + ABASE;
if ((o = U.getObject(a, j)) == null ||
!(o instanceof CountedCompleter))
break;
for (CountedCompleter<?> t = (CountedCompleter<?>)o, r = t;;) {
if (r == root) {
if (base == b &&
U.compareAndSwapObject(a, j, t, null)) {
base = b + 1;
t.doExec();
return true;
}
else
break; // restart
}
if ((r = r.completer) == null)
break outer; // not part of root computation
}
}
return false;
}
/**
* Executes a top-level task and any local tasks remaining
* after execution.
*/
final void runTask(ForkJoinTask<?> t) {
if (t != null) {
(currentSteal = t).doExec();
currentSteal = null;
++nsteals;
if (base - top < 0) { // process remaining local tasks
if (mode == 0)
popAndExecAll();
else
pollAndExecAll();
}
}
}
/**
* Executes a non-top-level (stolen) task.
*/
final void runSubtask(ForkJoinTask<?> t) {
if (t != null) {
ForkJoinTask<?> ps = currentSteal;
(currentSteal = t).doExec();
currentSteal = ps;
}
}
/**
* Returns true if owned and not known to be blocked.
*/
final boolean isApparentlyUnblocked() {
Thread wt; Thread.State s;
return (eventCount >= 0 &&
(wt = owner) != null &&
(s = wt.getState()) != Thread.State.BLOCKED &&
s != Thread.State.WAITING &&
s != Thread.State.TIMED_WAITING);
}
// Unsafe mechanics
private static final sun.misc.Unsafe U;
private static final long QLOCK;
private static final int ABASE;
private static final int ASHIFT;
static {
int s;
try {
U = sun.misc.Unsafe.getUnsafe();
Class<?> k = WorkQueue.class;
Class<?> ak = ForkJoinTask[].class;
QLOCK = U.objectFieldOffset
(k.getDeclaredField("qlock"));
ABASE = U.arrayBaseOffset(ak);
s = U.arrayIndexScale(ak);
} 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);
}
} }
// static fields (initialized in static initializer below)
/** /**
* Generator for assigning sequence numbers as pool names. * Creates a new ForkJoinWorkerThread. This factory is used unless
* overridden in ForkJoinPool constructors.
*/ */
private static final AtomicInteger poolNumberGenerator; public static final ForkJoinWorkerThreadFactory
defaultForkJoinWorkerThreadFactory;
/** /**
* Generator for initial random seeds for worker victim * Per-thread submission bookkeeping. Shared across all pools
* selection. This is used only to create initial seeds. Random * to reduce ThreadLocal pollution and because random motion
* steals use a cheaper xorshift generator per steal attempt. We * to avoid contention in one pool is likely to hold for others.
* don't expect much contention on seedGenerator, so just use a * Lazily initialized on first submission (but null-checked
* plain Random. * in other contexts to avoid unnecessary initialization).
*/ */
static final Random workerSeedGenerator; static final ThreadLocal<Submitter> submitters;
/** /**
* Array holding all worker threads in the pool. Initialized upon * Permission required for callers of methods that may start or
* construction. Array size must be a power of two. Updates and * kill threads.
* replacements are protected by scanGuard, but the array is
* always kept in a consistent enough state to be randomly
* accessed without locking by workers performing work-stealing,
* 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.
*/ */
ForkJoinWorkerThread[] workers; private static final RuntimePermission modifyThreadPermission;
/** /**
* Initial size for submission queue array. Must be a power of * Common (static) pool. Non-null for public use unless a static
* two. In many applications, these always stay small so we use a * construction exception, but internal usages null-check on use
* small initial cap. * to paranoically avoid potential initialization circularities
* as well as to simplify generated code.
*/ */
private static final int INITIAL_QUEUE_CAPACITY = 8; static final ForkJoinPool commonPool;
/** /**
* Maximum size for submission queue array. Must be a power of two * Common pool parallelism. Must equal commonPool.parallelism.
* less than or equal to 1 << (31 - width of array entry) to
* ensure lack of index wraparound, but is capped at a lower
* value to help users trap runaway computations.
*/ */
private static final int MAXIMUM_QUEUE_CAPACITY = 1 << 24; // 16M static final int commonPoolParallelism;
/** /**
* Array serving as submission queue. Initialized upon construction. * Sequence number for creating workerNamePrefix.
*/ */
private ForkJoinTask<?>[] submissionQueue; private static int poolNumberSequence;
/** /**
* Lock protecting submissions array for addSubmission * Return the next sequence number. We don't expect this to
* ever contend so use simple builtin sync.
*/ */
private final ReentrantLock submissionLock; private static final synchronized int nextPoolId() {
return ++poolNumberSequence;
}
// static constants
/** /**
* Condition for awaitTermination, using submissionLock for * Initial timeout value (in nanoseconds) for the thread
* convenience. * triggering quiescence to park waiting for new work. On timeout,
* the thread will instead try to shrink the number of
* workers. The value should be large enough to avoid overly
* aggressive shrinkage during most transient stalls (long GCs
* etc).
*/ */
private final Condition termination; private static final long IDLE_TIMEOUT = 2000L * 1000L * 1000L; // 2sec
/** /**
* Creation factory for worker threads. * Timeout value when there are more threads than parallelism level
*/ */
private final ForkJoinWorkerThreadFactory factory; private static final long FAST_IDLE_TIMEOUT = 200L * 1000L * 1000L;
/** /**
* The uncaught exception handler used when any worker abruptly * Tolerance for idle timeouts, to cope with timer undershoots
* terminates.
*/ */
final Thread.UncaughtExceptionHandler ueh; private static final long TIMEOUT_SLOP = 2000000L;
/** /**
* Prefix for assigning names to worker threads * The maximum stolen->joining link depth allowed in method
* tryHelpStealer. Must be a power of two. Depths for legitimate
* chains are unbounded, but we use a fixed constant to avoid
* (otherwise unchecked) cycles and to bound staleness of
* traversal parameters at the expense of sometimes blocking when
* we could be helping.
*/ */
private final String workerNamePrefix; private static final int MAX_HELP = 64;
/** /**
* Sum of per-thread steal counts, updated only when threads are * Increment for seed generators. See class ThreadLocal for
* idle or terminating. * explanation.
*/ */
private volatile long stealCount; private static final int SEED_INCREMENT = 0x61c88647;
/** /**
* Main pool control -- a long packed with: * Bits and masks for control variables
*
* Field ctl is a long packed with:
* AC: Number of active running workers minus target parallelism (16 bits) * AC: Number of active running workers minus target parallelism (16 bits)
* TC: Number of total workers minus target parallelism (16bits) * TC: Number of total workers minus target parallelism (16 bits)
* ST: true if pool is terminating (1 bit) * ST: true if pool is terminating (1 bit)
* EC: the wait count of top waiting thread (15 bits) * EC: the wait count of top waiting thread (15 bits)
* ID: ~poolIndex of top of Treiber stack of waiting threads (16 bits) * ID: poolIndex of top of Treiber stack of waiters (16 bits)
* *
* When convenient, we can extract the upper 32 bits of counts and * When convenient, we can extract the upper 32 bits of counts and
* the lower 32 bits of queue state, u = (int)(ctl >>> 32) and e = * the lower 32 bits of queue state, u = (int)(ctl >>> 32) and e =
...@@ -508,13 +1178,26 @@ public class ForkJoinPool extends AbstractExecutorService { ...@@ -508,13 +1178,26 @@ public class ForkJoinPool extends AbstractExecutorService {
* parallelism and the positionings of fields makes it possible to * parallelism and the positionings of fields makes it possible to
* perform the most common checks via sign tests of fields: When * perform the most common checks via sign tests of fields: When
* ac is negative, there are not enough active workers, when tc is * ac is negative, there are not enough active workers, when tc is
* negative, there are not enough total workers, when id is * negative, there are not enough total workers, and when e is
* negative, there is at least one waiting worker, and when e is
* negative, the pool is terminating. To deal with these possibly * negative, the pool is terminating. To deal with these possibly
* negative fields, we use casts in and out of "short" and/or * negative fields, we use casts in and out of "short" and/or
* signed shifts to maintain signedness. * signed shifts to maintain signedness.
*
* When a thread is queued (inactivated), its eventCount field is
* set negative, which is the only way to tell if a worker is
* prevented from executing tasks, even though it must continue to
* scan for them to avoid queuing races. Note however that
* eventCount updates lag releases so usage requires care.
*
* Field plock is an int packed with:
* SHUTDOWN: true if shutdown is enabled (1 bit)
* SEQ: a sequence lock, with PL_LOCK bit set if locked (30 bits)
* SIGNAL: set when threads may be waiting on the lock (1 bit)
*
* The sequence number enables simple consistency checks:
* Staleness of read-only operations on the workQueues array can
* be checked by comparing plock before vs after the reads.
*/ */
volatile long ctl;
// bit positions/shifts for fields // bit positions/shifts for fields
private static final int AC_SHIFT = 48; private static final int AC_SHIFT = 48;
...@@ -523,8 +1206,10 @@ public class ForkJoinPool extends AbstractExecutorService { ...@@ -523,8 +1206,10 @@ public class ForkJoinPool extends AbstractExecutorService {
private static final int EC_SHIFT = 16; private static final int EC_SHIFT = 16;
// bounds // bounds
private static final int MAX_ID = 0x7fff; // max poolIndex private static final int SMASK = 0xffff; // short bits
private static final int SMASK = 0xffff; // mask short bits private static final int MAX_CAP = 0x7fff; // max #workers - 1
private static final int EVENMASK = 0xfffe; // even short bits
private static final int SQMASK = 0x007e; // max 64 (even) slots
private static final int SHORT_SIGN = 1 << 15; private static final int SHORT_SIGN = 1 << 15;
private static final int INT_SIGN = 1 << 31; private static final int INT_SIGN = 1 << 31;
...@@ -546,853 +1231,1281 @@ public class ForkJoinPool extends AbstractExecutorService { ...@@ -546,853 +1231,1281 @@ public class ForkJoinPool extends AbstractExecutorService {
private static final int UTC_UNIT = 1 << UTC_SHIFT; private static final int UTC_UNIT = 1 << UTC_SHIFT;
// masks and units for dealing with e = (int)ctl // masks and units for dealing with e = (int)ctl
private static final int E_MASK = 0x7fffffff; // no STOP_BIT private static final int E_MASK = 0x7fffffff; // no STOP_BIT
private static final int EC_UNIT = 1 << EC_SHIFT; private static final int E_SEQ = 1 << EC_SHIFT;
/** // plock bits
* The target parallelism level. private static final int SHUTDOWN = 1 << 31;
*/ private static final int PL_LOCK = 2;
final int parallelism; private static final int PL_SIGNAL = 1;
private static final int PL_SPINS = 1 << 8;
/**
* Index (mod submission queue length) of next element to take
* from submission queue. Usage is identical to that for
* per-worker queues -- see ForkJoinWorkerThread internal
* documentation.
*/
volatile int queueBase;
/** // access mode for WorkQueue
* Index (mod submission queue length) of next element to add static final int LIFO_QUEUE = 0;
* in submission queue. Usage is identical to that for static final int FIFO_QUEUE = 1;
* per-worker queues -- see ForkJoinWorkerThread internal static final int SHARED_QUEUE = -1;
* documentation.
*/
int queueTop;
/**
* True when shutdown() has been called.
*/
volatile boolean shutdown;
/** // bounds for #steps in scan loop -- must be power 2 minus 1
* True if use local fifo, not default lifo, for local polling private static final int MIN_SCAN = 0x1ff; // cover estimation slop
* Read by, and replicated by ForkJoinWorkerThreads private static final int MAX_SCAN = 0x1ffff; // 4 * max workers
*/
final boolean locallyFifo;
/** // Instance fields
* The number of threads in ForkJoinWorkerThreads.helpQuiescePool.
* When non-zero, suppresses automatic shutdown when active
* counts become zero.
*/
volatile int quiescerCount;
/** /*
* The number of threads blocked in join. * Field layout of this class tends to matter more than one would
* like. Runtime layout order is only loosely related to
* declaration order and may differ across JVMs, but the following
* empirically works OK on current JVMs.
*/ */
volatile int blockedCount;
/** // Heuristic padding to ameliorate unfortunate memory placements
* Counter for worker Thread names (unrelated to their poolIndex) volatile long pad00, pad01, pad02, pad03, pad04, pad05, pad06;
*/
private volatile int nextWorkerNumber;
/** volatile long stealCount; // collects worker counts
* The index for the next created worker. Accessed under scanGuard. volatile long ctl; // main pool control
*/ volatile int plock; // shutdown status and seqLock
private int nextWorkerIndex; volatile int indexSeed; // worker/submitter index seed
final int config; // mode and parallelism level
WorkQueue[] workQueues; // main registry
final ForkJoinWorkerThreadFactory factory;
final Thread.UncaughtExceptionHandler ueh; // per-worker UEH
final String workerNamePrefix; // to create worker name string
/** volatile Object pad10, pad11, pad12, pad13, pad14, pad15, pad16, pad17;
* SeqLock and index masking for updates to workers array. Locked volatile Object pad18, pad19, pad1a, pad1b;
* 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.
*/
volatile int scanGuard;
private static final int SG_UNIT = 1 << 16; /*
* Acquires the plock lock to protect worker array and related
* updates. This method is called only if an initial CAS on plock
* fails. This acts as a spinLock for normal cases, but falls back
* to builtin monitor to block when (rarely) needed. This would be
* a terrible idea for a highly contended lock, but works fine as
* a more conservative alternative to a pure spinlock.
*/
private int acquirePlock() {
int spins = PL_SPINS, r = 0, ps, nps;
for (;;) {
if (((ps = plock) & PL_LOCK) == 0 &&
U.compareAndSwapInt(this, PLOCK, ps, nps = ps + PL_LOCK))
return nps;
else if (r == 0) { // randomize spins if possible
Thread t = Thread.currentThread(); WorkQueue w; Submitter z;
if ((t instanceof ForkJoinWorkerThread) &&
(w = ((ForkJoinWorkerThread)t).workQueue) != null)
r = w.seed;
else if ((z = submitters.get()) != null)
r = z.seed;
else
r = 1;
}
else if (spins >= 0) {
r ^= r << 1; r ^= r >>> 3; r ^= r << 10; // xorshift
if (r >= 0)
--spins;
}
else if (U.compareAndSwapInt(this, PLOCK, ps, ps | PL_SIGNAL)) {
synchronized (this) {
if ((plock & PL_SIGNAL) != 0) {
try {
wait();
} catch (InterruptedException ie) {
try {
Thread.currentThread().interrupt();
} catch (SecurityException ignore) {
}
}
}
else
notifyAll();
}
}
}
}
/** /**
* The wakeup interval (in nanoseconds) for a worker waiting for a * Unlocks and signals any thread waiting for plock. Called only
* task when the pool is quiescent to instead try to shrink the * when CAS of seq value for unlock fails.
* 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.
*/ */
private static final long SHRINK_RATE = private void releasePlock(int ps) {
4L * 1000L * 1000L * 1000L; // 4 seconds plock = ps;
synchronized (this) { notifyAll(); }
}
/** /**
* Top-level loop for worker threads: On each step: if the * Performs secondary initialization, called when plock is zero.
* previous step swept through all queues and found no tasks, or * Creates workQueue array and sets plock to a valid value. The
* there are excess threads, then possibly blocks. Otherwise, * lock body must be exception-free (so no try/finally) so we
* scans for and, if found, executes a task. Returns when pool * optimistically allocate new array outside the lock and throw
* and/or worker terminate. * away if (very rarely) not needed. (A similar tactic is used in
* fullExternalPush.) Because the plock seq value can eventually
* wrap around zero, this method harmlessly fails to reinitialize
* if workQueues exists, while still advancing plock.
* *
* @param w the worker * Additionally tries to create the first worker.
*/ */
final void work(ForkJoinWorkerThread w) { private void initWorkers() {
boolean swept = false; // true on empty scans WorkQueue[] ws, nws; int ps;
long c; int p = config & SMASK; // find power of two table size
while (!w.terminate && (int)(c = ctl) >= 0) { int n = (p > 1) ? p - 1 : 1; // ensure at least 2 slots
int a; // active count n |= n >>> 1; n |= n >>> 2; n |= n >>> 4; n |= n >>> 8; n |= n >>> 16;
if (!swept && (a = (int)(c >> AC_SHIFT)) <= 0) n = (n + 1) << 1;
swept = scan(w, a); if ((ws = workQueues) == null || ws.length == 0)
else if (tryAwaitWork(w, c)) nws = new WorkQueue[n];
swept = false; else
nws = null;
if (((ps = plock) & PL_LOCK) != 0 ||
!U.compareAndSwapInt(this, PLOCK, ps, ps += PL_LOCK))
ps = acquirePlock();
if (((ws = workQueues) == null || ws.length == 0) && nws != null)
workQueues = nws;
int nps = (ps & SHUTDOWN) | ((ps + PL_LOCK) & ~SHUTDOWN);
if (!U.compareAndSwapInt(this, PLOCK, ps, nps))
releasePlock(nps);
tryAddWorker();
}
/**
* Tries to create and start one worker if fewer than target
* parallelism level exist. Adjusts counts etc on failure.
*/
private void tryAddWorker() {
long c; int u;
while ((u = (int)((c = ctl) >>> 32)) < 0 &&
(u & SHORT_SIGN) != 0 && (int)c == 0) {
long nc = (long)(((u + UTC_UNIT) & UTC_MASK) |
((u + UAC_UNIT) & UAC_MASK)) << 32;
if (U.compareAndSwapLong(this, CTL, c, nc)) {
ForkJoinWorkerThreadFactory fac;
Throwable ex = null;
ForkJoinWorkerThread wt = null;
try {
if ((fac = factory) != null &&
(wt = fac.newThread(this)) != null) {
wt.start();
break;
}
} catch (Throwable e) {
ex = e;
}
deregisterWorker(wt, ex);
break;
}
} }
} }
// Signalling // Registering and deregistering workers
/** /**
* Wakes up or creates a worker. * Callback from ForkJoinWorkerThread to establish and record its
* WorkQueue. To avoid scanning bias due to packing entries in
* front of the workQueues array, we treat the array as a simple
* power-of-two hash table using per-thread seed as hash,
* expanding as needed.
*
* @param wt the worker thread
* @return the worker's queue
*/ */
final void signalWork() { final WorkQueue registerWorker(ForkJoinWorkerThread wt) {
/* Thread.UncaughtExceptionHandler handler; WorkQueue[] ws; int s, ps;
* The while condition is true if: (there is are too few total wt.setDaemon(true);
* workers OR there is at least one waiter) AND (there are too if ((handler = ueh) != null)
* few active workers OR the pool is terminating). The value wt.setUncaughtExceptionHandler(handler);
* of e distinguishes the remaining cases: zero (no waiters) do {} while (!U.compareAndSwapInt(this, INDEXSEED, s = indexSeed,
* for create, negative if terminating (in which case do s += SEED_INCREMENT) ||
* nothing), else release a waiter. The secondary checks for s == 0); // skip 0
* release (non-null array etc) can fail if the pool begins WorkQueue w = new WorkQueue(this, wt, config >>> 16, s);
* terminating after the test, and don't impose any added cost if (((ps = plock) & PL_LOCK) != 0 ||
* because JVMs must perform null and bounds checks anyway. !U.compareAndSwapInt(this, PLOCK, ps, ps += PL_LOCK))
*/ ps = acquirePlock();
long c; int e, u; int nps = (ps & SHUTDOWN) | ((ps + PL_LOCK) & ~SHUTDOWN);
while ((((e = (int)(c = ctl)) | (u = (int)(c >>> 32))) & try {
(INT_SIGN|SHORT_SIGN)) == (INT_SIGN|SHORT_SIGN) && e >= 0) { if ((ws = workQueues) != null) { // skip if shutting down
if (e > 0) { // release a waiting worker int n = ws.length, m = n - 1;
int i; ForkJoinWorkerThread w; ForkJoinWorkerThread[] ws; int r = (s << 1) | 1; // use odd-numbered indices
if ((ws = workers) == null || if (ws[r &= m] != null) { // collision
(i = ~e & SMASK) >= ws.length || int probes = 0; // step by approx half size
(w = ws[i]) == null) int step = (n <= 4) ? 2 : ((n >>> 1) & EVENMASK) + 2;
break; while (ws[r = (r + step) & m] != null) {
long nc = (((long)(w.nextWait & E_MASK)) | if (++probes >= n) {
((long)(u + UAC_UNIT) << 32)); workQueues = ws = Arrays.copyOf(ws, n <<= 1);
if (w.eventCount == e && m = n - 1;
UNSAFE.compareAndSwapLong(this, ctlOffset, c, nc)) { probes = 0;
w.eventCount = (e + EC_UNIT) & E_MASK; }
if (w.parked) }
UNSAFE.unpark(w);
break;
} }
w.eventCount = w.poolIndex = r; // volatile write orders
ws[r] = w;
} }
else if (UNSAFE.compareAndSwapLong } finally {
(this, ctlOffset, c, if (!U.compareAndSwapInt(this, PLOCK, ps, nps))
(long)(((u + UTC_UNIT) & UTC_MASK) | releasePlock(nps);
((u + UAC_UNIT) & UAC_MASK)) << 32)) { }
addWorker(); wt.setName(workerNamePrefix.concat(Integer.toString(w.poolIndex)));
break; return w;
}
/**
* Final callback from terminating worker, as well as upon failure
* to construct or start a worker. Removes record of worker from
* array, and adjusts counts. If pool is shutting down, tries to
* complete termination.
*
* @param wt the worker thread or null if construction failed
* @param ex the exception causing failure, or null if none
*/
final void deregisterWorker(ForkJoinWorkerThread wt, Throwable ex) {
WorkQueue w = null;
if (wt != null && (w = wt.workQueue) != null) {
int ps;
w.qlock = -1; // ensure set
long ns = w.nsteals, sc; // collect steal count
do {} while (!U.compareAndSwapLong(this, STEALCOUNT,
sc = stealCount, sc + ns));
if (((ps = plock) & PL_LOCK) != 0 ||
!U.compareAndSwapInt(this, PLOCK, ps, ps += PL_LOCK))
ps = acquirePlock();
int nps = (ps & SHUTDOWN) | ((ps + PL_LOCK) & ~SHUTDOWN);
try {
int idx = w.poolIndex;
WorkQueue[] ws = workQueues;
if (ws != null && idx >= 0 && idx < ws.length && ws[idx] == w)
ws[idx] = null;
} finally {
if (!U.compareAndSwapInt(this, PLOCK, ps, nps))
releasePlock(nps);
} }
} }
}
/** long c; // adjust ctl counts
* Variant of signalWork to help release waiters on rescans. do {} while (!U.compareAndSwapLong
* Tries once to release a waiter if active count < 0. (this, CTL, c = ctl, (((c - AC_UNIT) & AC_MASK) |
* ((c - TC_UNIT) & TC_MASK) |
* @return false if failed due to contention, else true (c & ~(AC_MASK|TC_MASK)))));
*/
private boolean tryReleaseWaiter() { if (!tryTerminate(false, false) && w != null && w.array != null) {
long c; int e, i; ForkJoinWorkerThread w; ForkJoinWorkerThread[] ws; w.cancelAll(); // cancel remaining tasks
if ((e = (int)(c = ctl)) > 0 && WorkQueue[] ws; WorkQueue v; Thread p; int u, i, e;
(int)(c >> AC_SHIFT) < 0 && while ((u = (int)((c = ctl) >>> 32)) < 0 && (e = (int)c) >= 0) {
(ws = workers) != null && if (e > 0) { // activate or create replacement
(i = ~e & SMASK) < ws.length && if ((ws = workQueues) == null ||
(w = ws[i]) != null) { (i = e & SMASK) >= ws.length ||
long nc = ((long)(w.nextWait & E_MASK) | (v = ws[i]) != null)
((c + AC_UNIT) & (AC_MASK|TC_MASK))); break;
if (w.eventCount != e || long nc = (((long)(v.nextWait & E_MASK)) |
!UNSAFE.compareAndSwapLong(this, ctlOffset, c, nc)) ((long)(u + UAC_UNIT) << 32));
return false; if (v.eventCount != (e | INT_SIGN))
w.eventCount = (e + EC_UNIT) & E_MASK; break;
if (w.parked) if (U.compareAndSwapLong(this, CTL, c, nc)) {
UNSAFE.unpark(w); v.eventCount = (e + E_SEQ) & E_MASK;
if ((p = v.parker) != null)
U.unpark(p);
break;
}
}
else {
if ((short)u < 0)
tryAddWorker();
break;
}
}
} }
return true; if (ex == null) // help clean refs on way out
ForkJoinTask.helpExpungeStaleExceptions();
else // rethrow
ForkJoinTask.rethrow(ex);
} }
// Scanning for tasks // Submissions
/** /**
* Scans for and, if found, executes one task. Scans start at a * Unless shutting down, adds the given task to a submission queue
* random index of workers array, and randomly select the first * at submitter's current queue index (modulo submission
* (2*#workers)-1 probes, and then, if all empty, resort to 2 * range). Only the most common path is directly handled in this
* circular sweeps, which is necessary to check quiescence. and * method. All others are relayed to fullExternalPush.
* taking a submission only if no stealable tasks were found. The *
* steal code inside the loop is a specialized form of * @param task the task. Caller must ensure non-null.
* ForkJoinWorkerThread.deqTask, followed bookkeeping to support */
* helpJoinTask and signal propagation. The code for submission final void externalPush(ForkJoinTask<?> task) {
* queues is almost identical. On each steal, the worker completes WorkQueue[] ws; WorkQueue q; Submitter z; int m; ForkJoinTask<?>[] a;
* not only the task, but also all local tasks that this task may if ((z = submitters.get()) != null && plock > 0 &&
* have generated. On detecting staleness or contention when (ws = workQueues) != null && (m = (ws.length - 1)) >= 0 &&
* trying to take a task, this method returns without finishing (q = ws[m & z.seed & SQMASK]) != null &&
* sweep, which allows global state rechecks before retry. U.compareAndSwapInt(q, QLOCK, 0, 1)) { // lock
* int b = q.base, s = q.top, n, an;
* @param w the worker if ((a = q.array) != null && (an = a.length) > (n = s + 1 - b)) {
* @param a the number of active workers int j = (((an - 1) & s) << ASHIFT) + ABASE;
* @return true if swept all queues without finding a task U.putOrderedObject(a, j, task);
*/ q.top = s + 1; // push on to deque
private boolean scan(ForkJoinWorkerThread w, int a) { q.qlock = 0;
int g = scanGuard; // mask 0 avoids useless scans if only one active if (n <= 2)
int m = (parallelism == 1 - a && blockedCount == 0) ? 0 : g & SMASK; signalWork(q);
ForkJoinWorkerThread[] ws = workers; return;
if (ws == null || ws.length <= m) // staleness check }
return false; q.qlock = 0;
for (int r = w.seed, k = r, j = -(m + m); j <= m + m; ++j) { }
ForkJoinTask<?> t; ForkJoinTask<?>[] q; int b, i; fullExternalPush(task);
ForkJoinWorkerThread v = ws[k & m]; }
if (v != null && (b = v.queueBase) != v.queueTop &&
(q = v.queue) != null && (i = (q.length - 1) & b) >= 0) { /**
long u = (i << ASHIFT) + ABASE; * Full version of externalPush. This method is called, among
if ((t = q[i]) != null && v.queueBase == b && * other times, upon the first submission of the first task to the
UNSAFE.compareAndSwapObject(q, u, t, null)) { * pool, so must perform secondary initialization (via
int d = (v.queueBase = b + 1) - v.queueTop; * initWorkers). It also detects first submission by an external
v.stealHint = w.poolIndex; * thread by looking up its ThreadLocal, and creates a new shared
if (d != 0) * queue if the one at index if empty or contended. The plock lock
signalWork(); // propagate if nonempty * body must be exception-free (so no try/finally) so we
w.execTask(t); * optimistically allocate new queues outside the lock and throw
* them away if (very rarely) not needed.
*/
private void fullExternalPush(ForkJoinTask<?> task) {
int r = 0; // random index seed
for (Submitter z = submitters.get();;) {
WorkQueue[] ws; WorkQueue q; int ps, m, k;
if (z == null) {
if (U.compareAndSwapInt(this, INDEXSEED, r = indexSeed,
r += SEED_INCREMENT) && r != 0)
submitters.set(z = new Submitter(r));
}
else if (r == 0) { // move to a different index
r = z.seed;
r ^= r << 13; // same xorshift as WorkQueues
r ^= r >>> 17;
z.seed = r ^ (r << 5);
}
else if ((ps = plock) < 0)
throw new RejectedExecutionException();
else if (ps == 0 || (ws = workQueues) == null ||
(m = ws.length - 1) < 0)
initWorkers();
else if ((q = ws[k = r & m & SQMASK]) != null) {
if (q.qlock == 0 && U.compareAndSwapInt(q, QLOCK, 0, 1)) {
ForkJoinTask<?>[] a = q.array;
int s = q.top;
boolean submitted = false;
try { // locked version of push
if ((a != null && a.length > s + 1 - q.base) ||
(a = q.growArray()) != null) { // must presize
int j = (((a.length - 1) & s) << ASHIFT) + ABASE;
U.putOrderedObject(a, j, task);
q.top = s + 1;
submitted = true;
}
} finally {
q.qlock = 0; // unlock
}
if (submitted) {
signalWork(q);
return;
}
} }
r ^= r << 13; r ^= r >>> 17; w.seed = r ^ (r << 5); r = 0; // move on failure
return false; // store next seed
} }
else if (j < 0) { // xorshift else if (((ps = plock) & PL_LOCK) == 0) { // create new queue
r ^= r << 13; r ^= r >>> 17; k = r ^= r << 5; q = new WorkQueue(this, null, SHARED_QUEUE, r);
if (((ps = plock) & PL_LOCK) != 0 ||
!U.compareAndSwapInt(this, PLOCK, ps, ps += PL_LOCK))
ps = acquirePlock();
if ((ws = workQueues) != null && k < ws.length && ws[k] == null)
ws[k] = q;
int nps = (ps & SHUTDOWN) | ((ps + PL_LOCK) & ~SHUTDOWN);
if (!U.compareAndSwapInt(this, PLOCK, ps, nps))
releasePlock(nps);
} }
else else
++k; r = 0; // try elsewhere while lock held
} }
if (scanGuard != g) // staleness check }
return false;
else { // try to take submission // Maintaining ctl counts
ForkJoinTask<?> t; ForkJoinTask<?>[] q; int b, i;
if ((b = queueBase) != queueTop && /**
(q = submissionQueue) != null && * Increments active count; mainly called upon return from blocking.
(i = (q.length - 1) & b) >= 0) { */
long u = (i << ASHIFT) + ABASE; final void incrementActiveCount() {
if ((t = q[i]) != null && queueBase == b && long c;
UNSAFE.compareAndSwapObject(q, u, t, null)) { do {} while (!U.compareAndSwapLong(this, CTL, c = ctl, c + AC_UNIT));
queueBase = b + 1; }
w.execTask(t);
/**
* Tries to create or activate a worker if too few are active.
*
* @param q the (non-null) queue holding tasks to be signalled
*/
final void signalWork(WorkQueue q) {
int hint = q.poolIndex;
long c; int e, u, i, n; WorkQueue[] ws; WorkQueue w; Thread p;
while ((u = (int)((c = ctl) >>> 32)) < 0) {
if ((e = (int)c) > 0) {
if ((ws = workQueues) != null && ws.length > (i = e & SMASK) &&
(w = ws[i]) != null && w.eventCount == (e | INT_SIGN)) {
long nc = (((long)(w.nextWait & E_MASK)) |
((long)(u + UAC_UNIT) << 32));
if (U.compareAndSwapLong(this, CTL, c, nc)) {
w.hint = hint;
w.eventCount = (e + E_SEQ) & E_MASK;
if ((p = w.parker) != null)
U.unpark(p);
break;
}
if (q.top - q.base <= 0)
break;
} }
return false; else
break;
}
else {
if ((short)u < 0)
tryAddWorker();
break;
} }
return true; // all queues empty
} }
} }
// Scanning for tasks
/**
* Top-level runloop for workers, called by ForkJoinWorkerThread.run.
*/
final void runWorker(WorkQueue w) {
w.growArray(); // allocate queue
do { w.runTask(scan(w)); } while (w.qlock >= 0);
}
/** /**
* Tries to enqueue worker w in wait queue and await change in * Scans for and, if found, returns one task, else possibly
* worker's eventCount. If the pool is quiescent and there is * inactivates the worker. This method operates on single reads of
* more than one worker, possibly terminates worker upon exit. * volatile state and is designed to be re-invoked continuously,
* Otherwise, before blocking, rescans queues to avoid missed * in part because it returns upon detecting inconsistencies,
* signals. Upon finding work, releases at least one worker * contention, or state changes that indicate possible success on
* (which may be the current worker). Rescans restart upon * re-invocation.
* detected staleness or failure to release due to *
* contention. Note the unusual conventions about Thread.interrupt * The scan searches for tasks across queues (starting at a random
* here and elsewhere: Because interrupts are used solely to alert * index, and relying on registerWorker to irregularly scatter
* threads to check termination, which is checked here anyway, we * them within array to avoid bias), checking each at least twice.
* clear status (using Thread.interrupted) before any call to * The scan terminates upon either finding a non-empty queue, or
* park, so that park does not immediately return due to status * completing the sweep. If the worker is not inactivated, it
* being set via some other unrelated call to interrupt in user * takes and returns a task from this queue. Otherwise, if not
* code. * activated, it signals workers (that may include itself) and
* returns so caller can retry. Also returns for true if the
* worker array may have changed during an empty scan. On failure
* to find a task, we take one of the following actions, after
* which the caller will retry calling this method unless
* terminated.
*
* * If pool is terminating, terminate the worker.
*
* * If not already enqueued, try to inactivate and enqueue the
* worker on wait queue. Or, if inactivating has caused the pool
* to be quiescent, relay to idleAwaitWork to possibly shrink
* pool.
* *
* @param w the calling worker * * If already enqueued and none of the above apply, possibly
* @param c the ctl value on entry * park awaiting signal, else lingering to help scan and signal.
* @return true if waited or another thread was released upon enq *
*/ * * If a non-empty queue discovered or left as a hint,
private boolean tryAwaitWork(ForkJoinWorkerThread w, long c) { * help wake up other workers before return
int v = w.eventCount; *
w.nextWait = (int)c; // w's successor record * @param w the worker (via its WorkQueue)
long nc = (long)(v & E_MASK) | ((c - AC_UNIT) & (AC_MASK|TC_MASK)); * @return a task or null if none found
if (ctl != c || !UNSAFE.compareAndSwapLong(this, ctlOffset, c, nc)) { */
long d = ctl; // return true if lost to a deq, to force scan private final ForkJoinTask<?> scan(WorkQueue w) {
return (int)d != (int)c && ((d - c) & AC_MASK) >= 0L; WorkQueue[] ws; int m;
} int ps = plock; // read plock before ws
for (int sc = w.stealCount; sc != 0;) { // accumulate stealCount if (w != null && (ws = workQueues) != null && (m = ws.length - 1) >= 0) {
long s = stealCount; int ec = w.eventCount; // ec is negative if inactive
if (UNSAFE.compareAndSwapLong(this, stealCountOffset, s, s + sc)) int r = w.seed; r ^= r << 13; r ^= r >>> 17; w.seed = r ^= r << 5;
sc = w.stealCount = 0; w.hint = -1; // update seed and clear hint
else if (w.eventCount != v) int j = ((m + m + 1) | MIN_SCAN) & MAX_SCAN;
return true; // update next time do {
} WorkQueue q; ForkJoinTask<?>[] a; int b;
if ((!shutdown || !tryTerminate(false)) && if ((q = ws[(r + j) & m]) != null && (b = q.base) - q.top < 0 &&
(int)c != 0 && parallelism + (int)(nc >> AC_SHIFT) == 0 && (a = q.array) != null) { // probably nonempty
blockedCount == 0 && quiescerCount == 0) int i = (((a.length - 1) & b) << ASHIFT) + ABASE;
idleAwaitWork(w, nc, c, v); // quiescent ForkJoinTask<?> t = (ForkJoinTask<?>)
for (boolean rescanned = false;;) { U.getObjectVolatile(a, i);
if (w.eventCount != v) if (q.base == b && ec >= 0 && t != null &&
return true; U.compareAndSwapObject(a, i, t, null)) {
if (!rescanned) { if ((q.base = b + 1) - q.top < 0)
int g = scanGuard, m = g & SMASK; signalWork(q);
ForkJoinWorkerThread[] ws = workers; return t; // taken
if (ws != null && m < ws.length) { }
rescanned = true; else if ((ec < 0 || j < m) && (int)(ctl >> AC_SHIFT) <= 0) {
for (int i = 0; i <= m; ++i) { w.hint = (r + j) & m; // help signal below
ForkJoinWorkerThread u = ws[i]; break; // cannot take
if (u != null) {
if (u.queueBase != u.queueTop &&
!tryReleaseWaiter())
rescanned = false; // contended
if (w.eventCount != v)
return true;
}
} }
} }
if (scanGuard != g || // stale } while (--j >= 0);
(queueBase != queueTop && !tryReleaseWaiter()))
rescanned = false; int h, e, ns; long c, sc; WorkQueue q;
if (!rescanned) if ((ns = w.nsteals) != 0) {
Thread.yield(); // reduce contention if (U.compareAndSwapLong(this, STEALCOUNT,
else sc = stealCount, sc + ns))
Thread.interrupted(); // clear before park w.nsteals = 0; // collect steals and rescan
} }
else if (plock != ps) // consistency check
; // skip
else if ((e = (int)(c = ctl)) < 0)
w.qlock = -1; // pool is terminating
else { else {
w.parked = true; // must recheck if ((h = w.hint) < 0) {
if (w.eventCount != v) { if (ec >= 0) { // try to enqueue/inactivate
w.parked = false; long nc = (((long)ec |
return true; ((c - AC_UNIT) & (AC_MASK|TC_MASK))));
w.nextWait = e; // link and mark inactive
w.eventCount = ec | INT_SIGN;
if (ctl != c || !U.compareAndSwapLong(this, CTL, c, nc))
w.eventCount = ec; // unmark on CAS failure
else if ((int)(c >> AC_SHIFT) == 1 - (config & SMASK))
idleAwaitWork(w, nc, c);
}
else if (w.eventCount < 0 && !tryTerminate(false, false) &&
ctl == c) { // block
Thread wt = Thread.currentThread();
Thread.interrupted(); // clear status
U.putObject(wt, PARKBLOCKER, this);
w.parker = wt; // emulate LockSupport.park
if (w.eventCount < 0) // recheck
U.park(false, 0L);
w.parker = null;
U.putObject(wt, PARKBLOCKER, null);
}
}
if ((h >= 0 || (h = w.hint) >= 0) &&
(ws = workQueues) != null && h < ws.length &&
(q = ws[h]) != null) { // signal others before retry
WorkQueue v; Thread p; int u, i, s;
for (int n = (config & SMASK) >>> 1;;) {
int idleCount = (w.eventCount < 0) ? 0 : -1;
if (((s = idleCount - q.base + q.top) <= n &&
(n = s) <= 0) ||
(u = (int)((c = ctl) >>> 32)) >= 0 ||
(e = (int)c) <= 0 || m < (i = e & SMASK) ||
(v = ws[i]) == null)
break;
long nc = (((long)(v.nextWait & E_MASK)) |
((long)(u + UAC_UNIT) << 32));
if (v.eventCount != (e | INT_SIGN) ||
!U.compareAndSwapLong(this, CTL, c, nc))
break;
v.hint = h;
v.eventCount = (e + E_SEQ) & E_MASK;
if ((p = v.parker) != null)
U.unpark(p);
if (--n <= 0)
break;
}
} }
LockSupport.park(this);
rescanned = w.parked = false;
} }
} }
return null;
} }
/** /**
* If inactivating worker w has caused pool to become * If inactivating worker w has caused the pool to become
* quiescent, check for pool termination, and wait for event * quiescent, checks for pool termination, and, so long as this is
* for up to SHRINK_RATE nanosecs (rescans are unnecessary in * not the only worker, waits for event for up to a given
* this case because quiescence reflects consensus about lack * duration. On timeout, if ctl has not changed, terminates the
* of work). On timeout, if ctl has not changed, terminate the * worker, which will in turn wake up another worker to possibly
* worker. Upon its termination (see deregisterWorker), it may * repeat this process.
* wake up another worker to possibly repeat this process.
* *
* @param w the calling worker * @param w the calling worker
* @param currentCtl the ctl value after enqueuing w * @param currentCtl the ctl value triggering possible quiescence
* @param prevCtl the ctl value if w terminated * @param prevCtl the ctl value to restore if thread is terminated
* @param v the eventCount w awaits change */
*/ private void idleAwaitWork(WorkQueue w, long currentCtl, long prevCtl) {
private void idleAwaitWork(ForkJoinWorkerThread w, long currentCtl, if (w != null && w.eventCount < 0 &&
long prevCtl, int v) { !tryTerminate(false, false) && (int)prevCtl != 0) {
if (w.eventCount == v) { int dc = -(short)(currentCtl >>> TC_SHIFT);
if (shutdown) long parkTime = dc < 0 ? FAST_IDLE_TIMEOUT: (dc + 1) * IDLE_TIMEOUT;
tryTerminate(false); long deadline = System.nanoTime() + parkTime - TIMEOUT_SLOP;
ForkJoinTask.helpExpungeStaleExceptions(); // help clean weak refs Thread wt = Thread.currentThread();
while (ctl == currentCtl) { while (ctl == currentCtl) {
long startTime = System.nanoTime(); Thread.interrupted(); // timed variant of version in scan()
w.parked = true; U.putObject(wt, PARKBLOCKER, this);
if (w.eventCount == v) // must recheck w.parker = wt;
LockSupport.parkNanos(this, SHRINK_RATE); if (ctl == currentCtl)
w.parked = false; U.park(false, parkTime);
if (w.eventCount != v) w.parker = null;
U.putObject(wt, PARKBLOCKER, null);
if (ctl != currentCtl)
break; break;
else if (System.nanoTime() - startTime < if (deadline - System.nanoTime() <= 0L &&
SHRINK_RATE - (SHRINK_RATE / 10)) // timing slop U.compareAndSwapLong(this, CTL, currentCtl, prevCtl)) {
Thread.interrupted(); // spurious wakeup w.eventCount = (w.eventCount + E_SEQ) | E_MASK;
else if (UNSAFE.compareAndSwapLong(this, ctlOffset, w.qlock = -1; // shrink
currentCtl, prevCtl)) {
w.terminate = true; // restore previous
w.eventCount = ((int)currentCtl + EC_UNIT) & E_MASK;
break; break;
} }
} }
} }
} }
// Submissions
/** /**
* Enqueues the given task in the submissionQueue. Same idea as * Scans through queues looking for work while joining a task; if
* ForkJoinWorkerThread.pushTask except for use of submissionLock. * any present, signals. May return early if more signalling is
* detectably unneeded.
* *
* @param t the task * @param task return early if done
* @param origin an index to start scan
*/ */
private void addSubmission(ForkJoinTask<?> t) { private void helpSignal(ForkJoinTask<?> task, int origin) {
final ReentrantLock lock = this.submissionLock; WorkQueue[] ws; WorkQueue w; Thread p; long c; int m, u, e, i, s;
lock.lock(); if (task != null && task.status >= 0 &&
try { (u = (int)(ctl >>> 32)) < 0 && (u >> UAC_SHIFT) < 0 &&
ForkJoinTask<?>[] q; int s, m; (ws = workQueues) != null && (m = ws.length - 1) >= 0) {
if ((q = submissionQueue) != null) { // ignore if queue removed outer: for (int k = origin, j = m; j >= 0; --j) {
long u = (((s = queueTop) & (m = q.length-1)) << ASHIFT)+ABASE; WorkQueue q = ws[k++ & m];
UNSAFE.putOrderedObject(q, u, t); for (int n = m;;) { // limit to at most m signals
queueTop = s + 1; if (task.status < 0)
if (s - queueBase == m) break outer;
growSubmissionQueue(); if (q == null ||
} ((s = -q.base + q.top) <= n && (n = s) <= 0))
} finally { break;
lock.unlock(); if ((u = (int)((c = ctl) >>> 32)) >= 0 ||
} (e = (int)c) <= 0 || m < (i = e & SMASK) ||
signalWork(); (w = ws[i]) == null)
} break outer;
long nc = (((long)(w.nextWait & E_MASK)) |
// (pollSubmission is defined below with exported methods) ((long)(u + UAC_UNIT) << 32));
if (w.eventCount != (e | INT_SIGN))
/** break outer;
* Creates or doubles submissionQueue array. if (U.compareAndSwapLong(this, CTL, c, nc)) {
* Basically identical to ForkJoinWorkerThread version. w.eventCount = (e + E_SEQ) & E_MASK;
*/ if ((p = w.parker) != null)
private void growSubmissionQueue() { U.unpark(p);
ForkJoinTask<?>[] oldQ = submissionQueue; if (--n <= 0)
int size = oldQ != null ? oldQ.length << 1 : INITIAL_QUEUE_CAPACITY; break;
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 to increment blockedCount, decrement active count * Tries to locate and execute tasks for a stealer of the given
* (sometimes implicitly) and possibly release or create a * task, or in turn one of its stealers, Traces currentSteal ->
* compensating worker in preparation for blocking. Fails * currentJoin links looking for a thread working on a descendant
* on contention or termination. * of the given task and with a non-empty queue to steal back and
* * execute tasks from. The first call to this method upon a
* @return true if the caller can block, else should recheck and retry * waiting join will often entail scanning/search, (which is OK
*/ * because the joiner has nothing better to do), but this method
private boolean tryPreBlock() { * leaves hints in workers to speed up subsequent calls. The
int b = blockedCount; * implementation is very branchy to cope with potential
if (UNSAFE.compareAndSwapInt(this, blockedCountOffset, b, b + 1)) { * inconsistencies or loops encountering chains that are stale,
int pc = parallelism; * unknown, or so long that they are likely cyclic.
do { *
ForkJoinWorkerThread[] ws; ForkJoinWorkerThread w; * @param joiner the joining worker
int e, ac, tc, rc, i; * @param task the task to join
long c = ctl; * @return 0 if no progress can be made, negative if task
int u = (int)(c >>> 32); * known complete, else positive
if ((e = (int)c) < 0) { */
// skip -- terminating private int tryHelpStealer(WorkQueue joiner, ForkJoinTask<?> task) {
} int stat = 0, steps = 0; // bound to avoid cycles
else if ((ac = (u >> UAC_SHIFT)) <= 0 && e != 0 && if (joiner != null && task != null) { // hoist null checks
(ws = workers) != null && restart: for (;;) {
(i = ~e & SMASK) < ws.length && ForkJoinTask<?> subtask = task; // current target
(w = ws[i]) != null) { for (WorkQueue j = joiner, v;;) { // v is stealer of subtask
long nc = ((long)(w.nextWait & E_MASK) | WorkQueue[] ws; int m, s, h;
(c & (AC_MASK|TC_MASK))); if ((s = task.status) < 0) {
if (w.eventCount == e && stat = s;
UNSAFE.compareAndSwapLong(this, ctlOffset, c, nc)) { break restart;
w.eventCount = (e + EC_UNIT) & E_MASK;
if (w.parked)
UNSAFE.unpark(w);
return true; // release an idle worker
} }
} if ((ws = workQueues) == null || (m = ws.length - 1) <= 0)
else if ((tc = (short)(u >>> UTC_SHIFT)) >= 0 && ac + pc > 1) { break restart; // shutting down
long nc = ((c - AC_UNIT) & AC_MASK) | (c & ~AC_MASK); if ((v = ws[h = (j.hint | 1) & m]) == null ||
if (UNSAFE.compareAndSwapLong(this, ctlOffset, c, nc)) v.currentSteal != subtask) {
return true; // no compensation needed for (int origin = h;;) { // find stealer
} if (((h = (h + 2) & m) & 15) == 1 &&
else if (tc + pc < MAX_ID) { (subtask.status < 0 || j.currentJoin != subtask))
long nc = ((c + TC_UNIT) & TC_MASK) | (c & ~TC_MASK); continue restart; // occasional staleness check
if (UNSAFE.compareAndSwapLong(this, ctlOffset, c, nc)) { if ((v = ws[h]) != null &&
addWorker(); v.currentSteal == subtask) {
return true; // create a replacement j.hint = h; // save hint
break;
}
if (h == origin)
break restart; // cannot find stealer
}
}
for (;;) { // help stealer or descend to its stealer
ForkJoinTask[] a; int b;
if (subtask.status < 0) // surround probes with
continue restart; // consistency checks
if ((b = v.base) - v.top < 0 && (a = v.array) != null) {
int i = (((a.length - 1) & b) << ASHIFT) + ABASE;
ForkJoinTask<?> t =
(ForkJoinTask<?>)U.getObjectVolatile(a, i);
if (subtask.status < 0 || j.currentJoin != subtask ||
v.currentSteal != subtask)
continue restart; // stale
stat = 1; // apparent progress
if (t != null && v.base == b &&
U.compareAndSwapObject(a, i, t, null)) {
v.base = b + 1; // help stealer
joiner.runSubtask(t);
}
else if (v.base == b && ++steps == MAX_HELP)
break restart; // v apparently stalled
}
else { // empty -- try to descend
ForkJoinTask<?> next = v.currentJoin;
if (subtask.status < 0 || j.currentJoin != subtask ||
v.currentSteal != subtask)
continue restart; // stale
else if (next == null || ++steps == MAX_HELP)
break restart; // dead-end or maybe cyclic
else {
subtask = next;
j = v;
break;
}
}
} }
} }
// try to back out on any failure and let caller retry }
} while (!UNSAFE.compareAndSwapInt(this, blockedCountOffset,
b = blockedCount, b - 1));
} }
return false; return stat;
} }
/** /**
* Decrements blockedCount and increments active count * Analog of tryHelpStealer for CountedCompleters. Tries to steal
*/ * and run tasks within the target's computation.
private void postBlock() { *
long c; * @param task the task to join
do {} while (!UNSAFE.compareAndSwapLong(this, ctlOffset, // no mask * @param mode if shared, exit upon completing any task
c = ctl, c + AC_UNIT)); * if all workers are active
int b;
do {} while (!UNSAFE.compareAndSwapInt(this, blockedCountOffset,
b = blockedCount, b - 1));
}
/**
* Possibly blocks waiting for the given task to complete, or
* cancels the task if terminating. Fails to wait if contended.
* *
* @param joinMe the task
*/ */
final void tryAwaitJoin(ForkJoinTask<?> joinMe) { private int helpComplete(ForkJoinTask<?> task, int mode) {
int s; WorkQueue[] ws; WorkQueue q; int m, n, s, u;
Thread.interrupted(); // clear interrupts before checking termination if (task != null && (ws = workQueues) != null &&
if (joinMe.status >= 0) { (m = ws.length - 1) >= 0) {
if (tryPreBlock()) { for (int j = 1, origin = j;;) {
joinMe.tryAwaitDone(0L); if ((s = task.status) < 0)
postBlock(); return s;
if ((q = ws[j & m]) != null && q.pollAndExecCC(task)) {
origin = j;
if (mode == SHARED_QUEUE &&
((u = (int)(ctl >>> 32)) >= 0 || (u >> UAC_SHIFT) >= 0))
break;
}
else if ((j = (j + 2) & m) == origin)
break;
}
}
return 0;
}
/**
* Tries to decrement active count (sometimes implicitly) and
* possibly release or create a compensating worker in preparation
* for blocking. Fails on contention or termination. Otherwise,
* adds a new thread if no idle workers are available and pool
* may become starved.
*/
final boolean tryCompensate() {
int pc = config & SMASK, e, i, tc; long c;
WorkQueue[] ws; WorkQueue w; Thread p;
if ((ws = workQueues) != null && (e = (int)(c = ctl)) >= 0) {
if (e != 0 && (i = e & SMASK) < ws.length &&
(w = ws[i]) != null && w.eventCount == (e | INT_SIGN)) {
long nc = ((long)(w.nextWait & E_MASK) |
(c & (AC_MASK|TC_MASK)));
if (U.compareAndSwapLong(this, CTL, c, nc)) {
w.eventCount = (e + E_SEQ) & E_MASK;
if ((p = w.parker) != null)
U.unpark(p);
return true; // replace with idle worker
}
}
else if ((tc = (short)(c >>> TC_SHIFT)) >= 0 &&
(int)(c >> AC_SHIFT) + pc > 1) {
long nc = ((c - AC_UNIT) & AC_MASK) | (c & ~AC_MASK);
if (U.compareAndSwapLong(this, CTL, c, nc))
return true; // no compensation
}
else if (tc + pc < MAX_CAP) {
long nc = ((c + TC_UNIT) & TC_MASK) | (c & ~TC_MASK);
if (U.compareAndSwapLong(this, CTL, c, nc)) {
ForkJoinWorkerThreadFactory fac;
Throwable ex = null;
ForkJoinWorkerThread wt = null;
try {
if ((fac = factory) != null &&
(wt = fac.newThread(this)) != null) {
wt.start();
return true;
}
} catch (Throwable rex) {
ex = rex;
}
deregisterWorker(wt, ex); // clean up and return false
}
} }
else if ((ctl & STOP_BIT) != 0L)
joinMe.cancelIgnoringExceptions();
} }
return false;
} }
/** /**
* Possibly blocks the given worker waiting for joinMe to * Helps and/or blocks until the given task is done.
* complete or timeout
* *
* @param joinMe the task * @param joiner the joining worker
* @param millis the wait time for underlying Object.wait * @param task the task
*/ * @return task status on exit
final void timedAwaitJoin(ForkJoinTask<?> joinMe, long nanos) { */
while (joinMe.status >= 0) { final int awaitJoin(WorkQueue joiner, ForkJoinTask<?> task) {
Thread.interrupted(); int s = 0;
if ((ctl & STOP_BIT) != 0L) { if (joiner != null && task != null && (s = task.status) >= 0) {
joinMe.cancelIgnoringExceptions(); ForkJoinTask<?> prevJoin = joiner.currentJoin;
break; joiner.currentJoin = task;
do {} while ((s = task.status) >= 0 && !joiner.isEmpty() &&
joiner.tryRemoveAndExec(task)); // process local tasks
if (s >= 0 && (s = task.status) >= 0) {
helpSignal(task, joiner.poolIndex);
if ((s = task.status) >= 0 &&
(task instanceof CountedCompleter))
s = helpComplete(task, LIFO_QUEUE);
} }
if (tryPreBlock()) { while (s >= 0 && (s = task.status) >= 0) {
long last = System.nanoTime(); if ((!joiner.isEmpty() || // try helping
while (joinMe.status >= 0) { (s = tryHelpStealer(joiner, task)) == 0) &&
long millis = TimeUnit.NANOSECONDS.toMillis(nanos); (s = task.status) >= 0) {
if (millis <= 0) helpSignal(task, joiner.poolIndex);
break; if ((s = task.status) >= 0 && tryCompensate()) {
joinMe.tryAwaitDone(millis); if (task.trySetSignal() && (s = task.status) >= 0) {
if (joinMe.status < 0) synchronized (task) {
break; if (task.status >= 0) {
if ((ctl & STOP_BIT) != 0L) { try { // see ForkJoinTask
joinMe.cancelIgnoringExceptions(); task.wait(); // for explanation
break; } catch (InterruptedException ie) {
}
}
else
task.notifyAll();
}
}
long c; // re-activate
do {} while (!U.compareAndSwapLong
(this, CTL, c = ctl, c + AC_UNIT));
} }
long now = System.nanoTime();
nanos -= now - last;
last = now;
} }
postBlock();
break;
} }
joiner.currentJoin = prevJoin;
} }
return s;
} }
/** /**
* If necessary, compensates for blocker, and blocks * Stripped-down variant of awaitJoin used by timed joins. Tries
* to help join only while there is continuous progress. (Caller
* will then enter a timed wait.)
*
* @param joiner the joining worker
* @param task the task
*/ */
private void awaitBlocker(ManagedBlocker blocker) final void helpJoinOnce(WorkQueue joiner, ForkJoinTask<?> task) {
throws InterruptedException { int s;
while (!blocker.isReleasable()) { if (joiner != null && task != null && (s = task.status) >= 0) {
if (tryPreBlock()) { ForkJoinTask<?> prevJoin = joiner.currentJoin;
try { joiner.currentJoin = task;
do {} while (!blocker.isReleasable() && !blocker.block()); do {} while ((s = task.status) >= 0 && !joiner.isEmpty() &&
} finally { joiner.tryRemoveAndExec(task));
postBlock(); if (s >= 0 && (s = task.status) >= 0) {
} helpSignal(task, joiner.poolIndex);
break; if ((s = task.status) >= 0 &&
(task instanceof CountedCompleter))
s = helpComplete(task, LIFO_QUEUE);
} }
if (s >= 0 && joiner.isEmpty()) {
do {} while (task.status >= 0 &&
tryHelpStealer(joiner, task) > 0);
}
joiner.currentJoin = prevJoin;
} }
} }
// Creating, registering and deregistring workers
/** /**
* Tries to create and start a worker; minimally rolls back counts * Returns a (probably) non-empty steal queue, if one is found
* on failure. * during a random, then cyclic scan, else null. This method must
*/ * be retried by caller if, by the time it tries to use the queue,
private void addWorker() { * it is empty.
Throwable ex = null; * @param r a (random) seed for scanning
ForkJoinWorkerThread t = null; */
try { private WorkQueue findNonEmptyStealQueue(int r) {
t = factory.newThread(this); for (WorkQueue[] ws;;) {
} catch (Throwable e) { int ps = plock, m, n;
ex = e; if ((ws = workQueues) == null || (m = ws.length - 1) < 1)
} return null;
if (t == null) { // null or exceptional factory return for (int j = (m + 1) << 2; ;) {
long c; // adjust counts WorkQueue q = ws[(((r + j) << 1) | 1) & m];
do {} while (!UNSAFE.compareAndSwapLong if (q != null && (n = q.base - q.top) < 0) {
(this, ctlOffset, c = ctl, if (n < -1)
(((c - AC_UNIT) & AC_MASK) | signalWork(q);
((c - TC_UNIT) & TC_MASK) | return q;
(c & ~(AC_MASK|TC_MASK))))); }
// Propagate exception if originating from an external caller else if (--j < 0) {
if (!tryTerminate(false) && ex != null && if (plock == ps)
!(Thread.currentThread() instanceof ForkJoinWorkerThread)) return null;
UNSAFE.throwException(ex); break;
}
}
} }
else
t.start();
} }
/** /**
* Callback from ForkJoinWorkerThread constructor to assign a * Runs tasks until {@code isQuiescent()}. We piggyback on
* public name * active count ctl maintenance, but rather than blocking
*/ * when tasks cannot be found, we rescan until all others cannot
final String nextWorkerName() { * find tasks either.
for (int n;;) { */
if (UNSAFE.compareAndSwapInt(this, nextWorkerNumberOffset, final void helpQuiescePool(WorkQueue w) {
n = nextWorkerNumber, ++n)) for (boolean active = true;;) {
return workerNamePrefix + n; ForkJoinTask<?> localTask; // exhaust local queue
while ((localTask = w.nextLocalTask()) != null)
localTask.doExec();
// Similar to loop in scan(), but ignoring submissions
WorkQueue q = findNonEmptyStealQueue(w.nextSeed());
if (q != null) {
ForkJoinTask<?> t; int b;
if (!active) { // re-establish active count
long c;
active = true;
do {} while (!U.compareAndSwapLong
(this, CTL, c = ctl, c + AC_UNIT));
}
if ((b = q.base) - q.top < 0 && (t = q.pollAt(b)) != null)
w.runSubtask(t);
}
else {
long c;
if (active) { // decrement active count without queuing
active = false;
do {} while (!U.compareAndSwapLong
(this, CTL, c = ctl, c -= AC_UNIT));
}
else
c = ctl; // re-increment on exit
if ((int)(c >> AC_SHIFT) + (config & SMASK) == 0) {
do {} while (!U.compareAndSwapLong
(this, CTL, c = ctl, c + AC_UNIT));
break;
}
}
} }
} }
/** /**
* Callback from ForkJoinWorkerThread constructor to * Gets and removes a local or stolen task for the given worker.
* determine its poolIndex and record in workers array.
* *
* @param w the worker * @return a task, if available
* @return the worker's pool index
*/ */
final int registerWorker(ForkJoinWorkerThread w) { final ForkJoinTask<?> nextTaskFor(WorkQueue w) {
/* for (ForkJoinTask<?> t;;) {
* In the typical case, a new worker acquires the lock, uses WorkQueue q; int b;
* next available index and returns quickly. Since we should if ((t = w.nextLocalTask()) != null)
* not block callers (ultimately from signalWork or return t;
* tryPreBlock) waiting for the lock needed to do this, we if ((q = findNonEmptyStealQueue(w.nextSeed())) == null)
* instead help release other workers while waiting for the return null;
* lock. if ((b = q.base) - q.top < 0 && (t = q.pollAt(b)) != null)
*/ return t;
for (int g;;) {
ForkJoinWorkerThread[] ws;
if (((g = scanGuard) & SG_UNIT) == 0 &&
UNSAFE.compareAndSwapInt(this, scanGuardOffset,
g, g | SG_UNIT)) {
int k = nextWorkerIndex;
try {
if ((ws = workers) != null) { // ignore on shutdown
int n = ws.length;
if (k < 0 || k >= n || ws[k] != null) {
for (k = 0; k < n && ws[k] != null; ++k)
;
if (k == n)
ws = workers = Arrays.copyOf(ws, n << 1);
}
ws[k] = w;
nextWorkerIndex = k + 1;
int m = g & SMASK;
g = (k > m) ? ((m << 1) + 1) & SMASK : g + (SG_UNIT<<1);
}
} finally {
scanGuard = g;
}
return k;
}
else if ((ws = workers) != null) { // help release others
for (ForkJoinWorkerThread u : ws) {
if (u != null && u.queueBase != u.queueTop) {
if (tryReleaseWaiter())
break;
}
}
}
} }
} }
/** /**
* Final callback from terminating worker. Removes record of * Returns a cheap heuristic guide for task partitioning when
* worker from array, and adjusts counts. If pool is shutting * programmers, frameworks, tools, or languages have little or no
* down, tries to complete termination. * idea about task granularity. In essence by offering this
* * method, we ask users only about tradeoffs in overhead vs
* @param w the worker * expected throughput and its variance, rather than how finely to
*/ * partition tasks.
final void deregisterWorker(ForkJoinWorkerThread w, Throwable ex) { *
int idx = w.poolIndex; * In a steady state strict (tree-structured) computation, each
int sc = w.stealCount; * thread makes available for stealing enough tasks for other
int steps = 0; * threads to remain active. Inductively, if all threads play by
// Remove from array, adjust worker counts and collect steal count. * the same rules, each thread should make available only a
// We can intermix failed removes or adjusts with steal updates * constant number of tasks.
do { *
long s, c; * The minimum useful constant is just 1. But using a value of 1
int g; * would require immediate replenishment upon each steal to
if (steps == 0 && ((g = scanGuard) & SG_UNIT) == 0 && * maintain enough tasks, which is infeasible. Further,
UNSAFE.compareAndSwapInt(this, scanGuardOffset, * partitionings/granularities of offered tasks should minimize
g, g |= SG_UNIT)) { * steal rates, which in general means that threads nearer the top
ForkJoinWorkerThread[] ws = workers; * of computation tree should generate more than those nearer the
if (ws != null && idx >= 0 && * bottom. In perfect steady state, each thread is at
idx < ws.length && ws[idx] == w) * approximately the same level of computation tree. However,
ws[idx] = null; // verify * producing extra tasks amortizes the uncertainty of progress and
nextWorkerIndex = idx; * diffusion assumptions.
scanGuard = g + SG_UNIT; *
steps = 1; * So, users will want to use values larger, but not much larger
} * than 1 to both smooth over transient shortages and hedge
if (steps == 1 && * against uneven progress; as traded off against the cost of
UNSAFE.compareAndSwapLong(this, ctlOffset, c = ctl, * extra task overhead. We leave the user to pick a threshold
(((c - AC_UNIT) & AC_MASK) | * value to compare with the results of this call to guide
((c - TC_UNIT) & TC_MASK) | * decisions, but recommend values such as 3.
(c & ~(AC_MASK|TC_MASK))))) *
steps = 2; * When all threads are active, it is on average OK to estimate
if (sc != 0 && * surplus strictly locally. In steady-state, if one thread is
UNSAFE.compareAndSwapLong(this, stealCountOffset, * maintaining say 2 surplus tasks, then so are others. So we can
s = stealCount, s + sc)) * just use estimated queue length. However, this strategy alone
sc = 0; * leads to serious mis-estimates in some non-steady-state
} while (steps != 2 || sc != 0); * conditions (ramp-up, ramp-down, other stalls). We can detect
if (!tryTerminate(false)) { * many of these by further considering the number of "idle"
if (ex != null) // possibly replace if died abnormally * threads, that are known to have zero queued tasks, so
signalWork(); * compensate by a factor of (#idle/#active) threads.
else *
tryReleaseWaiter(); * Note: The approximation of #busy workers as #active workers is
* not very good under current signalling scheme, and should be
* improved.
*/
static int getSurplusQueuedTaskCount() {
Thread t; ForkJoinWorkerThread wt; ForkJoinPool pool; WorkQueue q;
if (((t = Thread.currentThread()) instanceof ForkJoinWorkerThread)) {
int p = (pool = (wt = (ForkJoinWorkerThread)t).pool).config & SMASK;
int n = (q = wt.workQueue).top - q.base;
int a = (int)(pool.ctl >> AC_SHIFT) + p;
return n - (a > (p >>>= 1) ? 0 :
a > (p >>>= 1) ? 1 :
a > (p >>>= 1) ? 2 :
a > (p >>>= 1) ? 4 :
8);
} }
return 0;
} }
// Shutdown and termination // Termination
/** /**
* Possibly initiates and/or completes termination. * Possibly initiates and/or completes termination. The caller
* triggering termination runs three passes through workQueues:
* (0) Setting termination status, followed by wakeups of queued
* workers; (1) cancelling all tasks; (2) interrupting lagging
* threads (likely in external tasks, but possibly also blocked in
* joins). Each pass repeats previous steps because of potential
* lagging thread creation.
* *
* @param now if true, unconditionally terminate, else only * @param now if true, unconditionally terminate, else only
* if shutdown and empty queue and no active workers * if no work and no active workers
* @param enable if true, enable shutdown when next possible
* @return true if now terminating or terminated * @return true if now terminating or terminated
*/ */
private boolean tryTerminate(boolean now) { private boolean tryTerminate(boolean now, boolean enable) {
long c; if (this == commonPool) // cannot shut down
while (((c = ctl) & STOP_BIT) == 0) { return false;
if (!now) { for (long c;;) {
if ((int)(c >> AC_SHIFT) != -parallelism) if (((c = ctl) & STOP_BIT) != 0) { // already terminating
return false; if ((short)(c >>> TC_SHIFT) == -(config & SMASK)) {
if (!shutdown || blockedCount != 0 || quiescerCount != 0 || synchronized (this) {
queueBase != queueTop) { notifyAll(); // signal when 0 workers
if (ctl == c) // staleness check }
return false;
continue;
} }
return true;
} }
if (UNSAFE.compareAndSwapLong(this, ctlOffset, c, c | STOP_BIT)) if (plock >= 0) { // not yet enabled
startTerminating(); int ps;
} if (!enable)
if ((short)(c >>> TC_SHIFT) == -parallelism) { // signal when 0 workers return false;
final ReentrantLock lock = this.submissionLock; if (((ps = plock) & PL_LOCK) != 0 ||
lock.lock(); !U.compareAndSwapInt(this, PLOCK, ps, ps += PL_LOCK))
try { ps = acquirePlock();
termination.signalAll(); if (!U.compareAndSwapInt(this, PLOCK, ps, SHUTDOWN))
} finally { releasePlock(SHUTDOWN);
lock.unlock();
} }
} if (!now) { // check if idle & no tasks
return true; if ((int)(c >> AC_SHIFT) != -(config & SMASK) ||
} hasQueuedSubmissions())
return false;
/** // Check for unqueued inactive workers. One pass suffices.
* Runs up to three passes through workers: (0) Setting WorkQueue[] ws = workQueues; WorkQueue w;
* termination status for each worker, followed by wakeups up to if (ws != null) {
* queued workers; (1) helping cancel tasks; (2) interrupting for (int i = 1; i < ws.length; i += 2) {
* lagging threads (likely in external tasks, but possibly also if ((w = ws[i]) != null && w.eventCount >= 0)
* blocked in joins). Each pass repeats previous steps because of return false;
* potential lagging thread creation. }
*/ }
private void startTerminating() { }
cancelSubmissions(); if (U.compareAndSwapLong(this, CTL, c, c | STOP_BIT)) {
for (int pass = 0; pass < 3; ++pass) { for (int pass = 0; pass < 3; ++pass) {
ForkJoinWorkerThread[] ws = workers; WorkQueue[] ws = workQueues;
if (ws != null) { if (ws != null) {
for (ForkJoinWorkerThread w : ws) { WorkQueue w; Thread wt;
if (w != null) { int n = ws.length;
w.terminate = true; for (int i = 0; i < n; ++i) {
if (pass > 0) { if ((w = ws[i]) != null) {
w.cancelTasks(); w.qlock = -1;
if (pass > 1 && !w.isInterrupted()) { if (pass > 0) {
try { w.cancelAll();
w.interrupt(); if (pass > 1 && (wt = w.owner) != null) {
} catch (SecurityException ignore) { if (!wt.isInterrupted()) {
try {
wt.interrupt();
} catch (SecurityException ignore) {
}
}
U.unpark(wt);
}
} }
} }
} }
// Wake up workers parked on event queue
int i, e; long cc; Thread p;
while ((e = (int)(cc = ctl) & E_MASK) != 0 &&
(i = e & SMASK) < n &&
(w = ws[i]) != null) {
long nc = ((long)(w.nextWait & E_MASK) |
((cc + AC_UNIT) & AC_MASK) |
(cc & (TC_MASK|STOP_BIT)));
if (w.eventCount == (e | INT_SIGN) &&
U.compareAndSwapLong(this, CTL, cc, nc)) {
w.eventCount = (e + E_SEQ) & E_MASK;
w.qlock = -1;
if ((p = w.parker) != null)
U.unpark(p);
}
}
} }
} }
terminateWaiters();
} }
} }
} }
// external operations on common pool
/** /**
* Polls and cancels all submissions. Called only during termination. * Returns common pool queue for a thread that has submitted at
* least one task.
*/ */
private void cancelSubmissions() { static WorkQueue commonSubmitterQueue() {
while (queueBase != queueTop) { ForkJoinPool p; WorkQueue[] ws; int m; Submitter z;
ForkJoinTask<?> task = pollSubmission(); return ((z = submitters.get()) != null &&
if (task != null) { (p = commonPool) != null &&
try { (ws = p.workQueues) != null &&
task.cancel(false); (m = ws.length - 1) >= 0) ?
} catch (Throwable ignore) { ws[m & z.seed & SQMASK] : null;
}
}
}
} }
/** /**
* Tries to set the termination status of waiting workers, and * Tries to pop the given task from submitter's queue in common pool.
* then wakes them up (after which they will terminate). */
*/ static boolean tryExternalUnpush(ForkJoinTask<?> t) {
private void terminateWaiters() { ForkJoinPool p; WorkQueue[] ws; WorkQueue q; Submitter z;
ForkJoinWorkerThread[] ws = workers; ForkJoinTask<?>[] a; int m, s;
if (ws != null) { if (t != null &&
ForkJoinWorkerThread w; long c; int i, e; (z = submitters.get()) != null &&
int n = ws.length; (p = commonPool) != null &&
while ((i = ~(e = (int)(c = ctl)) & SMASK) < n && (ws = p.workQueues) != null &&
(w = ws[i]) != null && w.eventCount == (e & E_MASK)) { (m = ws.length - 1) >= 0 &&
if (UNSAFE.compareAndSwapLong(this, ctlOffset, c, (q = ws[m & z.seed & SQMASK]) != null &&
(long)(w.nextWait & E_MASK) | (s = q.top) != q.base &&
((c + AC_UNIT) & AC_MASK) | (a = q.array) != null) {
(c & (TC_MASK|STOP_BIT)))) { long j = (((a.length - 1) & (s - 1)) << ASHIFT) + ABASE;
w.terminate = true; if (U.getObject(a, j) == t &&
w.eventCount = e + EC_UNIT; U.compareAndSwapInt(q, QLOCK, 0, 1)) {
if (w.parked) if (q.array == a && q.top == s && // recheck
UNSAFE.unpark(w); U.compareAndSwapObject(a, j, t, null)) {
q.top = s - 1;
q.qlock = 0;
return true;
} }
q.qlock = 0;
} }
} }
return false;
} }
// misc ForkJoinWorkerThread support
/** /**
* Increment or decrement quiescerCount. Needed only to prevent * Tries to pop and run local tasks within the same computation
* triggering shutdown if a worker is transiently inactive while * as the given root. On failure, tries to help complete from
* checking quiescence. * other queues via helpComplete.
*
* @param delta 1 for increment, -1 for decrement
*/ */
final void addQuiescerCount(int delta) { private void externalHelpComplete(WorkQueue q, ForkJoinTask<?> root) {
int c; ForkJoinTask<?>[] a; int m;
do {} while (!UNSAFE.compareAndSwapInt(this, quiescerCountOffset, if (q != null && (a = q.array) != null && (m = (a.length - 1)) >= 0 &&
c = quiescerCount, c + delta)); root != null && root.status >= 0) {
for (;;) {
int s, u; Object o; CountedCompleter<?> task = null;
if ((s = q.top) - q.base > 0) {
long j = ((m & (s - 1)) << ASHIFT) + ABASE;
if ((o = U.getObject(a, j)) != null &&
(o instanceof CountedCompleter)) {
CountedCompleter<?> t = (CountedCompleter<?>)o, r = t;
do {
if (r == root) {
if (U.compareAndSwapInt(q, QLOCK, 0, 1)) {
if (q.array == a && q.top == s &&
U.compareAndSwapObject(a, j, t, null)) {
q.top = s - 1;
task = t;
}
q.qlock = 0;
}
break;
}
} while ((r = r.completer) != null);
}
}
if (task != null)
task.doExec();
if (root.status < 0 ||
(u = (int)(ctl >>> 32)) >= 0 || (u >> UAC_SHIFT) >= 0)
break;
if (task == null) {
helpSignal(root, q.poolIndex);
if (root.status >= 0)
helpComplete(root, SHARED_QUEUE);
break;
}
}
}
} }
/** /**
* Directly increment or decrement active count without * Tries to help execute or signal availability of the given task
* queuing. This method is used to transiently assert inactivation * from submitter's queue in common pool.
* while checking quiescence. */
* static void externalHelpJoin(ForkJoinTask<?> t) {
* @param delta 1 for increment, -1 for decrement // Some hard-to-avoid overlap with tryExternalUnpush
*/ ForkJoinPool p; WorkQueue[] ws; WorkQueue q, w; Submitter z;
final void addActiveCount(int delta) { ForkJoinTask<?>[] a; int m, s, n;
long d = delta < 0 ? -AC_UNIT : AC_UNIT; if (t != null &&
long c; (z = submitters.get()) != null &&
do {} while (!UNSAFE.compareAndSwapLong(this, ctlOffset, c = ctl, (p = commonPool) != null &&
((c + d) & AC_MASK) | (ws = p.workQueues) != null &&
(c & ~AC_MASK))); (m = ws.length - 1) >= 0 &&
(q = ws[m & z.seed & SQMASK]) != null &&
(a = q.array) != null) {
int am = a.length - 1;
if ((s = q.top) != q.base) {
long j = ((am & (s - 1)) << ASHIFT) + ABASE;
if (U.getObject(a, j) == t &&
U.compareAndSwapInt(q, QLOCK, 0, 1)) {
if (q.array == a && q.top == s &&
U.compareAndSwapObject(a, j, t, null)) {
q.top = s - 1;
q.qlock = 0;
t.doExec();
}
else
q.qlock = 0;
}
}
if (t.status >= 0) {
if (t instanceof CountedCompleter)
p.externalHelpComplete(q, t);
else
p.helpSignal(t, q.poolIndex);
}
}
} }
/** /**
* Returns the approximate (non-atomic) number of idle threads per * Restricted version of helpQuiescePool for external callers
* active thread.
*/ */
final int idlePerActive() { static void externalHelpQuiescePool() {
// Approximate at powers of two for small values, saturate past 4 ForkJoinPool p; ForkJoinTask<?> t; WorkQueue q; int b;
int p = parallelism; if ((p = commonPool) != null &&
int a = p + (int)(ctl >> AC_SHIFT); (q = p.findNonEmptyStealQueue(1)) != null &&
return (a > (p >>>= 1) ? 0 : (b = q.base) - q.top < 0 &&
a > (p >>>= 1) ? 1 : (t = q.pollAt(b)) != null)
a > (p >>>= 1) ? 2 : t.doExec();
a > (p >>>= 1) ? 4 :
8);
} }
// Exported methods // Exported methods
...@@ -1464,31 +2577,46 @@ public class ForkJoinPool extends AbstractExecutorService { ...@@ -1464,31 +2577,46 @@ public class ForkJoinPool extends AbstractExecutorService {
checkPermission(); checkPermission();
if (factory == null) if (factory == null)
throw new NullPointerException(); throw new NullPointerException();
if (parallelism <= 0 || parallelism > MAX_ID) if (parallelism <= 0 || parallelism > MAX_CAP)
throw new IllegalArgumentException(); throw new IllegalArgumentException();
this.parallelism = parallelism;
this.factory = factory; this.factory = factory;
this.ueh = handler; this.ueh = handler;
this.locallyFifo = asyncMode; this.config = parallelism | (asyncMode ? (FIFO_QUEUE << 16) : 0);
long np = (long)(-parallelism); // offset ctl counts long np = (long)(-parallelism); // offset ctl counts
this.ctl = ((np << AC_SHIFT) & AC_MASK) | ((np << TC_SHIFT) & TC_MASK); this.ctl = ((np << AC_SHIFT) & AC_MASK) | ((np << TC_SHIFT) & TC_MASK);
this.submissionQueue = new ForkJoinTask<?>[INITIAL_QUEUE_CAPACITY]; int pn = nextPoolId();
// initialize workers array with room for 2*parallelism if possible
int n = parallelism << 1;
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;
}
workers = new ForkJoinWorkerThread[n + 1];
this.submissionLock = new ReentrantLock();
this.termination = submissionLock.newCondition();
StringBuilder sb = new StringBuilder("ForkJoinPool-"); StringBuilder sb = new StringBuilder("ForkJoinPool-");
sb.append(poolNumberGenerator.incrementAndGet()); sb.append(Integer.toString(pn));
sb.append("-worker-"); sb.append("-worker-");
this.workerNamePrefix = sb.toString(); this.workerNamePrefix = sb.toString();
} }
/**
* Constructor for common pool, suitable only for static initialization.
* Basically the same as above, but uses smallest possible initial footprint.
*/
ForkJoinPool(int parallelism, long ctl,
ForkJoinWorkerThreadFactory factory,
Thread.UncaughtExceptionHandler handler) {
this.config = parallelism;
this.ctl = ctl;
this.factory = factory;
this.ueh = handler;
this.workerNamePrefix = "ForkJoinPool.commonPool-worker-";
}
/**
* Returns the common pool instance. This pool is statically
* constructed; its run state is unaffected by attempts to
* {@link #shutdown} or {@link #shutdownNow}.
*
* @return the common pool instance
*/
public static ForkJoinPool commonPool() {
// assert commonPool != null : "static init error";
return commonPool;
}
// Execution methods // Execution methods
/** /**
...@@ -1508,34 +2636,10 @@ public class ForkJoinPool extends AbstractExecutorService { ...@@ -1508,34 +2636,10 @@ 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 (shutdown) externalPush(task);
throw new RejectedExecutionException(); return task.join();
if ((t instanceof ForkJoinWorkerThread) &&
((ForkJoinWorkerThread)t).pool == this)
return task.invoke(); // bypass submit if in same pool
else {
addSubmission(task);
return task.join();
}
}
/**
* Unless terminating, forks task if within an ongoing FJ
* computation in the current pool, else submits as external task.
*/
private <T> void forkOrSubmit(ForkJoinTask<T> task) {
ForkJoinWorkerThread w;
Thread t = Thread.currentThread();
if (shutdown)
throw new RejectedExecutionException();
if ((t instanceof ForkJoinWorkerThread) &&
(w = (ForkJoinWorkerThread)t).pool == this)
w.pushTask(task);
else
addSubmission(task);
} }
/** /**
...@@ -1549,7 +2653,7 @@ public class ForkJoinPool extends AbstractExecutorService { ...@@ -1549,7 +2653,7 @@ public class ForkJoinPool extends AbstractExecutorService {
public void execute(ForkJoinTask<?> task) { public void execute(ForkJoinTask<?> task) {
if (task == null) if (task == null)
throw new NullPointerException(); throw new NullPointerException();
forkOrSubmit(task); externalPush(task);
} }
// AbstractExecutorService methods // AbstractExecutorService methods
...@@ -1566,8 +2670,8 @@ public class ForkJoinPool extends AbstractExecutorService { ...@@ -1566,8 +2670,8 @@ public class ForkJoinPool extends AbstractExecutorService {
if (task instanceof ForkJoinTask<?>) // avoid re-wrap if (task instanceof ForkJoinTask<?>) // avoid re-wrap
job = (ForkJoinTask<?>) task; job = (ForkJoinTask<?>) task;
else else
job = ForkJoinTask.adapt(task, null); job = new ForkJoinTask.AdaptedRunnableAction(task);
forkOrSubmit(job); externalPush(job);
} }
/** /**
...@@ -1582,7 +2686,7 @@ public class ForkJoinPool extends AbstractExecutorService { ...@@ -1582,7 +2686,7 @@ public class ForkJoinPool extends AbstractExecutorService {
public <T> ForkJoinTask<T> submit(ForkJoinTask<T> task) { public <T> ForkJoinTask<T> submit(ForkJoinTask<T> task) {
if (task == null) if (task == null)
throw new NullPointerException(); throw new NullPointerException();
forkOrSubmit(task); externalPush(task);
return task; return task;
} }
...@@ -1592,10 +2696,8 @@ public class ForkJoinPool extends AbstractExecutorService { ...@@ -1592,10 +2696,8 @@ public class ForkJoinPool extends AbstractExecutorService {
* scheduled for execution * scheduled for execution
*/ */
public <T> ForkJoinTask<T> submit(Callable<T> task) { public <T> ForkJoinTask<T> submit(Callable<T> task) {
if (task == null) ForkJoinTask<T> job = new ForkJoinTask.AdaptedCallable<T>(task);
throw new NullPointerException(); externalPush(job);
ForkJoinTask<T> job = ForkJoinTask.adapt(task);
forkOrSubmit(job);
return job; return job;
} }
...@@ -1605,10 +2707,8 @@ public class ForkJoinPool extends AbstractExecutorService { ...@@ -1605,10 +2707,8 @@ public class ForkJoinPool extends AbstractExecutorService {
* scheduled for execution * scheduled for execution
*/ */
public <T> ForkJoinTask<T> submit(Runnable task, T result) { public <T> ForkJoinTask<T> submit(Runnable task, T result) {
if (task == null) ForkJoinTask<T> job = new ForkJoinTask.AdaptedRunnable<T>(task, result);
throw new NullPointerException(); externalPush(job);
ForkJoinTask<T> job = ForkJoinTask.adapt(task, result);
forkOrSubmit(job);
return job; return job;
} }
...@@ -1624,8 +2724,8 @@ public class ForkJoinPool extends AbstractExecutorService { ...@@ -1624,8 +2724,8 @@ public class ForkJoinPool extends AbstractExecutorService {
if (task instanceof ForkJoinTask<?>) // avoid re-wrap if (task instanceof ForkJoinTask<?>) // avoid re-wrap
job = (ForkJoinTask<?>) task; job = (ForkJoinTask<?>) task;
else else
job = ForkJoinTask.adapt(task, null); job = new ForkJoinTask.AdaptedRunnableAction(task);
forkOrSubmit(job); externalPush(job);
return job; return job;
} }
...@@ -1634,25 +2734,31 @@ public class ForkJoinPool extends AbstractExecutorService { ...@@ -1634,25 +2734,31 @@ public class ForkJoinPool extends AbstractExecutorService {
* @throws RejectedExecutionException {@inheritDoc} * @throws RejectedExecutionException {@inheritDoc}
*/ */
public <T> List<Future<T>> invokeAll(Collection<? extends Callable<T>> tasks) { public <T> List<Future<T>> invokeAll(Collection<? extends Callable<T>> tasks) {
ArrayList<ForkJoinTask<T>> forkJoinTasks = // In previous versions of this class, this method constructed
new ArrayList<ForkJoinTask<T>>(tasks.size()); // a task to run ForkJoinTask.invokeAll, but now external
for (Callable<T> task : tasks) // invocation of multiple tasks is at least as efficient.
forkJoinTasks.add(ForkJoinTask.adapt(task)); List<ForkJoinTask<T>> fs = new ArrayList<ForkJoinTask<T>>(tasks.size());
invoke(new InvokeAll<T>(forkJoinTasks)); // Workaround needed because method wasn't declared with
// wildcards in return type but should have been.
@SuppressWarnings({"unchecked", "rawtypes"}) @SuppressWarnings({"unchecked", "rawtypes"})
List<Future<T>> futures = (List<Future<T>>) (List) forkJoinTasks; List<Future<T>> futures = (List<Future<T>>) (List) fs;
return futures;
}
static final class InvokeAll<T> extends RecursiveAction { boolean done = false;
final ArrayList<ForkJoinTask<T>> tasks; try {
InvokeAll(ArrayList<ForkJoinTask<T>> tasks) { this.tasks = tasks; } for (Callable<T> t : tasks) {
public void compute() { ForkJoinTask<T> f = new ForkJoinTask.AdaptedCallable<T>(t);
try { invokeAll(tasks); } externalPush(f);
catch (Exception ignore) {} fs.add(f);
}
for (ForkJoinTask<T> f : fs)
f.quietlyJoin();
done = true;
return futures;
} finally {
if (!done)
for (ForkJoinTask<T> f : fs)
f.cancel(false);
} }
private static final long serialVersionUID = -7914297376763021607L;
} }
/** /**
...@@ -1680,7 +2786,16 @@ public class ForkJoinPool extends AbstractExecutorService { ...@@ -1680,7 +2786,16 @@ public class ForkJoinPool extends AbstractExecutorService {
* @return the targeted parallelism level of this pool * @return the targeted parallelism level of this pool
*/ */
public int getParallelism() { public int getParallelism() {
return parallelism; return config & SMASK;
}
/**
* Returns the targeted parallelism level of the common pool.
*
* @return the targeted parallelism level of the common pool
*/
public static int getCommonPoolParallelism() {
return commonPoolParallelism;
} }
/** /**
...@@ -1692,7 +2807,7 @@ public class ForkJoinPool extends AbstractExecutorService { ...@@ -1692,7 +2807,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 parallelism + (short)(ctl >>> TC_SHIFT); return (config & SMASK) + (short)(ctl >>> TC_SHIFT);
} }
/** /**
...@@ -1702,7 +2817,7 @@ public class ForkJoinPool extends AbstractExecutorService { ...@@ -1702,7 +2817,7 @@ public class ForkJoinPool extends AbstractExecutorService {
* @return {@code true} if this pool uses async mode * @return {@code true} if this pool uses async mode
*/ */
public boolean getAsyncMode() { public boolean getAsyncMode() {
return locallyFifo; return (config >>> 16) == FIFO_QUEUE;
} }
/** /**
...@@ -1714,8 +2829,15 @@ public class ForkJoinPool extends AbstractExecutorService { ...@@ -1714,8 +2829,15 @@ public class ForkJoinPool extends AbstractExecutorService {
* @return the number of worker threads * @return the number of worker threads
*/ */
public int getRunningThreadCount() { public int getRunningThreadCount() {
int r = parallelism + (int)(ctl >> AC_SHIFT); int rc = 0;
return (r <= 0) ? 0 : r; // suppress momentarily negative values WorkQueue[] ws; WorkQueue w;
if ((ws = workQueues) != null) {
for (int i = 1; i < ws.length; i += 2) {
if ((w = ws[i]) != null && w.isApparentlyUnblocked())
++rc;
}
}
return rc;
} }
/** /**
...@@ -1726,7 +2848,7 @@ public class ForkJoinPool extends AbstractExecutorService { ...@@ -1726,7 +2848,7 @@ public class ForkJoinPool extends AbstractExecutorService {
* @return the number of active threads * @return the number of active threads
*/ */
public int getActiveThreadCount() { public int getActiveThreadCount() {
int r = parallelism + (int)(ctl >> AC_SHIFT) + blockedCount; int r = (config & SMASK) + (int)(ctl >> AC_SHIFT);
return (r <= 0) ? 0 : r; // suppress momentarily negative values return (r <= 0) ? 0 : r; // suppress momentarily negative values
} }
...@@ -1742,7 +2864,7 @@ public class ForkJoinPool extends AbstractExecutorService { ...@@ -1742,7 +2864,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 parallelism + (int)(ctl >> AC_SHIFT) + blockedCount == 0; return (int)(ctl >> AC_SHIFT) + (config & SMASK) == 0;
} }
/** /**
...@@ -1757,7 +2879,15 @@ public class ForkJoinPool extends AbstractExecutorService { ...@@ -1757,7 +2879,15 @@ public class ForkJoinPool extends AbstractExecutorService {
* @return the number of steals * @return the number of steals
*/ */
public long getStealCount() { public long getStealCount() {
return stealCount; long count = stealCount;
WorkQueue[] ws; WorkQueue w;
if ((ws = workQueues) != null) {
for (int i = 1; i < ws.length; i += 2) {
if ((w = ws[i]) != null)
count += w.nsteals;
}
}
return count;
} }
/** /**
...@@ -1772,12 +2902,12 @@ public class ForkJoinPool extends AbstractExecutorService { ...@@ -1772,12 +2902,12 @@ public class ForkJoinPool extends AbstractExecutorService {
*/ */
public long getQueuedTaskCount() { public long getQueuedTaskCount() {
long count = 0; long count = 0;
ForkJoinWorkerThread[] ws; WorkQueue[] ws; WorkQueue w;
if ((short)(ctl >>> TC_SHIFT) > -parallelism && if ((ws = workQueues) != null) {
(ws = workers) != null) { for (int i = 1; i < ws.length; i += 2) {
for (ForkJoinWorkerThread w : ws) if ((w = ws[i]) != null)
if (w != null) count += w.queueSize();
count -= w.queueBase - w.queueTop; // must read base first }
} }
return count; return count;
} }
...@@ -1790,7 +2920,15 @@ public class ForkJoinPool extends AbstractExecutorService { ...@@ -1790,7 +2920,15 @@ public class ForkJoinPool extends AbstractExecutorService {
* @return the number of queued submissions * @return the number of queued submissions
*/ */
public int getQueuedSubmissionCount() { public int getQueuedSubmissionCount() {
return -queueBase + queueTop; int count = 0;
WorkQueue[] ws; WorkQueue w;
if ((ws = workQueues) != null) {
for (int i = 0; i < ws.length; i += 2) {
if ((w = ws[i]) != null)
count += w.queueSize();
}
}
return count;
} }
/** /**
...@@ -1800,7 +2938,14 @@ public class ForkJoinPool extends AbstractExecutorService { ...@@ -1800,7 +2938,14 @@ 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 queueBase != queueTop; WorkQueue[] ws; WorkQueue w;
if ((ws = workQueues) != null) {
for (int i = 0; i < ws.length; i += 2) {
if ((w = ws[i]) != null && !w.isEmpty())
return true;
}
}
return false;
} }
/** /**
...@@ -1811,16 +2956,11 @@ public class ForkJoinPool extends AbstractExecutorService { ...@@ -1811,16 +2956,11 @@ 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() {
ForkJoinTask<?> t; ForkJoinTask<?>[] q; int b, i; WorkQueue[] ws; WorkQueue w; ForkJoinTask<?> t;
while ((b = queueBase) != queueTop && if ((ws = workQueues) != null) {
(q = submissionQueue) != null && for (int i = 0; i < ws.length; i += 2) {
(i = (q.length - 1) & b) >= 0) { if ((w = ws[i]) != null && (t = w.poll()) != null)
long u = (i << ASHIFT) + ABASE; return t;
if ((t = q[i]) != null &&
queueBase == b &&
UNSAFE.compareAndSwapObject(q, u, t, null)) {
queueBase = b + 1;
return t;
} }
} }
return null; return null;
...@@ -1845,20 +2985,17 @@ public class ForkJoinPool extends AbstractExecutorService { ...@@ -1845,20 +2985,17 @@ public class ForkJoinPool extends AbstractExecutorService {
*/ */
protected int drainTasksTo(Collection<? super ForkJoinTask<?>> c) { protected int drainTasksTo(Collection<? super ForkJoinTask<?>> c) {
int count = 0; int count = 0;
while (queueBase != queueTop) { WorkQueue[] ws; WorkQueue w; ForkJoinTask<?> t;
ForkJoinTask<?> t = pollSubmission(); if ((ws = workQueues) != null) {
if (t != null) { for (int i = 0; i < ws.length; ++i) {
c.add(t); if ((w = ws[i]) != null) {
++count; while ((t = w.poll()) != null) {
c.add(t);
++count;
}
}
} }
} }
ForkJoinWorkerThread[] ws;
if ((short)(ctl >>> TC_SHIFT) > -parallelism &&
(ws = workers) != null) {
for (ForkJoinWorkerThread w : ws)
if (w != null)
count += w.drainTasksTo(c);
}
return count; return count;
} }
...@@ -1870,21 +3007,36 @@ public class ForkJoinPool extends AbstractExecutorService { ...@@ -1870,21 +3007,36 @@ public class ForkJoinPool extends AbstractExecutorService {
* @return a string identifying this pool, as well as its state * @return a string identifying this pool, as well as its state
*/ */
public String toString() { public String toString() {
long st = getStealCount(); // Use a single pass through workQueues to collect counts
long qt = getQueuedTaskCount(); long qt = 0L, qs = 0L; int rc = 0;
long qs = getQueuedSubmissionCount(); long st = stealCount;
int pc = parallelism;
long c = ctl; long c = ctl;
WorkQueue[] ws; WorkQueue w;
if ((ws = workQueues) != null) {
for (int i = 0; i < ws.length; ++i) {
if ((w = ws[i]) != null) {
int size = w.queueSize();
if ((i & 1) == 0)
qs += size;
else {
qt += size;
st += w.nsteals;
if (w.isApparentlyUnblocked())
++rc;
}
}
}
}
int pc = (config & SMASK);
int tc = pc + (short)(c >>> TC_SHIFT); int tc = pc + (short)(c >>> TC_SHIFT);
int rc = pc + (int)(c >> AC_SHIFT); int ac = pc + (int)(c >> AC_SHIFT);
if (rc < 0) // ignore transient negative if (ac < 0) // ignore transient negative
rc = 0; ac = 0;
int ac = rc + blockedCount;
String level; String level;
if ((c & STOP_BIT) != 0) if ((c & STOP_BIT) != 0)
level = (tc == 0) ? "Terminated" : "Terminating"; level = (tc == 0) ? "Terminated" : "Terminating";
else else
level = shutdown ? "Shutting down" : "Running"; level = plock < 0 ? "Shutting down" : "Running";
return super.toString() + return super.toString() +
"[" + level + "[" + level +
", parallelism = " + pc + ", parallelism = " + pc +
...@@ -1898,11 +3050,13 @@ public class ForkJoinPool extends AbstractExecutorService { ...@@ -1898,11 +3050,13 @@ public class ForkJoinPool extends AbstractExecutorService {
} }
/** /**
* Initiates an orderly shutdown in which previously submitted * Possibly initiates an orderly shutdown in which previously
* tasks are executed, but no new tasks will be accepted. * submitted tasks are executed, but no new tasks will be
* Invocation has no additional effect if already shut down. * accepted. Invocation has no effect on execution state if this
* Tasks that are in the process of being submitted concurrently * is the {@link #commonPool}, and no additional effect if
* during the course of this method may or may not be rejected. * already shut down. Tasks that are in the process of being
* submitted concurrently during the course of this method may or
* may not be rejected.
* *
* @throws SecurityException if a security manager exists and * @throws SecurityException if a security manager exists and
* the caller is not permitted to modify threads * the caller is not permitted to modify threads
...@@ -1911,19 +3065,20 @@ public class ForkJoinPool extends AbstractExecutorService { ...@@ -1911,19 +3065,20 @@ public class ForkJoinPool extends AbstractExecutorService {
*/ */
public void shutdown() { public void shutdown() {
checkPermission(); checkPermission();
shutdown = true; tryTerminate(false, true);
tryTerminate(false);
} }
/** /**
* Attempts to cancel and/or stop all tasks, and reject all * Possibly attempts to cancel and/or stop all tasks, and reject
* subsequently submitted tasks. Tasks that are in the process of * all subsequently submitted tasks. Invocation has no effect on
* being submitted or executed concurrently during the course of * execution state if this is the {@link #commonPool}, and no
* this method may or may not be rejected. This method cancels * additional effect if already shut down. Otherwise, tasks that
* both existing and unexecuted tasks, in order to permit * are in the process of being submitted or executed concurrently
* termination in the presence of task dependencies. So the method * during the course of this method may or may not be
* always returns an empty list (unlike the case for some other * rejected. This method cancels both existing and unexecuted
* Executors). * tasks, in order to permit termination in the presence of task
* dependencies. So the method always returns an empty list
* (unlike the case for some other Executors).
* *
* @return an empty list * @return an empty list
* @throws SecurityException if a security manager exists and * @throws SecurityException if a security manager exists and
...@@ -1933,8 +3088,7 @@ public class ForkJoinPool extends AbstractExecutorService { ...@@ -1933,8 +3088,7 @@ public class ForkJoinPool extends AbstractExecutorService {
*/ */
public List<Runnable> shutdownNow() { public List<Runnable> shutdownNow() {
checkPermission(); checkPermission();
shutdown = true; tryTerminate(true, true);
tryTerminate(true);
return Collections.emptyList(); return Collections.emptyList();
} }
...@@ -1946,7 +3100,7 @@ public class ForkJoinPool extends AbstractExecutorService { ...@@ -1946,7 +3100,7 @@ public class ForkJoinPool extends AbstractExecutorService {
public boolean isTerminated() { public boolean isTerminated() {
long c = ctl; long c = ctl;
return ((c & STOP_BIT) != 0L && return ((c & STOP_BIT) != 0L &&
(short)(c >>> TC_SHIFT) == -parallelism); (short)(c >>> TC_SHIFT) == -(config & SMASK));
} }
/** /**
...@@ -1954,7 +3108,7 @@ public class ForkJoinPool extends AbstractExecutorService { ...@@ -1954,7 +3108,7 @@ public class ForkJoinPool extends AbstractExecutorService {
* commenced but not yet completed. This method may be useful for * commenced but not yet completed. This method may be useful for
* debugging. A return of {@code true} reported a sufficient * debugging. A return of {@code true} reported a sufficient
* period after shutdown may indicate that submitted tasks have * period after shutdown may indicate that submitted tasks have
* ignored or suppressed interruption, or are waiting for IO, * ignored or suppressed interruption, or are waiting for I/O,
* causing this executor not to properly terminate. (See the * causing this executor not to properly terminate. (See the
* advisory notes for class {@link ForkJoinTask} stating that * advisory notes for class {@link ForkJoinTask} stating that
* tasks should not normally entail blocking operations. But if * tasks should not normally entail blocking operations. But if
...@@ -1965,14 +3119,7 @@ public class ForkJoinPool extends AbstractExecutorService { ...@@ -1965,14 +3119,7 @@ public class ForkJoinPool extends AbstractExecutorService {
public boolean isTerminating() { public boolean isTerminating() {
long c = ctl; long c = ctl;
return ((c & STOP_BIT) != 0L && return ((c & STOP_BIT) != 0L &&
(short)(c >>> TC_SHIFT) != -parallelism); (short)(c >>> TC_SHIFT) != -(config & SMASK));
}
/**
* Returns true if terminating or terminated. Used by ForkJoinWorkerThread.
*/
final boolean isAtLeastTerminating() {
return (ctl & STOP_BIT) != 0L;
} }
/** /**
...@@ -1981,13 +3128,15 @@ public class ForkJoinPool extends AbstractExecutorService { ...@@ -1981,13 +3128,15 @@ 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 shutdown; return plock < 0;
} }
/** /**
* Blocks until all tasks have completed execution after a shutdown * Blocks until all tasks have completed execution after a
* request, or the timeout occurs, or the current thread is * shutdown request, or the timeout occurs, or the current thread
* interrupted, whichever happens first. * is interrupted, whichever happens first. Note that the {@link
* #commonPool()} never terminates until program shutdown so
* this method will always time out.
* *
* @param timeout the maximum time to wait * @param timeout the maximum time to wait
* @param unit the time unit of the timeout argument * @param unit the time unit of the timeout argument
...@@ -1998,19 +3147,21 @@ public class ForkJoinPool extends AbstractExecutorService { ...@@ -1998,19 +3147,21 @@ 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); long nanos = unit.toNanos(timeout);
final ReentrantLock lock = this.submissionLock; if (isTerminated())
lock.lock(); return true;
try { long startTime = System.nanoTime();
for (;;) { boolean terminated = false;
if (isTerminated()) synchronized (this) {
return true; for (long waitTime = nanos, millis = 0L;;) {
if (nanos <= 0) if (terminated = isTerminated() ||
return false; waitTime <= 0L ||
nanos = termination.awaitNanos(nanos); (millis = unit.toMillis(waitTime)) <= 0L)
break;
wait(millis);
waitTime = nanos - (System.nanoTime() - startTime);
} }
} finally {
lock.unlock();
} }
return terminated;
} }
/** /**
...@@ -2110,11 +3261,35 @@ public class ForkJoinPool extends AbstractExecutorService { ...@@ -2110,11 +3261,35 @@ public class ForkJoinPool extends AbstractExecutorService {
throws InterruptedException { throws InterruptedException {
Thread t = Thread.currentThread(); Thread t = Thread.currentThread();
if (t instanceof ForkJoinWorkerThread) { if (t instanceof ForkJoinWorkerThread) {
ForkJoinWorkerThread w = (ForkJoinWorkerThread) t; ForkJoinPool p = ((ForkJoinWorkerThread)t).pool;
w.pool.awaitBlocker(blocker); while (!blocker.isReleasable()) { // variant of helpSignal
WorkQueue[] ws; WorkQueue q; int m, u;
if ((ws = p.workQueues) != null && (m = ws.length - 1) >= 0) {
for (int i = 0; i <= m; ++i) {
if (blocker.isReleasable())
return;
if ((q = ws[i]) != null && q.base - q.top < 0) {
p.signalWork(q);
if ((u = (int)(p.ctl >>> 32)) >= 0 ||
(u >> UAC_SHIFT) >= 0)
break;
}
}
}
if (p.tryCompensate()) {
try {
do {} while (!blocker.isReleasable() &&
!blocker.block());
} finally {
p.incrementActiveCount();
}
break;
}
}
} }
else { else {
do {} while (!blocker.isReleasable() && !blocker.block()); do {} while (!blocker.isReleasable() &&
!blocker.block());
} }
} }
...@@ -2123,55 +3298,93 @@ public class ForkJoinPool extends AbstractExecutorService { ...@@ -2123,55 +3298,93 @@ public class ForkJoinPool extends AbstractExecutorService {
// implement RunnableFuture. // implement RunnableFuture.
protected <T> RunnableFuture<T> newTaskFor(Runnable runnable, T value) { protected <T> RunnableFuture<T> newTaskFor(Runnable runnable, T value) {
return (RunnableFuture<T>) ForkJoinTask.adapt(runnable, value); return new ForkJoinTask.AdaptedRunnable<T>(runnable, value);
} }
protected <T> RunnableFuture<T> newTaskFor(Callable<T> callable) { protected <T> RunnableFuture<T> newTaskFor(Callable<T> callable) {
return (RunnableFuture<T>) ForkJoinTask.adapt(callable); return new ForkJoinTask.AdaptedCallable<T>(callable);
} }
// Unsafe mechanics // Unsafe mechanics
private static final sun.misc.Unsafe UNSAFE; private static final sun.misc.Unsafe U;
private static final long ctlOffset; private static final long CTL;
private static final long stealCountOffset; private static final long PARKBLOCKER;
private static final long blockedCountOffset; private static final int ABASE;
private static final long quiescerCountOffset;
private static final long scanGuardOffset;
private static final long nextWorkerNumberOffset;
private static final long ABASE;
private static final int ASHIFT; private static final int ASHIFT;
private static final long STEALCOUNT;
private static final long PLOCK;
private static final long INDEXSEED;
private static final long QLOCK;
static { static {
poolNumberGenerator = new AtomicInteger(); int s; // initialize field offsets for CAS etc
workerSeedGenerator = new Random();
modifyThreadPermission = new RuntimePermission("modifyThread");
defaultForkJoinWorkerThreadFactory =
new DefaultForkJoinWorkerThreadFactory();
int s;
try { try {
UNSAFE = sun.misc.Unsafe.getUnsafe(); U = sun.misc.Unsafe.getUnsafe();
Class<?> k = ForkJoinPool.class; Class<?> k = ForkJoinPool.class;
ctlOffset = UNSAFE.objectFieldOffset CTL = U.objectFieldOffset
(k.getDeclaredField("ctl")); (k.getDeclaredField("ctl"));
stealCountOffset = UNSAFE.objectFieldOffset STEALCOUNT = U.objectFieldOffset
(k.getDeclaredField("stealCount")); (k.getDeclaredField("stealCount"));
blockedCountOffset = UNSAFE.objectFieldOffset PLOCK = U.objectFieldOffset
(k.getDeclaredField("blockedCount")); (k.getDeclaredField("plock"));
quiescerCountOffset = UNSAFE.objectFieldOffset INDEXSEED = U.objectFieldOffset
(k.getDeclaredField("quiescerCount")); (k.getDeclaredField("indexSeed"));
scanGuardOffset = UNSAFE.objectFieldOffset Class<?> tk = Thread.class;
(k.getDeclaredField("scanGuard")); PARKBLOCKER = U.objectFieldOffset
nextWorkerNumberOffset = UNSAFE.objectFieldOffset (tk.getDeclaredField("parkBlocker"));
(k.getDeclaredField("nextWorkerNumber")); Class<?> wk = WorkQueue.class;
Class<?> a = ForkJoinTask[].class; QLOCK = U.objectFieldOffset
ABASE = UNSAFE.arrayBaseOffset(a); (wk.getDeclaredField("qlock"));
s = UNSAFE.arrayIndexScale(a); Class<?> ak = ForkJoinTask[].class;
ABASE = U.arrayBaseOffset(ak);
s = U.arrayIndexScale(ak);
ASHIFT = 31 - Integer.numberOfLeadingZeros(s);
} catch (Exception e) { } catch (Exception e) {
throw new Error(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");
ASHIFT = 31 - Integer.numberOfLeadingZeros(s);
submitters = new ThreadLocal<Submitter>();
ForkJoinWorkerThreadFactory fac = defaultForkJoinWorkerThreadFactory =
new DefaultForkJoinWorkerThreadFactory();
modifyThreadPermission = new RuntimePermission("modifyThread");
/*
* Establish common pool parameters. For extra caution,
* computations to set up common pool state are here; the
* constructor just assigns these values to fields.
*/
int par = 0;
Thread.UncaughtExceptionHandler handler = null;
try { // TBD: limit or report ignored exceptions?
String pp = System.getProperty
("java.util.concurrent.ForkJoinPool.common.parallelism");
String hp = System.getProperty
("java.util.concurrent.ForkJoinPool.common.exceptionHandler");
String fp = System.getProperty
("java.util.concurrent.ForkJoinPool.common.threadFactory");
if (fp != null)
fac = ((ForkJoinWorkerThreadFactory)ClassLoader.
getSystemClassLoader().loadClass(fp).newInstance());
if (hp != null)
handler = ((Thread.UncaughtExceptionHandler)ClassLoader.
getSystemClassLoader().loadClass(hp).newInstance());
if (pp != null)
par = Integer.parseInt(pp);
} catch (Exception ignore) {
}
if (par <= 0)
par = Runtime.getRuntime().availableProcessors();
if (par > MAX_CAP)
par = MAX_CAP;
commonPoolParallelism = par;
long np = (long)(-par); // precompute initial ctl value
long ct = ((np << AC_SHIFT) & AC_MASK) | ((np << TC_SHIFT) & TC_MASK);
commonPool = new ForkJoinPool(par, ct, fac, handler);
} }
} }
...@@ -37,17 +37,13 @@ package java.util.concurrent; ...@@ -37,17 +37,13 @@ package java.util.concurrent;
import java.io.Serializable; import java.io.Serializable;
import java.util.Collection; import java.util.Collection;
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.lang.ref.WeakReference; import java.lang.ref.WeakReference;
import java.lang.ref.ReferenceQueue; 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;
import java.util.concurrent.Executor;
import java.util.concurrent.ExecutorService;
import java.util.concurrent.Future; import java.util.concurrent.Future;
import java.util.concurrent.RejectedExecutionException; import java.util.concurrent.RejectedExecutionException;
import java.util.concurrent.RunnableFuture; import java.util.concurrent.RunnableFuture;
...@@ -63,46 +59,59 @@ import java.lang.reflect.Constructor; ...@@ -63,46 +59,59 @@ import java.lang.reflect.Constructor;
* subtasks may be hosted by a small number of actual threads in a * subtasks may be hosted by a small number of actual threads in a
* ForkJoinPool, at the price of some usage limitations. * ForkJoinPool, at the price of some usage limitations.
* *
* <p>A "main" {@code ForkJoinTask} begins execution when submitted * <p>A "main" {@code ForkJoinTask} begins execution when it is
* to a {@link ForkJoinPool}. Once started, it will usually in turn * explicitly submitted to a {@link ForkJoinPool}, or, if not already
* start other subtasks. As indicated by the name of this class, * engaged in a ForkJoin computation, commenced in the {@link
* many programs using {@code ForkJoinTask} employ only methods * ForkJoinPool#commonPool()} via {@link #fork}, {@link #invoke}, or
* {@link #fork} and {@link #join}, or derivatives such as {@link * related methods. Once started, it will usually in turn start other
* subtasks. As indicated by the name of this class, many programs
* using {@code ForkJoinTask} employ only methods {@link #fork} and
* {@link #join}, or derivatives such as {@link
* #invokeAll(ForkJoinTask...) invokeAll}. However, this class also * #invokeAll(ForkJoinTask...) invokeAll}. However, this class also
* provides a number of other methods that can come into play in * provides a number of other methods that can come into play in
* advanced usages, as well as extension mechanics that allow * advanced usages, as well as extension mechanics that allow support
* support of new forms of fork/join processing. * of new forms of fork/join processing.
* *
* <p>A {@code ForkJoinTask} is a lightweight form of {@link Future}. * <p>A {@code ForkJoinTask} is a lightweight form of {@link Future}.
* The efficiency of {@code ForkJoinTask}s stems from a set of * The efficiency of {@code ForkJoinTask}s stems from a set of
* restrictions (that are only partially statically enforceable) * restrictions (that are only partially statically enforceable)
* reflecting their intended use as computational tasks calculating * reflecting their main use as computational tasks calculating pure
* pure functions or operating on purely isolated objects. The * functions or operating on purely isolated objects. The primary
* primary coordination mechanisms are {@link #fork}, that arranges * coordination mechanisms are {@link #fork}, that arranges
* asynchronous execution, and {@link #join}, that doesn't proceed * asynchronous execution, and {@link #join}, that doesn't proceed
* until the task's result has been computed. Computations should * until the task's result has been computed. Computations should
* avoid {@code synchronized} methods or blocks, and should minimize * ideally avoid {@code synchronized} methods or blocks, and should
* other blocking synchronization apart from joining other tasks or * minimize other blocking synchronization apart from joining other
* using synchronizers such as Phasers that are advertised to * tasks or using synchronizers such as Phasers that are advertised to
* cooperate with fork/join scheduling. Tasks should also not perform * cooperate with fork/join scheduling. Subdividable tasks should also
* blocking IO, and should ideally access variables that are * not perform blocking I/O, and should ideally access variables that
* completely independent of those accessed by other running * are completely independent of those accessed by other running
* tasks. Minor breaches of these restrictions, for example using * tasks. These guidelines are loosely enforced by not permitting
* shared output streams, may be tolerable in practice, but frequent * checked exceptions such as {@code IOExceptions} to be
* use may result in poor performance, and the potential to * thrown. However, computations may still encounter unchecked
* indefinitely stall if the number of threads not waiting for IO or * exceptions, that are rethrown to callers attempting to join
* other external synchronization becomes exhausted. This usage * them. These exceptions may additionally include {@link
* restriction is in part enforced by not permitting checked * RejectedExecutionException} stemming from internal resource
* exceptions such as {@code IOExceptions} to be thrown. However, * exhaustion, such as failure to allocate internal task
* computations may still encounter unchecked exceptions, that are * queues. Rethrown exceptions behave in the same way as regular
* rethrown to callers attempting to join them. These exceptions may * exceptions, but, when possible, contain stack traces (as displayed
* additionally include {@link RejectedExecutionException} stemming * for example using {@code ex.printStackTrace()}) of both the thread
* from internal resource exhaustion, such as failure to allocate * that initiated the computation as well as the thread actually
* internal task queues. Rethrown exceptions behave in the same way as * encountering the exception; minimally only the latter.
* regular exceptions, but, when possible, contain stack traces (as *
* displayed for example using {@code ex.printStackTrace()}) of both * <p>It is possible to define and use ForkJoinTasks that may block,
* the thread that initiated the computation as well as the thread * but doing do requires three further considerations: (1) Completion
* actually encountering the exception; minimally only the latter. * of few if any <em>other</em> tasks should be dependent on a task
* that blocks on external synchronization or I/O. Event-style async
* tasks that are never joined (for example, those subclassing {@link
* CountedCompleter}) often fall into this category. (2) To minimize
* resource impact, tasks should be small; ideally performing only the
* (possibly) blocking action. (3) Unless the {@link
* ForkJoinPool.ManagedBlocker} API is used, or the number of possibly
* blocked tasks is known to be less than the pool's {@link
* ForkJoinPool#getParallelism} level, the pool cannot guarantee that
* enough threads will be available to ensure progress or good
* performance.
* *
* <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:
...@@ -118,6 +127,13 @@ import java.lang.reflect.Constructor; ...@@ -118,6 +127,13 @@ import java.lang.reflect.Constructor;
* performs the most common form of parallel invocation: forking a set * performs the most common form of parallel invocation: forking a set
* of tasks and joining them all. * of tasks and joining them all.
* *
* <p>In the most typical usages, a fork-join pair act like a call
* (fork) and return (join) from a parallel recursive function. As is
* the case with other forms of recursive calls, returns (joins)
* should be performed innermost-first. For example, {@code a.fork();
* b.fork(); b.join(); a.join();} is likely to be substantially more
* efficient than joining {@code a} before {@code b}.
*
* <p>The execution status of tasks may be queried at several levels * <p>The execution status of tasks may be queried at several levels
* of detail: {@link #isDone} is true if a task completed in any way * of detail: {@link #isDone} is true if a task completed in any way
* (including the case where a task was cancelled without executing); * (including the case where a task was cancelled without executing);
...@@ -133,18 +149,13 @@ import java.lang.reflect.Constructor; ...@@ -133,18 +149,13 @@ import java.lang.reflect.Constructor;
* <p>The ForkJoinTask class is not usually directly subclassed. * <p>The ForkJoinTask class is not usually directly subclassed.
* Instead, you subclass one of the abstract classes that support a * Instead, you subclass one of the abstract classes that support a
* particular style of fork/join processing, typically {@link * particular style of fork/join processing, typically {@link
* RecursiveAction} for computations that do not return results, or * RecursiveAction} for most computations that do not return results,
* {@link RecursiveTask} for those that do. Normally, a concrete * {@link RecursiveTask} for those that do, and {@link
* ForkJoinTask subclass declares fields comprising its parameters, * CountedCompleter} for those in which completed actions trigger
* established in a constructor, and then defines a {@code compute} * other actions. Normally, a concrete ForkJoinTask subclass declares
* method that somehow uses the control methods supplied by this base * fields comprising its parameters, established in a constructor, and
* class. While these methods have {@code public} access (to allow * then defines a {@code compute} method that somehow uses the control
* instances of different task subclasses to call each other's * methods supplied by this base class.
* methods), some of them may only be called from within other
* ForkJoinTasks (as may be determined using method {@link
* #inForkJoinPool}). Attempts to invoke them in other contexts
* result in exceptions or errors, possibly including
* {@code ClassCastException}.
* *
* <p>Method {@link #join} and its variants are appropriate for use * <p>Method {@link #join} and its variants are appropriate for use
* only when completion dependencies are acyclic; that is, the * only when completion dependencies are acyclic; that is, the
...@@ -154,7 +165,17 @@ import java.lang.reflect.Constructor; ...@@ -154,7 +165,17 @@ import java.lang.reflect.Constructor;
* supports other methods and techniques (for example the use of * supports other methods and techniques (for example the use of
* {@link Phaser}, {@link #helpQuiesce}, and {@link #complete}) that * {@link Phaser}, {@link #helpQuiesce}, and {@link #complete}) that
* may be of use in constructing custom subclasses for problems that * may be of use in constructing custom subclasses for problems that
* are not statically structured as DAGs. * are not statically structured as DAGs. To support such usages a
* ForkJoinTask may be atomically <em>tagged</em> with a {@code short}
* value using {@link #setForkJoinTaskTag} or {@link
* #compareAndSetForkJoinTaskTag} and checked using {@link
* #getForkJoinTaskTag}. The ForkJoinTask implementation does not use
* these {@code protected} methods or tags for any purpose, but they
* may be of use in the construction of specialized subclasses. For
* example, parallel graph traversals can use the supplied methods to
* avoid revisiting nodes/tasks that have already been processed.
* (Method names for tagging are bulky in part to encourage definition
* of methods that reflect their usage patterns.)
* *
* <p>Most base support methods are {@code final}, to prevent * <p>Most base support methods are {@code final}, to prevent
* overriding of implementations that are intrinsically tied to the * overriding of implementations that are intrinsically tied to the
...@@ -194,41 +215,50 @@ public abstract class ForkJoinTask<V> implements Future<V>, Serializable { ...@@ -194,41 +215,50 @@ public abstract class ForkJoinTask<V> implements Future<V>, Serializable {
* See the internal documentation of class ForkJoinPool for a * See the internal documentation of class ForkJoinPool for a
* general implementation overview. ForkJoinTasks are mainly * general implementation overview. ForkJoinTasks are mainly
* responsible for maintaining their "status" field amidst relays * responsible for maintaining their "status" field amidst relays
* to methods in ForkJoinWorkerThread and ForkJoinPool. The * to methods in ForkJoinWorkerThread and ForkJoinPool.
* methods of this class are more-or-less layered into (1) basic *
* status maintenance (2) execution and awaiting completion (3) * The methods of this class are more-or-less layered into
* user-level methods that additionally report results. This is * (1) basic status maintenance
* sometimes hard to see because this file orders exported methods * (2) execution and awaiting completion
* in a way that flows well in javadocs. * (3) user-level methods that additionally report results.
* This is sometimes hard to see because this file orders exported
* methods in a way that flows well in javadocs.
*/ */
/* /*
* The status field holds run control status bits packed into a * The status field holds run control status bits packed into a
* single int to minimize footprint and to ensure atomicity (via * single int to minimize footprint and to ensure atomicity (via
* CAS). Status is initially zero, and takes on nonnegative * CAS). Status is initially zero, and takes on nonnegative
* values until completed, upon which status holds value * values until completed, upon which status (anded with
* NORMAL, CANCELLED, or EXCEPTIONAL. Tasks undergoing blocking * DONE_MASK) holds value NORMAL, CANCELLED, or EXCEPTIONAL. Tasks
* waits by other threads have the SIGNAL bit set. Completion of * undergoing blocking waits by other threads have the SIGNAL bit
* a stolen task with SIGNAL set awakens any waiters via * set. Completion of a stolen task with SIGNAL set awakens any
* notifyAll. Even though suboptimal for some purposes, we use * waiters via notifyAll. Even though suboptimal for some
* basic builtin wait/notify to take advantage of "monitor * purposes, we use basic builtin wait/notify to take advantage of
* inflation" in JVMs that we would otherwise need to emulate to * "monitor inflation" in JVMs that we would otherwise need to
* avoid adding further per-task bookkeeping overhead. We want * emulate to avoid adding further per-task bookkeeping overhead.
* these monitors to be "fat", i.e., not use biasing or thin-lock * We want these monitors to be "fat", i.e., not use biasing or
* techniques, so use some odd coding idioms that tend to avoid * thin-lock techniques, so use some odd coding idioms that tend
* them. * to avoid them, mainly by arranging that every synchronized
* block performs a wait, notifyAll or both.
*
* These control bits occupy only (some of) the upper half (16
* bits) of status field. The lower bits are used for user-defined
* tags.
*/ */
/** 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; static final int DONE_MASK = 0xf0000000; // mask out non-completion bits
private static final int CANCELLED = -2; static final int NORMAL = 0xf0000000; // must be negative
private static final int EXCEPTIONAL = -3; static final int CANCELLED = 0xc0000000; // must be < NORMAL
private static final int SIGNAL = 1; static final int EXCEPTIONAL = 0x80000000; // must be < CANCELLED
static final int SIGNAL = 0x00010000; // must be >= 1 << 16
static final int SMASK = 0x0000ffff; // short bits for tags
/** /**
* Marks completion and wakes up threads waiting to join this task, * Marks completion and wakes up threads waiting to join this
* also clearing signal request bits. * task.
* *
* @param completion one of NORMAL, CANCELLED, EXCEPTIONAL * @param completion one of NORMAL, CANCELLED, EXCEPTIONAL
* @return completion status on exit * @return completion status on exit
...@@ -237,8 +267,8 @@ public abstract class ForkJoinTask<V> implements Future<V>, Serializable { ...@@ -237,8 +267,8 @@ public abstract class ForkJoinTask<V> implements Future<V>, Serializable {
for (int s;;) { for (int s;;) {
if ((s = status) < 0) if ((s = status) < 0)
return s; return s;
if (UNSAFE.compareAndSwapInt(this, statusOffset, s, completion)) { if (U.compareAndSwapInt(this, STATUS, s, s | completion)) {
if (s != 0) if ((s >>> 16) != 0)
synchronized (this) { notifyAll(); } synchronized (this) { notifyAll(); }
return completion; return completion;
} }
...@@ -246,27 +276,36 @@ public abstract class ForkJoinTask<V> implements Future<V>, Serializable { ...@@ -246,27 +276,36 @@ public abstract class ForkJoinTask<V> implements Future<V>, Serializable {
} }
/** /**
* Tries to block a worker thread until completed or timed out. * Primary execution method for stolen tasks. Unless done, calls
* Uses Object.wait time argument conventions. * exec and records status if completed, but doesn't wait for
* May fail on contention or interrupt. * completion otherwise.
* *
* @param millis if > 0, wait time. * @return status on exit from this method
*/ */
final void tryAwaitDone(long millis) { final int doExec() {
int s; int s; boolean completed;
try { if ((s = status) >= 0) {
if (((s = status) > 0 || try {
(s == 0 && completed = exec();
UNSAFE.compareAndSwapInt(this, statusOffset, 0, SIGNAL))) && } catch (Throwable rex) {
status > 0) { return setExceptionalCompletion(rex);
synchronized (this) {
if (status > 0)
wait(millis);
}
} }
} catch (InterruptedException ie) { if (completed)
// caller must check termination s = setCompletion(NORMAL);
} }
return s;
}
/**
* Tries to set SIGNAL status unless already completed. Used by
* ForkJoinPool. Other variants are directly incorporated into
* externalAwaitDone etc.
*
* @return true if successful
*/
final boolean trySetSignal() {
int s = status;
return s >= 0 && U.compareAndSwapInt(this, STATUS, s, s | SIGNAL);
} }
/** /**
...@@ -275,113 +314,78 @@ public abstract class ForkJoinTask<V> implements Future<V>, Serializable { ...@@ -275,113 +314,78 @@ public abstract class ForkJoinTask<V> implements Future<V>, Serializable {
*/ */
private int externalAwaitDone() { private int externalAwaitDone() {
int s; int s;
if ((s = status) >= 0) { ForkJoinPool.externalHelpJoin(this);
boolean interrupted = false; boolean interrupted = false;
synchronized (this) { while ((s = status) >= 0) {
while ((s = status) >= 0) { if (U.compareAndSwapInt(this, STATUS, s, s | SIGNAL)) {
if (s == 0) synchronized (this) {
UNSAFE.compareAndSwapInt(this, statusOffset, if (status >= 0) {
0, SIGNAL);
else {
try { try {
wait(); wait();
} catch (InterruptedException ie) { } catch (InterruptedException ie) {
interrupted = true; interrupted = true;
} }
} }
else
notifyAll();
} }
} }
if (interrupted)
Thread.currentThread().interrupt();
} }
if (interrupted)
Thread.currentThread().interrupt();
return s; return s;
} }
/** /**
* Blocks a non-worker-thread until completion or interruption or timeout. * Blocks a non-worker-thread until completion or interruption.
*/ */
private int externalInterruptibleAwaitDone(long millis) private int externalInterruptibleAwaitDone() throws InterruptedException {
throws InterruptedException {
int s; int s;
if (Thread.interrupted()) if (Thread.interrupted())
throw new InterruptedException(); throw new InterruptedException();
if ((s = status) >= 0) { ForkJoinPool.externalHelpJoin(this);
synchronized (this) { while ((s = status) >= 0) {
while ((s = status) >= 0) { if (U.compareAndSwapInt(this, STATUS, s, s | SIGNAL)) {
if (s == 0) synchronized (this) {
UNSAFE.compareAndSwapInt(this, statusOffset, if (status >= 0)
0, SIGNAL); wait();
else { else
wait(millis); notifyAll();
if (millis > 0L)
break;
}
} }
} }
} }
return s; return s;
} }
/**
* Primary execution method for stolen tasks. Unless done, calls
* exec and records status if completed, but doesn't wait for
* completion otherwise.
*/
final void doExec() {
if (status >= 0) {
boolean completed;
try {
completed = exec();
} catch (Throwable rex) {
setExceptionalCompletion(rex);
return;
}
if (completed)
setCompletion(NORMAL); // must be outside try block
}
}
/** /**
* Primary mechanics for join, get, quietlyJoin. * Implementation for join, get, quietlyJoin. Directly handles
* only cases of already-completed, external wait, and
* unfork+exec. Others are relayed to ForkJoinPool.awaitJoin.
*
* @return status upon completion * @return status upon completion
*/ */
private int doJoin() { private int doJoin() {
Thread t; ForkJoinWorkerThread w; int s; boolean completed; int s; Thread t; ForkJoinWorkerThread wt; ForkJoinPool.WorkQueue w;
if ((t = Thread.currentThread()) instanceof ForkJoinWorkerThread) { return (s = status) < 0 ? s :
if ((s = status) < 0) ((t = Thread.currentThread()) instanceof ForkJoinWorkerThread) ?
return s; (w = (wt = (ForkJoinWorkerThread)t).workQueue).
if ((w = (ForkJoinWorkerThread)t).unpushTask(this)) { tryUnpush(this) && (s = doExec()) < 0 ? s :
try { wt.pool.awaitJoin(w, this) :
completed = exec(); externalAwaitDone();
} catch (Throwable rex) {
return setExceptionalCompletion(rex);
}
if (completed)
return setCompletion(NORMAL);
}
return w.joinTask(this);
}
else
return externalAwaitDone();
} }
/** /**
* Primary mechanics for invoke, quietlyInvoke. * Implementation for invoke, quietlyInvoke.
*
* @return status upon completion * @return status upon completion
*/ */
private int doInvoke() { private int doInvoke() {
int s; boolean completed; int s; Thread t; ForkJoinWorkerThread wt;
if ((s = status) < 0) return (s = doExec()) < 0 ? s :
return s; ((t = Thread.currentThread()) instanceof ForkJoinWorkerThread) ?
try { (wt = (ForkJoinWorkerThread)t).pool.awaitJoin(wt.workQueue, this) :
completed = exec(); externalAwaitDone();
} catch (Throwable rex) {
return setExceptionalCompletion(rex);
}
if (completed)
return setCompletion(NORMAL);
else
return doJoin();
} }
// Exception table support // Exception table support
...@@ -416,7 +420,7 @@ public abstract class ForkJoinTask<V> implements Future<V>, Serializable { ...@@ -416,7 +420,7 @@ public abstract class ForkJoinTask<V> implements Future<V>, Serializable {
* any ForkJoinPool will call helpExpungeStaleExceptions when its * any ForkJoinPool will call helpExpungeStaleExceptions when its
* pool becomes isQuiescent. * pool becomes isQuiescent.
*/ */
static final class ExceptionNode extends WeakReference<ForkJoinTask<?>>{ static final class ExceptionNode extends WeakReference<ForkJoinTask<?>> {
final Throwable ex; final Throwable ex;
ExceptionNode next; ExceptionNode next;
final long thrower; // use id not ref to avoid weak cycles final long thrower; // use id not ref to avoid weak cycles
...@@ -429,30 +433,67 @@ public abstract class ForkJoinTask<V> implements Future<V>, Serializable { ...@@ -429,30 +433,67 @@ public abstract class ForkJoinTask<V> implements Future<V>, Serializable {
} }
/** /**
* Records exception and sets exceptional completion. * Records exception and sets status.
* *
* @return status on exit * @return status on exit
*/ */
private int setExceptionalCompletion(Throwable ex) { final int recordExceptionalCompletion(Throwable ex) {
int h = System.identityHashCode(this); int s;
final ReentrantLock lock = exceptionTableLock; if ((s = status) >= 0) {
lock.lock(); int h = System.identityHashCode(this);
try { final ReentrantLock lock = exceptionTableLock;
expungeStaleExceptions(); lock.lock();
ExceptionNode[] t = exceptionTable; try {
int i = h & (t.length - 1); expungeStaleExceptions();
for (ExceptionNode e = t[i]; ; e = e.next) { ExceptionNode[] t = exceptionTable;
if (e == null) { int i = h & (t.length - 1);
t[i] = new ExceptionNode(this, ex, t[i]); for (ExceptionNode e = t[i]; ; e = e.next) {
break; if (e == null) {
t[i] = new ExceptionNode(this, ex, t[i]);
break;
}
if (e.get() == this) // already present
break;
} }
if (e.get() == this) // already present } finally {
break; lock.unlock();
}
s = setCompletion(EXCEPTIONAL);
}
return s;
}
/**
* Records exception and possibly propagates
*
* @return status on exit
*/
private int setExceptionalCompletion(Throwable ex) {
int s = recordExceptionalCompletion(ex);
if ((s & DONE_MASK) == EXCEPTIONAL)
internalPropagateException(ex);
return s;
}
/**
* Hook for exception propagation support for tasks with completers.
*/
void internalPropagateException(Throwable ex) {
}
/**
* Cancels, ignoring any exceptions thrown by cancel. Used during
* worker and pool shutdown. Cancel is spec'ed not to throw any
* exceptions, but if it does anyway, we have no recourse during
* shutdown, so guard against this case.
*/
static final void cancelIgnoringExceptions(ForkJoinTask<?> t) {
if (t != null && t.status >= 0) {
try {
t.cancel(false);
} catch (Throwable ignore) {
} }
} finally {
lock.unlock();
} }
return setCompletion(EXCEPTIONAL);
} }
/** /**
...@@ -501,7 +542,7 @@ public abstract class ForkJoinTask<V> implements Future<V>, Serializable { ...@@ -501,7 +542,7 @@ public abstract class ForkJoinTask<V> implements Future<V>, Serializable {
* @return the exception, or null if none * @return the exception, or null if none
*/ */
private Throwable getThrowableException() { private Throwable getThrowableException() {
if (status != EXCEPTIONAL) if ((status & DONE_MASK) != EXCEPTIONAL)
return null; return null;
int h = System.identityHashCode(this); int h = System.identityHashCode(this);
ExceptionNode e; ExceptionNode e;
...@@ -519,7 +560,7 @@ public abstract class ForkJoinTask<V> implements Future<V>, Serializable { ...@@ -519,7 +560,7 @@ public abstract class ForkJoinTask<V> implements Future<V>, Serializable {
Throwable ex; Throwable ex;
if (e == null || (ex = e.ex) == null) if (e == null || (ex = e.ex) == null)
return null; return null;
if (e.thrower != Thread.currentThread().getId()) { if (false && e.thrower != Thread.currentThread().getId()) {
Class<? extends Throwable> ec = ex.getClass(); Class<? extends Throwable> ec = ex.getClass();
try { try {
Constructor<?> noArgCtor = null; Constructor<?> noArgCtor = null;
...@@ -586,41 +627,61 @@ public abstract class ForkJoinTask<V> implements Future<V>, Serializable { ...@@ -586,41 +627,61 @@ public abstract class ForkJoinTask<V> implements Future<V>, Serializable {
} }
/** /**
* Report the result of invoke or join; called only upon * A version of "sneaky throw" to relay exceptions
* non-normal return of internal versions. */
static void rethrow(final Throwable ex) {
if (ex != null) {
if (ex instanceof Error)
throw (Error)ex;
if (ex instanceof RuntimeException)
throw (RuntimeException)ex;
throw uncheckedThrowable(ex, RuntimeException.class);
}
}
/**
* The sneaky part of sneaky throw, relying on generics
* limitations to evade compiler complaints about rethrowing
* unchecked exceptions
*/ */
private V reportResult() { @SuppressWarnings("unchecked") static <T extends Throwable>
int s; Throwable ex; T uncheckedThrowable(final Throwable t, final Class<T> c) {
if ((s = status) == CANCELLED) return (T)t; // rely on vacuous cast
}
/**
* Throws exception, if any, associated with the given status.
*/
private void reportException(int s) {
if (s == CANCELLED)
throw new CancellationException(); throw new CancellationException();
if (s == EXCEPTIONAL && (ex = getThrowableException()) != null) if (s == EXCEPTIONAL)
UNSAFE.throwException(ex); rethrow(getThrowableException());
return getRawResult();
} }
// public methods // public methods
/** /**
* Arranges to asynchronously execute this task. While it is not * Arranges to asynchronously execute this task in the pool the
* necessarily enforced, it is a usage error to fork a task more * current task is running in, if applicable, or using the {@link
* than once unless it has completed and been reinitialized. * ForkJoinPool#commonPool()} if not {@link #inForkJoinPool}. While
* Subsequent modifications to the state of this task or any data * it is not necessarily enforced, it is a usage error to fork a
* it operates on are not necessarily consistently observable by * task more than once unless it has completed and been
* any thread other than the one executing it unless preceded by a * reinitialized. Subsequent modifications to the state of this
* call to {@link #join} or related methods, or a call to {@link * task or any data it operates on are not necessarily
* #isDone} returning {@code true}. * consistently observable by any thread other than the one
* * executing it unless preceded by a call to {@link #join} or
* <p>This method may be invoked only from within {@code * related methods, or a call to {@link #isDone} returning {@code
* ForkJoinPool} computations (as may be determined using method * true}.
* {@link #inForkJoinPool}). Attempts to invoke in other contexts
* result in exceptions or errors, possibly including {@code
* ClassCastException}.
* *
* @return {@code this}, to simplify usage * @return {@code this}, to simplify usage
*/ */
public final ForkJoinTask<V> fork() { public final ForkJoinTask<V> fork() {
((ForkJoinWorkerThread) Thread.currentThread()) Thread t;
.pushTask(this); if ((t = Thread.currentThread()) instanceof ForkJoinWorkerThread)
((ForkJoinWorkerThread)t).workQueue.push(this);
else
ForkJoinPool.commonPool.externalPush(this);
return this; return this;
} }
...@@ -636,10 +697,10 @@ public abstract class ForkJoinTask<V> implements Future<V>, Serializable { ...@@ -636,10 +697,10 @@ 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() {
if (doJoin() != NORMAL) int s;
return reportResult(); if ((s = doJoin() & DONE_MASK) != NORMAL)
else reportException(s);
return getRawResult(); return getRawResult();
} }
/** /**
...@@ -651,10 +712,10 @@ public abstract class ForkJoinTask<V> implements Future<V>, Serializable { ...@@ -651,10 +712,10 @@ 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() {
if (doInvoke() != NORMAL) int s;
return reportResult(); if ((s = doInvoke() & DONE_MASK) != NORMAL)
else reportException(s);
return getRawResult(); return getRawResult();
} }
/** /**
...@@ -670,20 +731,17 @@ public abstract class ForkJoinTask<V> implements Future<V>, Serializable { ...@@ -670,20 +731,17 @@ public abstract class ForkJoinTask<V> implements Future<V>, Serializable {
* cancelled, completed normally or exceptionally, or left * cancelled, completed normally or exceptionally, or left
* unprocessed. * unprocessed.
* *
* <p>This method may be invoked only from within {@code
* ForkJoinPool} computations (as may be determined using method
* {@link #inForkJoinPool}). Attempts to invoke in other contexts
* result in exceptions or errors, possibly including {@code
* ClassCastException}.
*
* @param t1 the first task * @param t1 the first task
* @param t2 the second task * @param t2 the second task
* @throws NullPointerException if any task is null * @throws NullPointerException if any task is null
*/ */
public static void invokeAll(ForkJoinTask<?> t1, ForkJoinTask<?> t2) { public static void invokeAll(ForkJoinTask<?> t1, ForkJoinTask<?> t2) {
int s1, s2;
t2.fork(); t2.fork();
t1.invoke(); if ((s1 = t1.doInvoke() & DONE_MASK) != NORMAL)
t2.join(); t1.reportException(s1);
if ((s2 = t2.doJoin() & DONE_MASK) != NORMAL)
t2.reportException(s2);
} }
/** /**
...@@ -698,12 +756,6 @@ public abstract class ForkJoinTask<V> implements Future<V>, Serializable { ...@@ -698,12 +756,6 @@ public abstract class ForkJoinTask<V> implements Future<V>, Serializable {
* related methods to check if they have been cancelled, completed * related methods to check if they have been cancelled, completed
* normally or exceptionally, or left unprocessed. * normally or exceptionally, or left unprocessed.
* *
* <p>This method may be invoked only from within {@code
* ForkJoinPool} computations (as may be determined using method
* {@link #inForkJoinPool}). Attempts to invoke in other contexts
* result in exceptions or errors, possibly including {@code
* ClassCastException}.
*
* @param tasks the tasks * @param tasks the tasks
* @throws NullPointerException if any task is null * @throws NullPointerException if any task is null
*/ */
...@@ -726,12 +778,12 @@ public abstract class ForkJoinTask<V> implements Future<V>, Serializable { ...@@ -726,12 +778,12 @@ public abstract class ForkJoinTask<V> implements Future<V>, Serializable {
if (t != null) { if (t != null) {
if (ex != null) if (ex != null)
t.cancel(false); t.cancel(false);
else if (t.doJoin() < NORMAL && ex == null) else if (t.doJoin() < NORMAL)
ex = t.getException(); ex = t.getException();
} }
} }
if (ex != null) if (ex != null)
UNSAFE.throwException(ex); rethrow(ex);
} }
/** /**
...@@ -747,12 +799,6 @@ public abstract class ForkJoinTask<V> implements Future<V>, Serializable { ...@@ -747,12 +799,6 @@ public abstract class ForkJoinTask<V> implements Future<V>, Serializable {
* cancelled, completed normally or exceptionally, or left * cancelled, completed normally or exceptionally, or left
* unprocessed. * unprocessed.
* *
* <p>This method may be invoked only from within {@code
* ForkJoinPool} computations (as may be determined using method
* {@link #inForkJoinPool}). Attempts to invoke in other contexts
* result in exceptions or errors, possibly including {@code
* ClassCastException}.
*
* @param tasks the collection of tasks * @param tasks the collection of tasks
* @return the tasks argument, to simplify usage * @return the tasks argument, to simplify usage
* @throws NullPointerException if tasks or any element are null * @throws NullPointerException if tasks or any element are null
...@@ -783,12 +829,12 @@ public abstract class ForkJoinTask<V> implements Future<V>, Serializable { ...@@ -783,12 +829,12 @@ public abstract class ForkJoinTask<V> implements Future<V>, Serializable {
if (t != null) { if (t != null) {
if (ex != null) if (ex != null)
t.cancel(false); t.cancel(false);
else if (t.doJoin() < NORMAL && ex == null) else if (t.doJoin() < NORMAL)
ex = t.getException(); ex = t.getException();
} }
} }
if (ex != null) if (ex != null)
UNSAFE.throwException(ex); rethrow(ex);
return tasks; return tasks;
} }
...@@ -820,20 +866,7 @@ public abstract class ForkJoinTask<V> implements Future<V>, Serializable { ...@@ -820,20 +866,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) {
return setCompletion(CANCELLED) == CANCELLED; return (setCompletion(CANCELLED) & DONE_MASK) == CANCELLED;
}
/**
* Cancels, ignoring any exceptions thrown by cancel. Used during
* worker and pool shutdown. Cancel is spec'ed not to throw any
* exceptions, but if it does anyway, we have no recourse during
* shutdown, so guard against this case.
*/
final void cancelIgnoringExceptions() {
try {
cancel(false);
} catch (Throwable ignore) {
}
} }
public final boolean isDone() { public final boolean isDone() {
...@@ -841,7 +874,7 @@ public abstract class ForkJoinTask<V> implements Future<V>, Serializable { ...@@ -841,7 +874,7 @@ public abstract class ForkJoinTask<V> implements Future<V>, Serializable {
} }
public final boolean isCancelled() { public final boolean isCancelled() {
return status == CANCELLED; return (status & DONE_MASK) == CANCELLED;
} }
/** /**
...@@ -861,7 +894,7 @@ public abstract class ForkJoinTask<V> implements Future<V>, Serializable { ...@@ -861,7 +894,7 @@ public abstract class ForkJoinTask<V> implements Future<V>, Serializable {
* exception and was not cancelled * exception and was not cancelled
*/ */
public final boolean isCompletedNormally() { public final boolean isCompletedNormally() {
return status == NORMAL; return (status & DONE_MASK) == NORMAL;
} }
/** /**
...@@ -872,7 +905,7 @@ public abstract class ForkJoinTask<V> implements Future<V>, Serializable { ...@@ -872,7 +905,7 @@ public abstract class ForkJoinTask<V> implements Future<V>, Serializable {
* @return the exception, or {@code null} if none * @return the exception, or {@code null} if none
*/ */
public final Throwable getException() { public final Throwable getException() {
int s = status; int s = status & DONE_MASK;
return ((s >= NORMAL) ? null : return ((s >= NORMAL) ? null :
(s == CANCELLED) ? new CancellationException() : (s == CANCELLED) ? new CancellationException() :
getThrowableException()); getThrowableException());
...@@ -921,6 +954,18 @@ public abstract class ForkJoinTask<V> implements Future<V>, Serializable { ...@@ -921,6 +954,18 @@ public abstract class ForkJoinTask<V> implements Future<V>, Serializable {
setCompletion(NORMAL); setCompletion(NORMAL);
} }
/**
* Completes this task normally without setting a value. The most
* recent value established by {@link #setRawResult} (or {@code
* null} by default) will be returned as the result of subsequent
* invocations of {@code join} and related operations.
*
* @since 1.8
*/
public final void quietlyComplete() {
setCompletion(NORMAL);
}
/** /**
* Waits if necessary for the computation to complete, and then * Waits if necessary for the computation to complete, and then
* retrieves its result. * retrieves its result.
...@@ -934,9 +979,9 @@ public abstract class ForkJoinTask<V> implements Future<V>, Serializable { ...@@ -934,9 +979,9 @@ public abstract class ForkJoinTask<V> implements Future<V>, Serializable {
*/ */
public final V get() throws InterruptedException, ExecutionException { public final V get() throws InterruptedException, ExecutionException {
int s = (Thread.currentThread() instanceof ForkJoinWorkerThread) ? int s = (Thread.currentThread() instanceof ForkJoinWorkerThread) ?
doJoin() : externalInterruptibleAwaitDone(0L); doJoin() : externalInterruptibleAwaitDone();
Throwable ex; Throwable ex;
if (s == CANCELLED) if ((s &= DONE_MASK) == CANCELLED)
throw new CancellationException(); throw new CancellationException();
if (s == EXCEPTIONAL && (ex = getThrowableException()) != null) if (s == EXCEPTIONAL && (ex = getThrowableException()) != null)
throw new ExecutionException(ex); throw new ExecutionException(ex);
...@@ -959,32 +1004,62 @@ public abstract class ForkJoinTask<V> implements Future<V>, Serializable { ...@@ -959,32 +1004,62 @@ 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 {
Thread t = Thread.currentThread(); if (Thread.interrupted())
if (t instanceof ForkJoinWorkerThread) { throw new InterruptedException();
ForkJoinWorkerThread w = (ForkJoinWorkerThread) t; // Messy in part because we measure in nanosecs, but wait in millisecs
long nanos = unit.toNanos(timeout); int s; long ns, ms;
if (status >= 0) { if ((s = status) >= 0 && (ns = unit.toNanos(timeout)) > 0L) {
boolean completed = false; long deadline = System.nanoTime() + ns;
if (w.unpushTask(this)) { ForkJoinPool p = null;
try { ForkJoinPool.WorkQueue w = null;
completed = exec(); Thread t = Thread.currentThread();
} catch (Throwable rex) { if (t instanceof ForkJoinWorkerThread) {
setExceptionalCompletion(rex); ForkJoinWorkerThread wt = (ForkJoinWorkerThread)t;
p = wt.pool;
w = wt.workQueue;
p.helpJoinOnce(w, this); // no retries on failure
}
else
ForkJoinPool.externalHelpJoin(this);
boolean canBlock = false;
boolean interrupted = false;
try {
while ((s = status) >= 0) {
if (w != null && w.qlock < 0)
cancelIgnoringExceptions(this);
else if (!canBlock) {
if (p == null || p.tryCompensate())
canBlock = true;
}
else {
if ((ms = TimeUnit.NANOSECONDS.toMillis(ns)) > 0L &&
U.compareAndSwapInt(this, STATUS, s, s | SIGNAL)) {
synchronized (this) {
if (status >= 0) {
try {
wait(ms);
} catch (InterruptedException ie) {
if (p == null)
interrupted = true;
}
}
else
notifyAll();
}
}
if ((s = status) < 0 || interrupted ||
(ns = deadline - System.nanoTime()) <= 0L)
break;
} }
} }
if (completed) } finally {
setCompletion(NORMAL); if (p != null && canBlock)
else if (status >= 0 && nanos > 0) p.incrementActiveCount();
w.pool.timedAwaitJoin(this, nanos);
} }
if (interrupted)
throw new InterruptedException();
} }
else { if ((s &= DONE_MASK) != NORMAL) {
long millis = unit.toMillis(timeout);
if (millis > 0)
externalInterruptibleAwaitDone(millis);
}
int s = status;
if (s != NORMAL) {
Throwable ex; Throwable ex;
if (s == CANCELLED) if (s == CANCELLED)
throw new CancellationException(); throw new CancellationException();
...@@ -1021,16 +1096,15 @@ public abstract class ForkJoinTask<V> implements Future<V>, Serializable { ...@@ -1021,16 +1096,15 @@ public abstract class ForkJoinTask<V> implements Future<V>, Serializable {
* be of use in designs in which many tasks are forked, but none * be of use in designs in which many tasks are forked, but none
* are explicitly joined, instead executing them until all are * are explicitly joined, instead executing them until all are
* processed. * processed.
*
* <p>This method may be invoked only from within {@code
* ForkJoinPool} computations (as may be determined using method
* {@link #inForkJoinPool}). Attempts to invoke in other contexts
* result in exceptions or errors, possibly including {@code
* ClassCastException}.
*/ */
public static void helpQuiesce() { public static void helpQuiesce() {
((ForkJoinWorkerThread) Thread.currentThread()) Thread t;
.helpQuiescePool(); if ((t = Thread.currentThread()) instanceof ForkJoinWorkerThread) {
ForkJoinWorkerThread wt = (ForkJoinWorkerThread)t;
wt.pool.helpQuiescePool(wt.workQueue);
}
else
ForkJoinPool.externalHelpQuiescePool();
} }
/** /**
...@@ -1050,7 +1124,7 @@ public abstract class ForkJoinTask<V> implements Future<V>, Serializable { ...@@ -1050,7 +1124,7 @@ public abstract class ForkJoinTask<V> implements Future<V>, Serializable {
* setRawResult(null)}. * setRawResult(null)}.
*/ */
public void reinitialize() { public void reinitialize() {
if (status == EXCEPTIONAL) if ((status & DONE_MASK) == EXCEPTIONAL)
clearExceptionalCompletion(); clearExceptionalCompletion();
else else
status = 0; status = 0;
...@@ -1083,23 +1157,19 @@ public abstract class ForkJoinTask<V> implements Future<V>, Serializable { ...@@ -1083,23 +1157,19 @@ public abstract class ForkJoinTask<V> implements Future<V>, Serializable {
/** /**
* Tries to unschedule this task for execution. This method will * Tries to unschedule this task for execution. This method will
* typically succeed if this task is the most recently forked task * typically (but is not guaranteed to) succeed if this task is
* by the current thread, and has not commenced executing in * the most recently forked task by the current thread, and has
* another thread. This method may be useful when arranging * not commenced executing in another thread. This method may be
* alternative local processing of tasks that could have been, but * useful when arranging alternative local processing of tasks
* were not, stolen. * that could have been, but were not, stolen.
*
* <p>This method may be invoked only from within {@code
* ForkJoinPool} computations (as may be determined using method
* {@link #inForkJoinPool}). Attempts to invoke in other contexts
* result in exceptions or errors, possibly including {@code
* ClassCastException}.
* *
* @return {@code true} if unforked * @return {@code true} if unforked
*/ */
public boolean tryUnfork() { public boolean tryUnfork() {
return ((ForkJoinWorkerThread) Thread.currentThread()) Thread t;
.unpushTask(this); return (((t = Thread.currentThread()) instanceof ForkJoinWorkerThread) ?
((ForkJoinWorkerThread)t).workQueue.tryUnpush(this) :
ForkJoinPool.tryExternalUnpush(this));
} }
/** /**
...@@ -1108,40 +1178,32 @@ public abstract class ForkJoinTask<V> implements Future<V>, Serializable { ...@@ -1108,40 +1178,32 @@ public abstract class ForkJoinTask<V> implements Future<V>, Serializable {
* value may be useful for heuristic decisions about whether to * value may be useful for heuristic decisions about whether to
* fork other tasks. * fork other tasks.
* *
* <p>This method may be invoked only from within {@code
* ForkJoinPool} computations (as may be determined using method
* {@link #inForkJoinPool}). Attempts to invoke in other contexts
* result in exceptions or errors, possibly including {@code
* ClassCastException}.
*
* @return the number of tasks * @return the number of tasks
*/ */
public static int getQueuedTaskCount() { public static int getQueuedTaskCount() {
return ((ForkJoinWorkerThread) Thread.currentThread()) Thread t; ForkJoinPool.WorkQueue q;
.getQueueSize(); if ((t = Thread.currentThread()) instanceof ForkJoinWorkerThread)
q = ((ForkJoinWorkerThread)t).workQueue;
else
q = ForkJoinPool.commonSubmitterQueue();
return (q == null) ? 0 : q.queueSize();
} }
/** /**
* Returns an estimate of how many more locally queued tasks are * Returns an estimate of how many more locally queued tasks are
* held by the current worker thread than there are other worker * held by the current worker thread than there are other worker
* threads that might steal them. This value may be useful for * threads that might steal them, or zero if this thread is not
* operating in a ForkJoinPool. This value may be useful for
* heuristic decisions about whether to fork other tasks. In many * heuristic decisions about whether to fork other tasks. In many
* usages of ForkJoinTasks, at steady state, each worker should * usages of ForkJoinTasks, at steady state, each worker should
* aim to maintain a small constant surplus (for example, 3) of * aim to maintain a small constant surplus (for example, 3) of
* tasks, and to process computations locally if this threshold is * tasks, and to process computations locally if this threshold is
* exceeded. * exceeded.
* *
* <p>This method may be invoked only from within {@code
* ForkJoinPool} computations (as may be determined using method
* {@link #inForkJoinPool}). Attempts to invoke in other contexts
* result in exceptions or errors, possibly including {@code
* ClassCastException}.
*
* @return the surplus number of tasks, which may be negative * @return the surplus number of tasks, which may be negative
*/ */
public static int getSurplusQueuedTaskCount() { public static int getSurplusQueuedTaskCount() {
return ((ForkJoinWorkerThread) Thread.currentThread()) return ForkJoinPool.getSurplusQueuedTaskCount();
.getEstimatedSurplusTaskCount();
} }
// Extension methods // Extension methods
...@@ -1167,15 +1229,18 @@ public abstract class ForkJoinTask<V> implements Future<V>, Serializable { ...@@ -1167,15 +1229,18 @@ public abstract class ForkJoinTask<V> implements Future<V>, Serializable {
protected abstract void setRawResult(V value); protected abstract void setRawResult(V value);
/** /**
* Immediately performs the base action of this task. This method * Immediately performs the base action of this task and returns
* is designed to support extensions, and should not in general be * true if, upon return from this method, this task is guaranteed
* called otherwise. The return value controls whether this task * to have completed normally. This method may return false
* is considered to be done normally. It may return false in * otherwise, to indicate that this task is not necessarily
* complete (or is not known to be complete), for example in
* asynchronous actions that require explicit invocations of * asynchronous actions that require explicit invocations of
* {@link #complete} to become joinable. It may also throw an * completion methods. This method may also throw an (unchecked)
* (unchecked) exception to indicate abnormal exit. * exception to indicate abnormal exit. This method is designed to
* support extensions, and should not in general be called
* otherwise.
* *
* @return {@code true} if completed normally * @return {@code true} if this task is known to have completed normally
*/ */
protected abstract boolean exec(); protected abstract boolean exec();
...@@ -1189,59 +1254,102 @@ public abstract class ForkJoinTask<V> implements Future<V>, Serializable { ...@@ -1189,59 +1254,102 @@ public abstract class ForkJoinTask<V> implements Future<V>, Serializable {
* primarily to support extensions, and is unlikely to be useful * primarily to support extensions, and is unlikely to be useful
* otherwise. * otherwise.
* *
* <p>This method may be invoked only from within {@code
* ForkJoinPool} computations (as may be determined using method
* {@link #inForkJoinPool}). Attempts to invoke in other contexts
* result in exceptions or errors, possibly including {@code
* ClassCastException}.
*
* @return the next task, or {@code null} if none are available * @return the next task, or {@code null} if none are available
*/ */
protected static ForkJoinTask<?> peekNextLocalTask() { protected static ForkJoinTask<?> peekNextLocalTask() {
return ((ForkJoinWorkerThread) Thread.currentThread()) Thread t; ForkJoinPool.WorkQueue q;
.peekTask(); if ((t = Thread.currentThread()) instanceof ForkJoinWorkerThread)
q = ((ForkJoinWorkerThread)t).workQueue;
else
q = ForkJoinPool.commonSubmitterQueue();
return (q == null) ? null : q.peek();
} }
/** /**
* Unschedules and returns, without executing, the next task * Unschedules and returns, without executing, the next task
* queued by the current thread but not yet executed. This method * queued by the current thread but not yet executed, if the
* is designed primarily to support extensions, and is unlikely to * current thread is operating in a ForkJoinPool. This method is
* be useful otherwise. * designed primarily to support extensions, and is unlikely to be
* * useful otherwise.
* <p>This method may be invoked only from within {@code
* ForkJoinPool} computations (as may be determined using method
* {@link #inForkJoinPool}). Attempts to invoke in other contexts
* result in exceptions or errors, possibly including {@code
* ClassCastException}.
* *
* @return the next task, or {@code null} if none are available * @return the next task, or {@code null} if none are available
*/ */
protected static ForkJoinTask<?> pollNextLocalTask() { protected static ForkJoinTask<?> pollNextLocalTask() {
return ((ForkJoinWorkerThread) Thread.currentThread()) Thread t;
.pollLocalTask(); return ((t = Thread.currentThread()) instanceof ForkJoinWorkerThread) ?
((ForkJoinWorkerThread)t).workQueue.nextLocalTask() :
null;
} }
/** /**
* Unschedules and returns, without executing, the next task * If the current thread is operating in a ForkJoinPool,
* unschedules and returns, without executing, the next task
* queued by the current thread but not yet executed, if one is * queued by the current thread but not yet executed, if one is
* available, or if not available, a task that was forked by some * available, or if not available, a task that was forked by some
* other thread, if available. Availability may be transient, so a * other thread, if available. Availability may be transient, so a
* {@code null} result does not necessarily imply quiescence * {@code null} result does not necessarily imply quiescence of
* of the pool this task is operating in. This method is designed * the pool this task is operating in. This method is designed
* primarily to support extensions, and is unlikely to be useful * primarily to support extensions, and is unlikely to be useful
* otherwise. * otherwise.
* *
* <p>This method may be invoked only from within {@code
* ForkJoinPool} computations (as may be determined using method
* {@link #inForkJoinPool}). Attempts to invoke in other contexts
* result in exceptions or errors, possibly including {@code
* ClassCastException}.
*
* @return a task, or {@code null} if none are available * @return a task, or {@code null} if none are available
*/ */
protected static ForkJoinTask<?> pollTask() { protected static ForkJoinTask<?> pollTask() {
return ((ForkJoinWorkerThread) Thread.currentThread()) Thread t; ForkJoinWorkerThread wt;
.pollTask(); return ((t = Thread.currentThread()) instanceof ForkJoinWorkerThread) ?
(wt = (ForkJoinWorkerThread)t).pool.nextTaskFor(wt.workQueue) :
null;
}
// tag operations
/**
* Returns the tag for this task.
*
* @return the tag for this task
* @since 1.8
*/
public final short getForkJoinTaskTag() {
return (short)status;
}
/**
* Atomically sets the tag value for this task.
*
* @param tag the tag value
* @return the previous value of the tag
* @since 1.8
*/
public final short setForkJoinTaskTag(short tag) {
for (int s;;) {
if (U.compareAndSwapInt(this, STATUS, s = status,
(s & ~SMASK) | (tag & SMASK)))
return (short)s;
}
}
/**
* Atomically conditionally sets the tag value for this task.
* Among other applications, tags can be used as visit markers
* in tasks operating on graphs, as in methods that check: {@code
* if (task.compareAndSetForkJoinTaskTag((short)0, (short)1))}
* before processing, otherwise exiting because the node has
* already been visited.
*
* @param e the expected tag value
* @param tag the new tag value
* @return true if successful; i.e., the current value was
* equal to e and is now tag.
* @since 1.8
*/
public final boolean compareAndSetForkJoinTaskTag(short e, short tag) {
for (int s;;) {
if ((short)(s = status) != e)
return false;
if (U.compareAndSwapInt(this, STATUS, s,
(s & ~SMASK) | (tag & SMASK)))
return true;
}
} }
/** /**
...@@ -1252,21 +1360,33 @@ public abstract class ForkJoinTask<V> implements Future<V>, Serializable { ...@@ -1252,21 +1360,33 @@ public abstract class ForkJoinTask<V> implements Future<V>, Serializable {
static final class AdaptedRunnable<T> extends ForkJoinTask<T> static final class AdaptedRunnable<T> extends ForkJoinTask<T>
implements RunnableFuture<T> { implements RunnableFuture<T> {
final Runnable runnable; final Runnable runnable;
final T resultOnCompletion;
T result; T result;
AdaptedRunnable(Runnable runnable, T result) { AdaptedRunnable(Runnable runnable, T result) {
if (runnable == null) throw new NullPointerException(); if (runnable == null) throw new NullPointerException();
this.runnable = runnable; this.runnable = runnable;
this.resultOnCompletion = result; this.result = result; // OK to set this even before completion
} }
public T getRawResult() { return result; } public final T getRawResult() { return result; }
public void setRawResult(T v) { result = v; } public final void setRawResult(T v) { result = v; }
public boolean exec() { public final boolean exec() { runnable.run(); return true; }
runnable.run(); public final void run() { invoke(); }
result = resultOnCompletion; private static final long serialVersionUID = 5232453952276885070L;
return true; }
/**
* Adaptor for Runnables without results
*/
static final class AdaptedRunnableAction extends ForkJoinTask<Void>
implements RunnableFuture<Void> {
final Runnable runnable;
AdaptedRunnableAction(Runnable runnable) {
if (runnable == null) throw new NullPointerException();
this.runnable = runnable;
} }
public void run() { invoke(); } public final Void getRawResult() { return null; }
public final void setRawResult(Void v) { }
public final boolean exec() { runnable.run(); return true; }
public final void run() { invoke(); }
private static final long serialVersionUID = 5232453952276885070L; private static final long serialVersionUID = 5232453952276885070L;
} }
...@@ -1281,9 +1401,9 @@ public abstract class ForkJoinTask<V> implements Future<V>, Serializable { ...@@ -1281,9 +1401,9 @@ public abstract class ForkJoinTask<V> implements Future<V>, Serializable {
if (callable == null) throw new NullPointerException(); if (callable == null) throw new NullPointerException();
this.callable = callable; this.callable = callable;
} }
public T getRawResult() { return result; } public final T getRawResult() { return result; }
public void setRawResult(T v) { result = v; } public final void setRawResult(T v) { result = v; }
public boolean exec() { public final boolean exec() {
try { try {
result = callable.call(); result = callable.call();
return true; return true;
...@@ -1295,7 +1415,7 @@ public abstract class ForkJoinTask<V> implements Future<V>, Serializable { ...@@ -1295,7 +1415,7 @@ public abstract class ForkJoinTask<V> implements Future<V>, Serializable {
throw new RuntimeException(ex); throw new RuntimeException(ex);
} }
} }
public void run() { invoke(); } public final void run() { invoke(); }
private static final long serialVersionUID = 2838392045355241008L; private static final long serialVersionUID = 2838392045355241008L;
} }
...@@ -1308,7 +1428,7 @@ public abstract class ForkJoinTask<V> implements Future<V>, Serializable { ...@@ -1308,7 +1428,7 @@ public abstract class ForkJoinTask<V> implements Future<V>, Serializable {
* @return the task * @return the task
*/ */
public static ForkJoinTask<?> adapt(Runnable runnable) { public static ForkJoinTask<?> adapt(Runnable runnable) {
return new AdaptedRunnable<Void>(runnable, null); return new AdaptedRunnableAction(runnable);
} }
/** /**
...@@ -1342,11 +1462,10 @@ public abstract class ForkJoinTask<V> implements Future<V>, Serializable { ...@@ -1342,11 +1462,10 @@ public abstract class ForkJoinTask<V> implements Future<V>, Serializable {
private static final long serialVersionUID = -7721805057305804111L; private static final long serialVersionUID = -7721805057305804111L;
/** /**
* Saves the state to a stream (that is, serializes it). * Saves this task to a stream (that is, serializes it).
* *
* @serialData the current run status and the exception thrown * @serialData the current run status and the exception thrown
* during execution, or {@code null} if none * during execution, or {@code null} if none
* @param s the stream
*/ */
private void writeObject(java.io.ObjectOutputStream s) private void writeObject(java.io.ObjectOutputStream s)
throws java.io.IOException { throws java.io.IOException {
...@@ -1355,9 +1474,7 @@ public abstract class ForkJoinTask<V> implements Future<V>, Serializable { ...@@ -1355,9 +1474,7 @@ public abstract class ForkJoinTask<V> implements Future<V>, Serializable {
} }
/** /**
* Reconstitutes the instance from a stream (that is, deserializes it). * Reconstitutes this task from a stream (that is, deserializes it).
*
* @param s the stream
*/ */
private void readObject(java.io.ObjectInputStream s) private void readObject(java.io.ObjectInputStream s)
throws java.io.IOException, ClassNotFoundException { throws java.io.IOException, ClassNotFoundException {
...@@ -1368,16 +1485,18 @@ public abstract class ForkJoinTask<V> implements Future<V>, Serializable { ...@@ -1368,16 +1485,18 @@ public abstract class ForkJoinTask<V> implements Future<V>, Serializable {
} }
// Unsafe mechanics // Unsafe mechanics
private static final sun.misc.Unsafe UNSAFE; private static final sun.misc.Unsafe U;
private static final long statusOffset; private static final long STATUS;
static { static {
exceptionTableLock = new ReentrantLock(); exceptionTableLock = new ReentrantLock();
exceptionTableRefQueue = new ReferenceQueue<Object>(); exceptionTableRefQueue = new ReferenceQueue<Object>();
exceptionTable = new ExceptionNode[EXCEPTION_MAP_CAPACITY]; exceptionTable = new ExceptionNode[EXCEPTION_MAP_CAPACITY];
try { try {
UNSAFE = sun.misc.Unsafe.getUnsafe(); U = sun.misc.Unsafe.getUnsafe();
statusOffset = UNSAFE.objectFieldOffset Class<?> k = ForkJoinTask.class;
(ForkJoinTask.class.getDeclaredField("status")); STATUS = U.objectFieldOffset
(k.getDeclaredField("status"));
} catch (Exception e) { } catch (Exception e) {
throw new Error(e); throw new Error(e);
} }
......
...@@ -35,9 +35,6 @@ ...@@ -35,9 +35,6 @@
package java.util.concurrent; package java.util.concurrent;
import java.util.Collection;
import java.util.concurrent.RejectedExecutionException;
/** /**
* A thread managed by a {@link ForkJoinPool}, which executes * A thread managed by a {@link ForkJoinPool}, which executes
* {@link ForkJoinTask}s. * {@link ForkJoinTask}s.
...@@ -54,238 +51,20 @@ import java.util.concurrent.RejectedExecutionException; ...@@ -54,238 +51,20 @@ import java.util.concurrent.RejectedExecutionException;
*/ */
public class ForkJoinWorkerThread extends Thread { public class ForkJoinWorkerThread extends Thread {
/* /*
* Overview:
*
* ForkJoinWorkerThreads are managed by ForkJoinPools and perform * ForkJoinWorkerThreads are managed by ForkJoinPools and perform
* ForkJoinTasks. This class includes bookkeeping in support of * ForkJoinTasks. For explanation, see the internal documentation
* worker activation, suspension, and lifecycle control described * of class ForkJoinPool.
* in more detail in the internal documentation of class
* ForkJoinPool. And as described further below, this class also
* includes special-cased support for some ForkJoinTask
* methods. But the main mechanics involve work-stealing:
*
* Work-stealing queues are special forms of Deques that support
* only three of the four possible end-operations -- push, pop,
* and deq (aka steal), under the further constraints that push
* and pop are called only from the owning thread, while deq may
* be called from other threads. (If you are unfamiliar with
* them, you probably want to read Herlihy and Shavit's book "The
* Art of Multiprocessor programming", chapter 16 describing these
* in more detail before proceeding.) The main work-stealing
* queue design is roughly similar to those in the papers "Dynamic
* Circular Work-Stealing Deque" by Chase and Lev, SPAA 2005
* (http://research.sun.com/scalable/pubs/index.html) and
* "Idempotent work stealing" by Michael, Saraswat, and Vechev,
* PPoPP 2009 (http://portal.acm.org/citation.cfm?id=1504186).
* The main differences ultimately stem from gc requirements that
* we null out taken slots as soon as we can, to maintain as small
* a footprint as possible even in programs generating huge
* numbers of tasks. To accomplish this, we shift the CAS
* arbitrating pop vs deq (steal) from being on the indices
* ("queueBase" and "queueTop") to the slots themselves (mainly
* via method "casSlotNull()"). So, both a successful pop and deq
* mainly entail a CAS of a slot from non-null to null. Because
* we rely on CASes of references, we do not need tag bits on
* queueBase or queueTop. They are simple ints as used in any
* circular array-based queue (see for example ArrayDeque).
* Updates to the indices must still be ordered in a way that
* guarantees that queueTop == queueBase means the queue is empty,
* but otherwise may err on the side of possibly making the queue
* appear nonempty when a push, pop, or deq have not fully
* committed. Note that this means that the deq operation,
* considered individually, is not wait-free. One thief cannot
* 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.
* If an attempted steal fails, a thief always chooses a different
* 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
* any empty queue to complete.
*
* This approach also enables support for "async mode" where local
* task processing is in FIFO, not LIFO order; simply by using a
* version of deq rather than pop when locallyFifo is true (as set
* by the ForkJoinPool). This allows use in message-passing
* 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 * This class just maintains links to its pool and WorkQueue. The
* task, it first tries a form of linear helping: Each worker * pool field is set immediately upon construction, but the
* records (in field currentSteal) the most recent task it stole * workQueue field is not set until a call to registerWorker
* from some other worker. Plus, it records (in field currentJoin) * completes. This leads to a visibility race, that is tolerated
* the task it is currently actively joining. Method joinTask uses * by requiring that the workQueue field is only accessed by the
* these markers to try to find a worker to help (i.e., steal back * owning thread.
* a task from and execute it) that could hasten completion of the
* actively joined task. In essence, the joiner executes a task
* that would be on its own local deque had the to-be-joined task
* not been stolen. This may be seen as a conservative variant of
* the approach in Wagner & Calder "Leapfrogging: a portable
* technique for implementing efficient futures" SIGPLAN Notices,
* 1993 (http://portal.acm.org/citation.cfm?id=155354). It differs
* in that: (1) We only maintain dependency links across workers
* upon steals, rather than use per-task bookkeeping. This may
* require a linear scan of workers array to locate stealers, but
* usually doesn't because stealers leave hints (that may become
* stale/wrong) of where to locate them. This isolates cost to
* when it is needed, rather than adding to per-task overhead.
* (2) It is "shallow", ignoring nesting and potentially cyclic
* mutual steals. (3) It is intentionally racy: field currentJoin
* is updated only while actively joining, which means that we
* miss links in the chain during long-lived tasks, GC stalls etc
* (which is OK since blocking in such cases is usually a good
* idea). (4) We bound the number of attempts to find work (see
* MAX_HELP) and fall back to suspending the worker and if
* necessary replacing it with another.
*
* Efficient implementation of these algorithms currently relies
* on an uncomfortable amount of "Unsafe" mechanics. To maintain
* correct orderings, reads and writes of variable queueBase
* require volatile ordering. Variable queueTop need not be
* volatile because non-local reads always follow those of
* queueBase. Similarly, because they are protected by volatile
* queueBase reads, reads of the queue array and its slots by
* other threads do not need volatile load semantics, but writes
* (in push) require store order and CASes (in pop and deq)
* require (volatile) CAS semantics. (Michael, Saraswat, and
* Vechev's algorithm has similar properties, but without support
* for nulling slots.) Since these combinations aren't supported
* using ordinary volatiles, the only way to accomplish these
* efficiently is to use direct Unsafe calls. (Using external
* AtomicIntegers and AtomicReferenceArrays for the indices and
* array is significantly slower because of memory locality and
* indirection effects.)
*
* Further, performance on most platforms is very sensitive to
* placement and sizing of the (resizable) queue array. Even
* though these queues don't usually become all that big, the
* initial size must be large enough to counteract cache
* contention effects across multiple queues (especially in the
* presence of GC cardmarking). Also, to improve thread-locality,
* queues are initialized after starting.
*/
/**
* Mask for pool indices encoded as shorts
*/
private static final int SMASK = 0xffff;
/**
* Capacity of work-stealing queue array upon initialization.
* Must be a power of two. Initial size must be at least 4, but is
* padded to minimize cache effects.
*/
private static final int INITIAL_QUEUE_CAPACITY = 1 << 13;
/**
* Maximum size for queue array. Must be a power of two
* less than or equal to 1 << (31 - width of array entry) to
* ensure lack of index wraparound, but is capped at a lower
* 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 opposed to when constructed), to
* improve memory locality.
*/
ForkJoinTask<?>[] queue;
/**
* The pool this thread works in. Accessed directly by ForkJoinTask.
*/
final ForkJoinPool pool;
/**
* 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) queueBase. Both queueTop
* and queueBase are allowed to wrap around on overflow, but
* (queueTop - queueBase) still estimates size.
*/
int queueTop;
/**
* Index (mod queue.length) of least valid queue slot, which is
* always the next position to steal from if nonempty.
*/ */
volatile int queueBase;
/** final ForkJoinPool pool; // the pool this thread works in
* The index of most recent stealer, used as a hint to avoid final ForkJoinPool.WorkQueue workQueue; // work-stealing mechanics
* traversal in method helpJoinTask. This is only a hint because a
* worker might have had multiple steals and this only holds one
* of them (usually the most current). Declared non-volatile,
* relying on other prevailing sync to keep reasonably current.
*/
int stealHint;
/**
* Index of this worker in pool array. Set once by pool before
* running, and accessed directly by pool to locate this worker in
* its workers array.
*/
final int poolIndex;
/**
* Encoded record for pool task waits. Usages are always
* surrounded by volatile reads/writes
*/
int nextWait;
/**
* Complement of poolIndex, offset by count of entries of task
* waits. Accessed by ForkJoinPool to manage event waiters.
*/
volatile int eventCount;
/**
* Seed for random number generator for choosing steal victims.
* Uses Marsaglia xorshift. Must be initialized as nonzero.
*/
int seed;
/**
* Number of steals. Directly accessed (and reset) by pool when
* idle.
*/
int stealCount;
/**
* True if this worker should or did terminate
*/
volatile boolean terminate;
/**
* Set to true before LockSupport.park; false on return
*/
volatile boolean parked;
/**
* True if use local fifo, not default lifo, for local polling.
* Shadows value from ForkJoinPool.
*/
final boolean locallyFifo;
/**
* The task most recently stolen from another worker (or
* submission queue). All uses are surrounded by enough volatile
* reads/writes to maintain as non-volatile.
*/
ForkJoinTask<?> currentSteal;
/**
* The task currently being joined, set only when actively trying
* to help other stealers in helpJoinTask. All uses are surrounded
* by enough volatile reads/writes to maintain as non-volatile.
*/
ForkJoinTask<?> currentJoin;
/** /**
* Creates a ForkJoinWorkerThread operating in the given pool. * Creates a ForkJoinWorkerThread operating in the given pool.
...@@ -294,20 +73,12 @@ public class ForkJoinWorkerThread extends Thread { ...@@ -294,20 +73,12 @@ 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()); // Use a placeholder until a useful name can be set in registerWorker
super("aForkJoinWorkerThread");
this.pool = pool; this.pool = pool;
int k = pool.registerWorker(this); this.workQueue = pool.registerWorker(this);
poolIndex = k;
eventCount = ~k & SMASK; // clear wait count
locallyFifo = pool.locallyFifo;
Thread.UncaughtExceptionHandler ueh = pool.ueh;
if (ueh != null)
setUncaughtExceptionHandler(ueh);
setDaemon(true);
} }
// Public methods
/** /**
* Returns the pool hosting this thread. * Returns the pool hosting this thread.
* *
...@@ -327,28 +98,9 @@ public class ForkJoinWorkerThread extends Thread { ...@@ -327,28 +98,9 @@ public class ForkJoinWorkerThread extends Thread {
* @return the index number * @return the index number
*/ */
public int getPoolIndex() { public int getPoolIndex() {
return poolIndex; return workQueue.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
...@@ -359,9 +111,6 @@ public class ForkJoinWorkerThread extends Thread { ...@@ -359,9 +111,6 @@ public class ForkJoinWorkerThread extends Thread {
* processing tasks. * processing tasks.
*/ */
protected void onStart() { protected void onStart() {
queue = new ForkJoinTask<?>[INITIAL_QUEUE_CAPACITY];
int r = ForkJoinPool.workerSeedGenerator.nextInt();
seed = (r == 0) ? 1 : r; // must be nonzero
} }
/** /**
...@@ -373,17 +122,6 @@ public class ForkJoinWorkerThread extends Thread { ...@@ -373,17 +122,6 @@ public class ForkJoinWorkerThread extends Thread {
* to an unrecoverable error, or {@code null} if completed normally * to an unrecoverable error, or {@code null} if completed normally
*/ */
protected void onTermination(Throwable exception) { protected void onTermination(Throwable exception) {
try {
terminate = true;
cancelTasks();
pool.deregisterWorker(this, exception);
} catch (Throwable ex) { // Shouldn't ever happen
if (exception == null) // but if so, at least rethrown
exception = ex;
} finally {
if (exception != null)
UNSAFE.throwException(exception);
}
} }
/** /**
...@@ -395,604 +133,18 @@ public class ForkJoinWorkerThread extends Thread { ...@@ -395,604 +133,18 @@ public class ForkJoinWorkerThread extends Thread {
Throwable exception = null; Throwable exception = null;
try { try {
onStart(); onStart();
pool.work(this); pool.runWorker(workQueue);
} catch (Throwable ex) { } catch (Throwable ex) {
exception = ex; exception = ex;
} finally { } finally {
onTermination(exception); try {
} onTermination(exception);
} } catch (Throwable ex) {
if (exception == null)
/* exception = ex;
* Intrinsics-based atomic writes for queue slots. These are } finally {
* basically the same as methods in AtomicReferenceArray, but pool.deregisterWorker(this, exception);
* specialized for (1) ForkJoinTask elements (2) requirement that
* nullness and bounds checks have already been performed by
* callers and (3) effective offsets are known not to overflow
* from int to long (because of MAXIMUM_QUEUE_CAPACITY). We don't
* need corresponding version for reads: plain array reads are OK
* because they are protected by other volatile reads and are
* confirmed by CASes.
*
* Most uses don't actually call these methods, but instead
* contain inlined forms that enable more predictable
* optimization. We don't define the version of write used in
* 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.
*/
/**
* CASes slot i of array q from t to null. Caller must ensure q is
* non-null and index is in range.
*/
private static final boolean casSlotNull(ForkJoinTask<?>[] q, int i,
ForkJoinTask<?> t) {
return UNSAFE.compareAndSwapObject(q, (i << ASHIFT) + ABASE, t, null);
}
/**
* Performs a volatile write of the given task at given slot of
* array q. Caller must ensure q is non-null and index is in
* range. This method is used only during resets and backouts.
*/
private static final void writeSlot(ForkJoinTask<?>[] q, int i,
ForkJoinTask<?> t) {
UNSAFE.putObjectVolatile(q, (i << ASHIFT) + ABASE, t);
}
// queue methods
/**
* Pushes a task. Call only from this thread.
*
* @param t the task. Caller must ensure non-null.
*/
final void pushTask(ForkJoinTask<?> t) {
ForkJoinTask<?>[] q; int s, m;
if ((q = queue) != null) { // ignore if queue removed
long u = (((s = queueTop) & (m = q.length - 1)) << ASHIFT) + ABASE;
UNSAFE.putOrderedObject(q, u, t);
queueTop = s + 1; // or use putOrderedInt
if ((s -= queueBase) <= 2)
pool.signalWork();
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);
}
}
}
/**
* Tries to take a task from the base of the queue, failing if
* empty or contended. Note: Specializations of this code appear
* in locallyDeqTask and elsewhere.
*
* @return a task, or null if none or contended
*/
final ForkJoinTask<?> deqTask() {
ForkJoinTask<?> t; ForkJoinTask<?>[] q; int b, i;
if (queueTop != (b = queueBase) &&
(q = queue) != null && // must read q after b
(i = (q.length - 1) & b) >= 0 &&
(t = q[i]) != null && queueBase == b &&
UNSAFE.compareAndSwapObject(q, (i << ASHIFT) + ABASE, t, null)) {
queueBase = b + 1;
return t;
}
return null;
}
/**
* Tries to take a task from the base of own queue. Called only
* by this thread.
*
* @return a task, or null if none
*/
final ForkJoinTask<?> locallyDeqTask() {
ForkJoinTask<?> t; int m, b, i;
ForkJoinTask<?>[] q = queue;
if (q != null && (m = q.length - 1) >= 0) {
while (queueTop != (b = queueBase)) {
if ((t = q[i = m & b]) != null &&
queueBase == b &&
UNSAFE.compareAndSwapObject(q, (i << ASHIFT) + ABASE,
t, null)) {
queueBase = b + 1;
return t;
}
} }
} }
return null;
} }
/**
* Returns a popped task, or null if empty.
* Called only by this thread.
*/
private ForkJoinTask<?> popTask() {
int m;
ForkJoinTask<?>[] q = queue;
if (q != null && (m = q.length - 1) >= 0) {
for (int s; (s = queueTop) != queueBase;) {
int i = m & --s;
long u = (i << ASHIFT) + ABASE; // raw offset
ForkJoinTask<?> t = q[i];
if (t == null) // lost to stealer
break;
if (UNSAFE.compareAndSwapObject(q, u, t, null)) {
queueTop = s; // or putOrderedInt
return t;
}
}
}
return null;
}
/**
* Specialized version of popTask to pop only if topmost element
* is the given task. Called only by this thread.
*
* @param t the task. Caller must ensure non-null.
*/
final boolean unpushTask(ForkJoinTask<?> t) {
ForkJoinTask<?>[] q;
int s;
if ((q = queue) != null && (s = queueTop) != queueBase &&
UNSAFE.compareAndSwapObject
(q, (((q.length - 1) & --s) << ASHIFT) + ABASE, t, null)) {
queueTop = s; // or putOrderedInt
return true;
}
return false;
}
/**
* Returns next task, or null if empty or contended.
*/
final ForkJoinTask<?> peekTask() {
int m;
ForkJoinTask<?>[] q = queue;
if (q == null || (m = q.length - 1) < 0)
return null;
int i = locallyFifo ? queueBase : (queueTop - 1);
return q[i & m];
}
// Support methods for ForkJoinPool
/**
* Runs the given task, plus any local tasks until queue is empty
*/
final void execTask(ForkJoinTask<?> t) {
currentSteal = t;
for (;;) {
if (t != null)
t.doExec();
if (queueTop == queueBase)
break;
t = locallyFifo ? locallyDeqTask() : popTask();
}
++stealCount;
currentSteal = null;
}
/**
* Removes and cancels all tasks in queue. Can be called from any
* thread.
*/
final void cancelTasks() {
ForkJoinTask<?> cj = currentJoin; // try to cancel ongoing tasks
if (cj != null && cj.status >= 0)
cj.cancelIgnoringExceptions();
ForkJoinTask<?> cs = currentSteal;
if (cs != null && cs.status >= 0)
cs.cancelIgnoringExceptions();
while (queueBase != queueTop) {
ForkJoinTask<?> t = deqTask();
if (t != null)
t.cancelIgnoringExceptions();
}
}
/**
* Drains tasks to given collection c.
*
* @return the number of tasks drained
*/
final int drainTasksTo(Collection<? super ForkJoinTask<?>> c) {
int n = 0;
while (queueBase != queueTop) {
ForkJoinTask<?> t = deqTask();
if (t != null) {
c.add(t);
++n;
}
}
return n;
}
// 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.
*
* @return a task, if available
*/
final ForkJoinTask<?> pollLocalTask() {
return locallyFifo ? locallyDeqTask() : popTask();
}
/**
* Gets and removes a local or stolen task.
*
* @return a task, if available
*/
final ForkJoinTask<?> pollTask() {
ForkJoinWorkerThread[] ws;
ForkJoinTask<?> t = pollLocalTask();
if (t != null || (ws = pool.workers) == null)
return t;
int n = ws.length; // cheap version of FJP.scan
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;
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 joinMe is done.
*
* @param joinMe the task to join
* @return completion status on exit
*/
final int joinTask(ForkJoinTask<?> joinMe) {
ForkJoinTask<?> prevJoin = currentJoin;
currentJoin = joinMe;
for (int s, retries = MAX_HELP;;) {
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);
}
}
}
/**
* If present, pops and executes the given task, or any other
* cancelled task
*
* @return false if any other non-cancelled task exists in local queue
*/
private boolean localHelpJoinTask(ForkJoinTask<?> joinMe) {
int s, i; ForkJoinTask<?>[] q; ForkJoinTask<?> t;
if ((s = queueTop) != queueBase && (q = queue) != null &&
(i = (q.length - 1) & --s) >= 0 &&
(t = q[i]) != null) {
if (t != joinMe && t.status >= 0)
return false;
if (UNSAFE.compareAndSwapObject
(q, (i << ASHIFT) + ABASE, t, null)) {
queueTop = s; // or putOrderedInt
t.doExec();
}
}
return true;
}
/**
* Tries to locate and execute tasks for a stealer of the given
* 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
outer:for (ForkJoinWorkerThread thread = this;;) {
// Try to find v, the stealer of task, by first using hint
ForkJoinWorkerThread v = ws[thread.stealHint & m];
if (v == null || v.currentSteal != task) {
for (int j = 0; ;) { // search array
if ((v = ws[j]) != null && v.currentSteal == task) {
thread.stealHint = j;
break; // save hint for next time
}
if (++j > m)
break outer; // can't find stealer
}
}
// Try to help v, using specialized form of deqTask
for (;;) {
ForkJoinTask<?>[] q; int b, i;
if (joinMe.status < 0)
break outer;
if ((b = v.queueBase) == v.queueTop ||
(q = v.queue) == null ||
(i = (q.length-1) & b) < 0)
break; // empty
long u = (i << ASHIFT) + ABASE;
ForkJoinTask<?> t = q[i];
if (task.status < 0)
break outer; // stale
if (t != null && 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;
helped = true;
}
}
// Try to descend to find v's stealer
ForkJoinTask<?> next = v.currentJoin;
if (--levels > 0 && task.status >= 0 &&
next != null && next != task) {
task = next;
thread = v;
}
else
break; // max levels, stale, dead-end, or cyclic
}
}
return helped;
}
/**
* Performs an uncommon case for joinTask: If task t is at base of
* some workers queue, steals and executes it.
*
* @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
* partitioning when programmers, frameworks, tools, or languages
* have little or no idea about task granularity. In essence by
* offering this method, we ask users only about tradeoffs in
* overhead vs expected throughput and its variance, rather than
* how finely to partition tasks.
*
* In a steady state strict (tree-structured) computation, each
* thread makes available for stealing enough tasks for other
* threads to remain active. Inductively, if all threads play by
* the same rules, each thread should make available only a
* constant number of tasks.
*
* The minimum useful constant is just 1. But using a value of 1
* would require immediate replenishment upon each steal to
* maintain enough tasks, which is infeasible. Further,
* partitionings/granularities of offered tasks should minimize
* steal rates, which in general means that threads nearer the top
* of computation tree should generate more than those nearer the
* bottom. In perfect steady state, each thread is at
* approximately the same level of computation tree. However,
* producing extra tasks amortizes the uncertainty of progress and
* diffusion assumptions.
*
* So, users will want to use values larger, but not much larger
* than 1 to both smooth over transient shortages and hedge
* against uneven progress; as traded off against the cost of
* extra task overhead. We leave the user to pick a threshold
* value to compare with the results of this call to guide
* decisions, but recommend values such as 3.
*
* When all threads are active, it is on average OK to estimate
* surplus strictly locally. In steady-state, if one thread is
* maintaining say 2 surplus tasks, then so are others. So we can
* just use estimated queue length (although note that (queueTop -
* queueBase) can be an overestimate because of stealers lagging
* increments of queueBase). However, this strategy alone leads
* to serious mis-estimates in some non-steady-state conditions
* (ramp-up, ramp-down, other stalls). We can detect many of these
* by further considering the number of "idle" threads, that are
* known to have zero queued tasks, so compensate by a factor of
* (#idle/#active) threads.
*/
final int getEstimatedSurplusTaskCount() {
return queueTop - queueBase - pool.idlePerActive();
}
/**
* 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() {
boolean active = true;
ForkJoinTask<?> ps = currentSteal; // to restore below
ForkJoinPool p = pool;
p.addQuiescerCount(1);
for (;;) {
ForkJoinWorkerThread[] ws = p.workers;
ForkJoinWorkerThread v = null;
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 {
if (active) {
active = false;
p.addActiveCount(-1);
}
if (p.isQuiescent()) {
p.addActiveCount(1);
p.addQuiescerCount(-1);
break;
}
}
}
}
// Unsafe mechanics
private static final sun.misc.Unsafe UNSAFE;
private static final long ABASE;
private static final int ASHIFT;
static {
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)
throw new Error("data type scale not a power of two");
ASHIFT = 31 - Integer.numberOfLeadingZeros(s);
}
} }
Markdown is supported
0% .
You are about to add 0 people to the discussion. Proceed with caution.
先完成此消息的编辑!
想要评论请 注册