ViewVC Help
View File | Revision Log | Show Annotations | Download File | Root Listing
root/jsr166/jsr166/src/main/java/util/concurrent/ConcurrentHashMap.java
Revision: 1.130
Committed: Sat Sep 29 16:01:28 2012 UTC (11 years, 8 months ago) by dl
Branch: MAIN
Changes since 1.129: +170 -397 lines
Log Message:
Faster task construction

File Contents

# User Rev Content
1 dl 1.2 /*
2     * Written by Doug Lea with assistance from members of JCP JSR-166
3 dl 1.36 * Expert Group and released to the public domain, as explained at
4 dl 1.100 * http://creativecommons.org/publicdomain/zero/1.0/
5 dl 1.2 */
6    
7 tim 1.1 package java.util.concurrent;
8 dl 1.119 import java.util.concurrent.atomic.LongAdder;
9     import java.util.concurrent.ForkJoinPool;
10     import java.util.concurrent.ForkJoinTask;
11    
12     import java.util.Comparator;
13     import java.util.Arrays;
14     import java.util.Map;
15     import java.util.Set;
16     import java.util.Collection;
17     import java.util.AbstractMap;
18     import java.util.AbstractSet;
19     import java.util.AbstractCollection;
20     import java.util.Hashtable;
21     import java.util.HashMap;
22     import java.util.Iterator;
23     import java.util.Enumeration;
24     import java.util.ConcurrentModificationException;
25     import java.util.NoSuchElementException;
26     import java.util.concurrent.ConcurrentMap;
27     import java.util.concurrent.ThreadLocalRandom;
28     import java.util.concurrent.locks.LockSupport;
29     import java.util.concurrent.locks.AbstractQueuedSynchronizer;
30     import java.util.concurrent.atomic.AtomicReference;
31    
32 tim 1.1 import java.io.Serializable;
33    
34     /**
35 dl 1.4 * A hash table supporting full concurrency of retrievals and
36 dl 1.119 * high expected concurrency for updates. This class obeys the
37 dl 1.22 * same functional specification as {@link java.util.Hashtable}, and
38 dl 1.19 * includes versions of methods corresponding to each method of
39 dl 1.119 * {@code Hashtable}. However, even though all operations are
40 dl 1.19 * thread-safe, retrieval operations do <em>not</em> entail locking,
41     * and there is <em>not</em> any support for locking the entire table
42     * in a way that prevents all access. This class is fully
43 dl 1.119 * interoperable with {@code Hashtable} in programs that rely on its
44 dl 1.4 * thread safety but not on its synchronization details.
45 tim 1.11 *
46 dl 1.119 * <p> Retrieval operations (including {@code get}) generally do not
47     * block, so may overlap with update operations (including {@code put}
48     * and {@code remove}). Retrievals reflect the results of the most
49     * recently <em>completed</em> update operations holding upon their
50 dl 1.126 * onset. (More formally, an update operation for a given key bears a
51     * <em>happens-before</em> relation with any (non-null) retrieval for
52     * that key reporting the updated value.) For aggregate operations
53     * such as {@code putAll} and {@code clear}, concurrent retrievals may
54     * reflect insertion or removal of only some entries. Similarly,
55     * Iterators and Enumerations return elements reflecting the state of
56     * the hash table at some point at or since the creation of the
57     * iterator/enumeration. They do <em>not</em> throw {@link
58     * ConcurrentModificationException}. However, iterators are designed
59     * to be used by only one thread at a time. Bear in mind that the
60     * results of aggregate status methods including {@code size}, {@code
61     * isEmpty}, and {@code containsValue} are typically useful only when
62     * a map is not undergoing concurrent updates in other threads.
63     * Otherwise the results of these methods reflect transient states
64     * that may be adequate for monitoring or estimation purposes, but not
65     * for program control.
66 tim 1.1 *
67 dl 1.119 * <p> The table is dynamically expanded when there are too many
68     * collisions (i.e., keys that have distinct hash codes but fall into
69     * the same slot modulo the table size), with the expected average
70     * effect of maintaining roughly two bins per mapping (corresponding
71     * to a 0.75 load factor threshold for resizing). There may be much
72     * variance around this average as mappings are added and removed, but
73     * overall, this maintains a commonly accepted time/space tradeoff for
74     * hash tables. However, resizing this or any other kind of hash
75     * table may be a relatively slow operation. When possible, it is a
76     * good idea to provide a size estimate as an optional {@code
77     * initialCapacity} constructor argument. An additional optional
78     * {@code loadFactor} constructor argument provides a further means of
79     * customizing initial table capacity by specifying the table density
80     * to be used in calculating the amount of space to allocate for the
81     * given number of elements. Also, for compatibility with previous
82     * versions of this class, constructors may optionally specify an
83     * expected {@code concurrencyLevel} as an additional hint for
84     * internal sizing. Note that using many keys with exactly the same
85     * {@code hashCode()} is a sure way to slow down performance of any
86     * hash table.
87 tim 1.1 *
88 dl 1.45 * <p>This class and its views and iterators implement all of the
89     * <em>optional</em> methods of the {@link Map} and {@link Iterator}
90     * interfaces.
91 dl 1.23 *
92 jsr166 1.68 * <p> Like {@link Hashtable} but unlike {@link HashMap}, this class
93 dl 1.119 * does <em>not</em> allow {@code null} to be used as a key or value.
94 tim 1.1 *
95 dl 1.42 * <p>This class is a member of the
96 jsr166 1.88 * <a href="{@docRoot}/../technotes/guides/collections/index.html">
97 dl 1.42 * Java Collections Framework</a>.
98     *
99 dl 1.130 * <p><em> During transition to JDK8, this
100 dl 1.119 * class declares and uses nested functional interfaces with different
101     * names but the same forms as those expected for JDK8.<em>
102     *
103 dl 1.8 * @since 1.5
104     * @author Doug Lea
105 dl 1.27 * @param <K> the type of keys maintained by this map
106 jsr166 1.64 * @param <V> the type of mapped values
107 dl 1.8 */
108 dl 1.119 public class ConcurrentHashMap<K, V>
109     implements ConcurrentMap<K, V>, Serializable {
110 dl 1.20 private static final long serialVersionUID = 7249069246763182397L;
111 tim 1.1
112 dl 1.119 /**
113     * A partitionable iterator. A Spliterator can be traversed
114     * directly, but can also be partitioned (before traversal) by
115     * creating another Spliterator that covers a non-overlapping
116     * portion of the elements, and so may be amenable to parallel
117     * execution.
118     *
119     * <p> This interface exports a subset of expected JDK8
120     * functionality.
121     *
122     * <p>Sample usage: Here is one (of the several) ways to compute
123     * the sum of the values held in a map using the ForkJoin
124     * framework. As illustrated here, Spliterators are well suited to
125     * designs in which a task repeatedly splits off half its work
126     * into forked subtasks until small enough to process directly,
127     * and then joins these subtasks. Variants of this style can also
128     * be used in completion-based designs.
129     *
130     * <pre>
131     * {@code ConcurrentHashMap<String, Long> m = ...
132     * // split as if have 8 * parallelism, for load balance
133     * int n = m.size();
134     * int p = aForkJoinPool.getParallelism() * 8;
135     * int split = (n < p)? n : p;
136     * long sum = aForkJoinPool.invoke(new SumValues(m.valueSpliterator(), split, null));
137     * // ...
138     * static class SumValues extends RecursiveTask<Long> {
139     * final Spliterator<Long> s;
140     * final int split; // split while > 1
141     * final SumValues nextJoin; // records forked subtasks to join
142     * SumValues(Spliterator<Long> s, int depth, SumValues nextJoin) {
143     * this.s = s; this.depth = depth; this.nextJoin = nextJoin;
144     * }
145     * public Long compute() {
146     * long sum = 0;
147     * SumValues subtasks = null; // fork subtasks
148     * for (int s = split >>> 1; s > 0; s >>>= 1)
149     * (subtasks = new SumValues(s.split(), s, subtasks)).fork();
150     * while (s.hasNext()) // directly process remaining elements
151     * sum += s.next();
152     * for (SumValues t = subtasks; t != null; t = t.nextJoin)
153     * sum += t.join(); // collect subtask results
154     * return sum;
155     * }
156     * }
157     * }</pre>
158     */
159     public static interface Spliterator<T> extends Iterator<T> {
160     /**
161     * Returns a Spliterator covering approximately half of the
162     * elements, guaranteed not to overlap with those subsequently
163     * returned by this Spliterator. After invoking this method,
164     * the current Spliterator will <em>not</em> produce any of
165     * the elements of the returned Spliterator, but the two
166     * Spliterators together will produce all of the elements that
167     * would have been produced by this Spliterator had this
168     * method not been called. The exact number of elements
169     * produced by the returned Spliterator is not guaranteed, and
170     * may be zero (i.e., with {@code hasNext()} reporting {@code
171     * false}) if this Spliterator cannot be further split.
172     *
173     * @return a Spliterator covering approximately half of the
174     * elements
175     * @throws IllegalStateException if this Spliterator has
176     * already commenced traversing elements
177     */
178     Spliterator<T> split();
179     }
180    
181 tim 1.1 /*
182 dl 1.119 * Overview:
183     *
184     * The primary design goal of this hash table is to maintain
185     * concurrent readability (typically method get(), but also
186     * iterators and related methods) while minimizing update
187     * contention. Secondary goals are to keep space consumption about
188     * the same or better than java.util.HashMap, and to support high
189     * initial insertion rates on an empty table by many threads.
190     *
191     * Each key-value mapping is held in a Node. Because Node fields
192     * can contain special values, they are defined using plain Object
193     * types. Similarly in turn, all internal methods that use them
194     * work off Object types. And similarly, so do the internal
195     * methods of auxiliary iterator and view classes. All public
196     * generic typed methods relay in/out of these internal methods,
197     * supplying null-checks and casts as needed. This also allows
198     * many of the public methods to be factored into a smaller number
199     * of internal methods (although sadly not so for the five
200     * variants of put-related operations). The validation-based
201     * approach explained below leads to a lot of code sprawl because
202     * retry-control precludes factoring into smaller methods.
203     *
204     * The table is lazily initialized to a power-of-two size upon the
205     * first insertion. Each bin in the table normally contains a
206     * list of Nodes (most often, the list has only zero or one Node).
207     * Table accesses require volatile/atomic reads, writes, and
208     * CASes. Because there is no other way to arrange this without
209     * adding further indirections, we use intrinsics
210     * (sun.misc.Unsafe) operations. The lists of nodes within bins
211     * are always accurately traversable under volatile reads, so long
212     * as lookups check hash code and non-nullness of value before
213     * checking key equality.
214     *
215     * We use the top two bits of Node hash fields for control
216     * purposes -- they are available anyway because of addressing
217     * constraints. As explained further below, these top bits are
218     * used as follows:
219     * 00 - Normal
220     * 01 - Locked
221     * 11 - Locked and may have a thread waiting for lock
222     * 10 - Node is a forwarding node
223     *
224     * The lower 30 bits of each Node's hash field contain a
225     * transformation of the key's hash code, except for forwarding
226     * nodes, for which the lower bits are zero (and so always have
227     * hash field == MOVED).
228     *
229     * Insertion (via put or its variants) of the first node in an
230     * empty bin is performed by just CASing it to the bin. This is
231     * by far the most common case for put operations under most
232     * key/hash distributions. Other update operations (insert,
233     * delete, and replace) require locks. We do not want to waste
234     * the space required to associate a distinct lock object with
235     * each bin, so instead use the first node of a bin list itself as
236     * a lock. Blocking support for these locks relies on the builtin
237     * "synchronized" monitors. However, we also need a tryLock
238     * construction, so we overlay these by using bits of the Node
239     * hash field for lock control (see above), and so normally use
240     * builtin monitors only for blocking and signalling using
241     * wait/notifyAll constructions. See Node.tryAwaitLock.
242     *
243     * Using the first node of a list as a lock does not by itself
244     * suffice though: When a node is locked, any update must first
245     * validate that it is still the first node after locking it, and
246     * retry if not. Because new nodes are always appended to lists,
247     * once a node is first in a bin, it remains first until deleted
248     * or the bin becomes invalidated (upon resizing). However,
249     * operations that only conditionally update may inspect nodes
250     * until the point of update. This is a converse of sorts to the
251     * lazy locking technique described by Herlihy & Shavit.
252     *
253     * The main disadvantage of per-bin locks is that other update
254     * operations on other nodes in a bin list protected by the same
255     * lock can stall, for example when user equals() or mapping
256     * functions take a long time. However, statistically, under
257     * random hash codes, this is not a common problem. Ideally, the
258     * frequency of nodes in bins follows a Poisson distribution
259     * (http://en.wikipedia.org/wiki/Poisson_distribution) with a
260     * parameter of about 0.5 on average, given the resizing threshold
261     * of 0.75, although with a large variance because of resizing
262     * granularity. Ignoring variance, the expected occurrences of
263     * list size k are (exp(-0.5) * pow(0.5, k) / factorial(k)). The
264     * first values are:
265     *
266     * 0: 0.60653066
267     * 1: 0.30326533
268     * 2: 0.07581633
269     * 3: 0.01263606
270     * 4: 0.00157952
271     * 5: 0.00015795
272     * 6: 0.00001316
273     * 7: 0.00000094
274     * 8: 0.00000006
275     * more: less than 1 in ten million
276     *
277     * Lock contention probability for two threads accessing distinct
278     * elements is roughly 1 / (8 * #elements) under random hashes.
279     *
280     * Actual hash code distributions encountered in practice
281     * sometimes deviate significantly from uniform randomness. This
282     * includes the case when N > (1<<30), so some keys MUST collide.
283     * Similarly for dumb or hostile usages in which multiple keys are
284     * designed to have identical hash codes. Also, although we guard
285     * against the worst effects of this (see method spread), sets of
286     * hashes may differ only in bits that do not impact their bin
287     * index for a given power-of-two mask. So we use a secondary
288     * strategy that applies when the number of nodes in a bin exceeds
289     * a threshold, and at least one of the keys implements
290     * Comparable. These TreeBins use a balanced tree to hold nodes
291     * (a specialized form of red-black trees), bounding search time
292     * to O(log N). Each search step in a TreeBin is around twice as
293     * slow as in a regular list, but given that N cannot exceed
294     * (1<<64) (before running out of addresses) this bounds search
295     * steps, lock hold times, etc, to reasonable constants (roughly
296     * 100 nodes inspected per operation worst case) so long as keys
297     * are Comparable (which is very common -- String, Long, etc).
298     * TreeBin nodes (TreeNodes) also maintain the same "next"
299     * traversal pointers as regular nodes, so can be traversed in
300     * iterators in the same way.
301     *
302     * The table is resized when occupancy exceeds a percentage
303     * threshold (nominally, 0.75, but see below). Only a single
304     * thread performs the resize (using field "sizeCtl", to arrange
305     * exclusion), but the table otherwise remains usable for reads
306     * and updates. Resizing proceeds by transferring bins, one by
307     * one, from the table to the next table. Because we are using
308     * power-of-two expansion, the elements from each bin must either
309     * stay at same index, or move with a power of two offset. We
310     * eliminate unnecessary node creation by catching cases where old
311     * nodes can be reused because their next fields won't change. On
312     * average, only about one-sixth of them need cloning when a table
313     * doubles. The nodes they replace will be garbage collectable as
314     * soon as they are no longer referenced by any reader thread that
315     * may be in the midst of concurrently traversing table. Upon
316     * transfer, the old table bin contains only a special forwarding
317     * node (with hash field "MOVED") that contains the next table as
318     * its key. On encountering a forwarding node, access and update
319     * operations restart, using the new table.
320     *
321     * Each bin transfer requires its bin lock. However, unlike other
322     * cases, a transfer can skip a bin if it fails to acquire its
323     * lock, and revisit it later (unless it is a TreeBin). Method
324     * rebuild maintains a buffer of TRANSFER_BUFFER_SIZE bins that
325     * have been skipped because of failure to acquire a lock, and
326     * blocks only if none are available (i.e., only very rarely).
327     * The transfer operation must also ensure that all accessible
328     * bins in both the old and new table are usable by any traversal.
329     * When there are no lock acquisition failures, this is arranged
330     * simply by proceeding from the last bin (table.length - 1) up
331     * towards the first. Upon seeing a forwarding node, traversals
332     * (see class Iter) arrange to move to the new table
333     * without revisiting nodes. However, when any node is skipped
334     * during a transfer, all earlier table bins may have become
335     * visible, so are initialized with a reverse-forwarding node back
336     * to the old table until the new ones are established. (This
337     * sometimes requires transiently locking a forwarding node, which
338     * is possible under the above encoding.) These more expensive
339     * mechanics trigger only when necessary.
340     *
341     * The traversal scheme also applies to partial traversals of
342     * ranges of bins (via an alternate Traverser constructor)
343     * to support partitioned aggregate operations. Also, read-only
344     * operations give up if ever forwarded to a null table, which
345     * provides support for shutdown-style clearing, which is also not
346     * currently implemented.
347     *
348     * Lazy table initialization minimizes footprint until first use,
349     * and also avoids resizings when the first operation is from a
350     * putAll, constructor with map argument, or deserialization.
351     * These cases attempt to override the initial capacity settings,
352     * but harmlessly fail to take effect in cases of races.
353     *
354     * The element count is maintained using a LongAdder, which avoids
355     * contention on updates but can encounter cache thrashing if read
356     * too frequently during concurrent access. To avoid reading so
357     * often, resizing is attempted either when a bin lock is
358     * contended, or upon adding to a bin already holding two or more
359     * nodes (checked before adding in the xIfAbsent methods, after
360     * adding in others). Under uniform hash distributions, the
361     * probability of this occurring at threshold is around 13%,
362     * meaning that only about 1 in 8 puts check threshold (and after
363     * resizing, many fewer do so). But this approximation has high
364     * variance for small table sizes, so we check on any collision
365     * for sizes <= 64. The bulk putAll operation further reduces
366     * contention by only committing count updates upon these size
367     * checks.
368     *
369     * Maintaining API and serialization compatibility with previous
370     * versions of this class introduces several oddities. Mainly: We
371     * leave untouched but unused constructor arguments refering to
372     * concurrencyLevel. We accept a loadFactor constructor argument,
373     * but apply it only to initial table capacity (which is the only
374     * time that we can guarantee to honor it.) We also declare an
375     * unused "Segment" class that is instantiated in minimal form
376     * only when serializing.
377 dl 1.4 */
378 tim 1.1
379 dl 1.4 /* ---------------- Constants -------------- */
380 tim 1.11
381 dl 1.4 /**
382 dl 1.119 * The largest possible table capacity. This value must be
383     * exactly 1<<30 to stay within Java array allocation and indexing
384     * bounds for power of two table sizes, and is further required
385     * because the top two bits of 32bit hash fields are used for
386     * control purposes.
387 dl 1.4 */
388 dl 1.119 private static final int MAXIMUM_CAPACITY = 1 << 30;
389 dl 1.56
390     /**
391 dl 1.119 * The default initial table capacity. Must be a power of 2
392     * (i.e., at least 1) and at most MAXIMUM_CAPACITY.
393 dl 1.56 */
394 dl 1.119 private static final int DEFAULT_CAPACITY = 16;
395 dl 1.56
396     /**
397 dl 1.119 * The largest possible (non-power of two) array size.
398     * Needed by toArray and related methods.
399 jsr166 1.59 */
400 dl 1.119 static final int MAX_ARRAY_SIZE = Integer.MAX_VALUE - 8;
401 tim 1.1
402     /**
403 dl 1.119 * The default concurrency level for this table. Unused but
404     * defined for compatibility with previous versions of this class.
405 dl 1.4 */
406 dl 1.119 private static final int DEFAULT_CONCURRENCY_LEVEL = 16;
407 tim 1.11
408 tim 1.1 /**
409 dl 1.119 * The load factor for this table. Overrides of this value in
410     * constructors affect only the initial table capacity. The
411     * actual floating point value isn't normally used -- it is
412     * simpler to use expressions such as {@code n - (n >>> 2)} for
413     * the associated resizing threshold.
414 dl 1.99 */
415 dl 1.119 private static final float LOAD_FACTOR = 0.75f;
416 dl 1.99
417     /**
418 dl 1.119 * The buffer size for skipped bins during transfers. The
419     * value is arbitrary but should be large enough to avoid
420     * most locking stalls during resizes.
421 dl 1.21 */
422 dl 1.119 private static final int TRANSFER_BUFFER_SIZE = 32;
423 dl 1.21
424 dl 1.46 /**
425 dl 1.119 * The bin count threshold for using a tree rather than list for a
426     * bin. The value reflects the approximate break-even point for
427     * using tree-based operations.
428     */
429     private static final int TREE_THRESHOLD = 8;
430    
431     /*
432     * Encodings for special uses of Node hash fields. See above for
433     * explanation.
434 dl 1.46 */
435 dl 1.119 static final int MOVED = 0x80000000; // hash field for forwarding nodes
436     static final int LOCKED = 0x40000000; // set/tested only as a bit
437     static final int WAITING = 0xc0000000; // both bits set/tested together
438     static final int HASH_BITS = 0x3fffffff; // usable bits of normal node hash
439 dl 1.46
440 dl 1.4 /* ---------------- Fields -------------- */
441 tim 1.1
442     /**
443 dl 1.119 * The array of bins. Lazily initialized upon first insertion.
444     * Size is always a power of two. Accessed directly by iterators.
445 jsr166 1.59 */
446 dl 1.119 transient volatile Node[] table;
447 tim 1.1
448     /**
449 dl 1.119 * The counter maintaining number of elements.
450 jsr166 1.59 */
451 dl 1.119 private transient final LongAdder counter;
452 tim 1.1
453     /**
454 dl 1.119 * Table initialization and resizing control. When negative, the
455     * table is being initialized or resized. Otherwise, when table is
456     * null, holds the initial table size to use upon creation, or 0
457     * for default. After initialization, holds the next element count
458     * value upon which to resize the table.
459 tim 1.1 */
460 dl 1.119 private transient volatile int sizeCtl;
461 dl 1.4
462 dl 1.119 // views
463     private transient KeySet<K,V> keySet;
464     private transient Values<K,V> values;
465     private transient EntrySet<K,V> entrySet;
466    
467     /** For serialization compatibility. Null unless serialized; see below */
468     private Segment<K,V>[] segments;
469    
470     /* ---------------- Table element access -------------- */
471    
472     /*
473     * Volatile access methods are used for table elements as well as
474     * elements of in-progress next table while resizing. Uses are
475     * null checked by callers, and implicitly bounds-checked, relying
476     * on the invariants that tab arrays have non-zero size, and all
477     * indices are masked with (tab.length - 1) which is never
478     * negative and always less than length. Note that, to be correct
479     * wrt arbitrary concurrency errors by users, bounds checks must
480     * operate on local variables, which accounts for some odd-looking
481     * inline assignments below.
482     */
483    
484     static final Node tabAt(Node[] tab, int i) { // used by Iter
485     return (Node)UNSAFE.getObjectVolatile(tab, ((long)i<<ASHIFT)+ABASE);
486     }
487    
488     private static final boolean casTabAt(Node[] tab, int i, Node c, Node v) {
489     return UNSAFE.compareAndSwapObject(tab, ((long)i<<ASHIFT)+ABASE, c, v);
490     }
491    
492     private static final void setTabAt(Node[] tab, int i, Node v) {
493     UNSAFE.putObjectVolatile(tab, ((long)i<<ASHIFT)+ABASE, v);
494     }
495    
496     /* ---------------- Nodes -------------- */
497 dl 1.4
498 dl 1.99 /**
499 dl 1.119 * Key-value entry. Note that this is never exported out as a
500     * user-visible Map.Entry (see MapEntry below). Nodes with a hash
501     * field of MOVED are special, and do not contain user keys or
502     * values. Otherwise, keys are never null, and null val fields
503     * indicate that a node is in the process of being deleted or
504     * created. For purposes of read-only access, a key may be read
505     * before a val, but can only be used after checking val to be
506     * non-null.
507 dl 1.99 */
508 dl 1.119 static class Node {
509     volatile int hash;
510     final Object key;
511     volatile Object val;
512     volatile Node next;
513 dl 1.99
514 dl 1.119 Node(int hash, Object key, Object val, Node next) {
515 dl 1.99 this.hash = hash;
516     this.key = key;
517 dl 1.119 this.val = val;
518 dl 1.99 this.next = next;
519     }
520    
521 dl 1.119 /** CompareAndSet the hash field */
522     final boolean casHash(int cmp, int val) {
523     return UNSAFE.compareAndSwapInt(this, hashOffset, cmp, val);
524     }
525    
526     /** The number of spins before blocking for a lock */
527     static final int MAX_SPINS =
528     Runtime.getRuntime().availableProcessors() > 1 ? 64 : 1;
529    
530 dl 1.99 /**
531 dl 1.119 * Spins a while if LOCKED bit set and this node is the first
532     * of its bin, and then sets WAITING bits on hash field and
533     * blocks (once) if they are still set. It is OK for this
534     * method to return even if lock is not available upon exit,
535     * which enables these simple single-wait mechanics.
536     *
537     * The corresponding signalling operation is performed within
538     * callers: Upon detecting that WAITING has been set when
539     * unlocking lock (via a failed CAS from non-waiting LOCKED
540     * state), unlockers acquire the sync lock and perform a
541     * notifyAll.
542 dl 1.128 *
543     * The initial sanity check on tab and bounds is not currently
544     * necessary in the only usages of this method, but enables
545     * use in other future contexts.
546 dl 1.99 */
547 dl 1.119 final void tryAwaitLock(Node[] tab, int i) {
548 dl 1.128 if (tab != null && i >= 0 && i < tab.length) { // sanity check
549 dl 1.119 int r = ThreadLocalRandom.current().nextInt(); // randomize spins
550     int spins = MAX_SPINS, h;
551     while (tabAt(tab, i) == this && ((h = hash) & LOCKED) != 0) {
552     if (spins >= 0) {
553     r ^= r << 1; r ^= r >>> 3; r ^= r << 10; // xorshift
554     if (r >= 0 && --spins == 0)
555     Thread.yield(); // yield before block
556     }
557     else if (casHash(h, h | WAITING)) {
558     synchronized (this) {
559     if (tabAt(tab, i) == this &&
560     (hash & WAITING) == WAITING) {
561     try {
562     wait();
563     } catch (InterruptedException ie) {
564     Thread.currentThread().interrupt();
565     }
566     }
567     else
568     notifyAll(); // possibly won race vs signaller
569     }
570     break;
571     }
572     }
573     }
574 dl 1.99 }
575    
576 dl 1.119 // Unsafe mechanics for casHash
577     private static final sun.misc.Unsafe UNSAFE;
578     private static final long hashOffset;
579    
580 dl 1.99 static {
581     try {
582     UNSAFE = sun.misc.Unsafe.getUnsafe();
583 dl 1.119 Class<?> k = Node.class;
584     hashOffset = UNSAFE.objectFieldOffset
585     (k.getDeclaredField("hash"));
586 dl 1.99 } catch (Exception e) {
587     throw new Error(e);
588     }
589     }
590     }
591    
592 dl 1.119 /* ---------------- TreeBins -------------- */
593    
594 dl 1.99 /**
595 dl 1.119 * Nodes for use in TreeBins
596 dl 1.99 */
597 dl 1.119 static final class TreeNode extends Node {
598     TreeNode parent; // red-black tree links
599     TreeNode left;
600     TreeNode right;
601     TreeNode prev; // needed to unlink next upon deletion
602     boolean red;
603 dl 1.99
604 dl 1.119 TreeNode(int hash, Object key, Object val, Node next, TreeNode parent) {
605     super(hash, key, val, next);
606     this.parent = parent;
607     }
608 dl 1.99 }
609 tim 1.1
610     /**
611 dl 1.119 * A specialized form of red-black tree for use in bins
612     * whose size exceeds a threshold.
613     *
614     * TreeBins use a special form of comparison for search and
615     * related operations (which is the main reason we cannot use
616     * existing collections such as TreeMaps). TreeBins contain
617     * Comparable elements, but may contain others, as well as
618     * elements that are Comparable but not necessarily Comparable<T>
619     * for the same T, so we cannot invoke compareTo among them. To
620     * handle this, the tree is ordered primarily by hash value, then
621     * by getClass().getName() order, and then by Comparator order
622     * among elements of the same class. On lookup at a node, if
623     * elements are not comparable or compare as 0, both left and
624     * right children may need to be searched in the case of tied hash
625     * values. (This corresponds to the full list search that would be
626     * necessary if all elements were non-Comparable and had tied
627     * hashes.) The red-black balancing code is updated from
628     * pre-jdk-collections
629     * (http://gee.cs.oswego.edu/dl/classes/collections/RBCell.java)
630     * based in turn on Cormen, Leiserson, and Rivest "Introduction to
631     * Algorithms" (CLR).
632     *
633     * TreeBins also maintain a separate locking discipline than
634     * regular bins. Because they are forwarded via special MOVED
635     * nodes at bin heads (which can never change once established),
636     * we cannot use those nodes as locks. Instead, TreeBin
637     * extends AbstractQueuedSynchronizer to support a simple form of
638     * read-write lock. For update operations and table validation,
639     * the exclusive form of lock behaves in the same way as bin-head
640     * locks. However, lookups use shared read-lock mechanics to allow
641     * multiple readers in the absence of writers. Additionally,
642     * these lookups do not ever block: While the lock is not
643     * available, they proceed along the slow traversal path (via
644     * next-pointers) until the lock becomes available or the list is
645     * exhausted, whichever comes first. (These cases are not fast,
646     * but maximize aggregate expected throughput.) The AQS mechanics
647     * for doing this are straightforward. The lock state is held as
648     * AQS getState(). Read counts are negative; the write count (1)
649     * is positive. There are no signalling preferences among readers
650     * and writers. Since we don't need to export full Lock API, we
651     * just override the minimal AQS methods and use them directly.
652     */
653     static final class TreeBin extends AbstractQueuedSynchronizer {
654 dl 1.24 private static final long serialVersionUID = 2249069246763182397L;
655 dl 1.119 transient TreeNode root; // root of tree
656     transient TreeNode first; // head of next-pointer list
657 dl 1.24
658 dl 1.119 /* AQS overrides */
659     public final boolean isHeldExclusively() { return getState() > 0; }
660     public final boolean tryAcquire(int ignore) {
661     if (compareAndSetState(0, 1)) {
662     setExclusiveOwnerThread(Thread.currentThread());
663     return true;
664     }
665     return false;
666     }
667     public final boolean tryRelease(int ignore) {
668     setExclusiveOwnerThread(null);
669     setState(0);
670     return true;
671     }
672     public final int tryAcquireShared(int ignore) {
673     for (int c;;) {
674     if ((c = getState()) > 0)
675     return -1;
676     if (compareAndSetState(c, c -1))
677     return 1;
678     }
679     }
680     public final boolean tryReleaseShared(int ignore) {
681     int c;
682     do {} while (!compareAndSetState(c = getState(), c + 1));
683     return c == -1;
684     }
685    
686     /** From CLR */
687     private void rotateLeft(TreeNode p) {
688     if (p != null) {
689     TreeNode r = p.right, pp, rl;
690     if ((rl = p.right = r.left) != null)
691     rl.parent = p;
692     if ((pp = r.parent = p.parent) == null)
693     root = r;
694     else if (pp.left == p)
695     pp.left = r;
696     else
697     pp.right = r;
698     r.left = p;
699     p.parent = r;
700     }
701     }
702 dl 1.4
703 dl 1.119 /** From CLR */
704     private void rotateRight(TreeNode p) {
705     if (p != null) {
706     TreeNode l = p.left, pp, lr;
707     if ((lr = p.left = l.right) != null)
708     lr.parent = p;
709     if ((pp = l.parent = p.parent) == null)
710     root = l;
711     else if (pp.right == p)
712     pp.right = l;
713     else
714     pp.left = l;
715     l.right = p;
716     p.parent = l;
717     }
718     }
719 dl 1.4
720     /**
721 jsr166 1.123 * Returns the TreeNode (or null if not found) for the given key
722 dl 1.119 * starting at given root.
723 dl 1.4 */
724 dl 1.128 @SuppressWarnings("unchecked") final TreeNode getTreeNode
725     (int h, Object k, TreeNode p) {
726 dl 1.119 Class<?> c = k.getClass();
727     while (p != null) {
728     int dir, ph; Object pk; Class<?> pc;
729     if ((ph = p.hash) == h) {
730     if ((pk = p.key) == k || k.equals(pk))
731     return p;
732     if (c != (pc = pk.getClass()) ||
733     !(k instanceof Comparable) ||
734     (dir = ((Comparable)k).compareTo((Comparable)pk)) == 0) {
735     dir = (c == pc) ? 0 : c.getName().compareTo(pc.getName());
736     TreeNode r = null, s = null, pl, pr;
737     if (dir >= 0) {
738     if ((pl = p.left) != null && h <= pl.hash)
739     s = pl;
740 dl 1.99 }
741 dl 1.119 else if ((pr = p.right) != null && h >= pr.hash)
742     s = pr;
743     if (s != null && (r = getTreeNode(h, k, s)) != null)
744     return r;
745 dl 1.45 }
746     }
747 dl 1.119 else
748     dir = (h < ph) ? -1 : 1;
749     p = (dir > 0) ? p.right : p.left;
750 dl 1.33 }
751 dl 1.119 return null;
752 dl 1.33 }
753    
754 dl 1.99 /**
755 dl 1.119 * Wrapper for getTreeNode used by CHM.get. Tries to obtain
756     * read-lock to call getTreeNode, but during failure to get
757     * lock, searches along next links.
758 dl 1.99 */
759 dl 1.119 final Object getValue(int h, Object k) {
760     Node r = null;
761     int c = getState(); // Must read lock state first
762     for (Node e = first; e != null; e = e.next) {
763     if (c <= 0 && compareAndSetState(c, c - 1)) {
764     try {
765     r = getTreeNode(h, k, root);
766     } finally {
767     releaseShared(0);
768 dl 1.99 }
769     break;
770     }
771 dl 1.119 else if ((e.hash & HASH_BITS) == h && k.equals(e.key)) {
772     r = e;
773 dl 1.99 break;
774     }
775 dl 1.119 else
776     c = getState();
777 dl 1.99 }
778 dl 1.119 return r == null ? null : r.val;
779 dl 1.99 }
780    
781     /**
782 dl 1.119 * Finds or adds a node.
783     * @return null if added
784 dl 1.6 */
785 dl 1.128 @SuppressWarnings("unchecked") final TreeNode putTreeNode
786     (int h, Object k, Object v) {
787 dl 1.119 Class<?> c = k.getClass();
788     TreeNode pp = root, p = null;
789     int dir = 0;
790     while (pp != null) { // find existing node or leaf to insert at
791     int ph; Object pk; Class<?> pc;
792     p = pp;
793     if ((ph = p.hash) == h) {
794     if ((pk = p.key) == k || k.equals(pk))
795     return p;
796     if (c != (pc = pk.getClass()) ||
797     !(k instanceof Comparable) ||
798     (dir = ((Comparable)k).compareTo((Comparable)pk)) == 0) {
799     dir = (c == pc) ? 0 : c.getName().compareTo(pc.getName());
800     TreeNode r = null, s = null, pl, pr;
801     if (dir >= 0) {
802     if ((pl = p.left) != null && h <= pl.hash)
803     s = pl;
804 dl 1.99 }
805 dl 1.119 else if ((pr = p.right) != null && h >= pr.hash)
806     s = pr;
807     if (s != null && (r = getTreeNode(h, k, s)) != null)
808     return r;
809 dl 1.99 }
810     }
811 dl 1.119 else
812     dir = (h < ph) ? -1 : 1;
813     pp = (dir > 0) ? p.right : p.left;
814 dl 1.99 }
815    
816 dl 1.119 TreeNode f = first;
817     TreeNode x = first = new TreeNode(h, k, v, f, p);
818     if (p == null)
819     root = x;
820     else { // attach and rebalance; adapted from CLR
821     TreeNode xp, xpp;
822     if (f != null)
823     f.prev = x;
824     if (dir <= 0)
825     p.left = x;
826     else
827     p.right = x;
828     x.red = true;
829     while (x != null && (xp = x.parent) != null && xp.red &&
830     (xpp = xp.parent) != null) {
831     TreeNode xppl = xpp.left;
832     if (xp == xppl) {
833     TreeNode y = xpp.right;
834     if (y != null && y.red) {
835     y.red = false;
836     xp.red = false;
837     xpp.red = true;
838     x = xpp;
839     }
840     else {
841     if (x == xp.right) {
842     rotateLeft(x = xp);
843     xpp = (xp = x.parent) == null ? null : xp.parent;
844     }
845     if (xp != null) {
846     xp.red = false;
847     if (xpp != null) {
848     xpp.red = true;
849     rotateRight(xpp);
850     }
851     }
852     }
853     }
854     else {
855     TreeNode y = xppl;
856     if (y != null && y.red) {
857     y.red = false;
858     xp.red = false;
859     xpp.red = true;
860     x = xpp;
861     }
862     else {
863     if (x == xp.left) {
864     rotateRight(x = xp);
865     xpp = (xp = x.parent) == null ? null : xp.parent;
866     }
867     if (xp != null) {
868     xp.red = false;
869     if (xpp != null) {
870     xpp.red = true;
871     rotateLeft(xpp);
872     }
873     }
874 dl 1.99 }
875     }
876     }
877 dl 1.119 TreeNode r = root;
878     if (r != null && r.red)
879     r.red = false;
880 dl 1.99 }
881 dl 1.119 return null;
882 dl 1.99 }
883 dl 1.45
884 dl 1.119 /**
885     * Removes the given node, that must be present before this
886     * call. This is messier than typical red-black deletion code
887     * because we cannot swap the contents of an interior node
888     * with a leaf successor that is pinned by "next" pointers
889     * that are accessible independently of lock. So instead we
890     * swap the tree linkages.
891     */
892     final void deleteTreeNode(TreeNode p) {
893     TreeNode next = (TreeNode)p.next; // unlink traversal pointers
894     TreeNode pred = p.prev;
895     if (pred == null)
896     first = next;
897     else
898     pred.next = next;
899     if (next != null)
900     next.prev = pred;
901     TreeNode replacement;
902     TreeNode pl = p.left;
903     TreeNode pr = p.right;
904     if (pl != null && pr != null) {
905     TreeNode s = pr, sl;
906     while ((sl = s.left) != null) // find successor
907     s = sl;
908     boolean c = s.red; s.red = p.red; p.red = c; // swap colors
909     TreeNode sr = s.right;
910     TreeNode pp = p.parent;
911     if (s == pr) { // p was s's direct parent
912     p.parent = s;
913     s.right = p;
914     }
915     else {
916     TreeNode sp = s.parent;
917     if ((p.parent = sp) != null) {
918     if (s == sp.left)
919     sp.left = p;
920     else
921     sp.right = p;
922 dl 1.45 }
923 dl 1.119 if ((s.right = pr) != null)
924     pr.parent = s;
925 dl 1.4 }
926 dl 1.119 p.left = null;
927     if ((p.right = sr) != null)
928     sr.parent = p;
929     if ((s.left = pl) != null)
930     pl.parent = s;
931     if ((s.parent = pp) == null)
932     root = s;
933     else if (p == pp.left)
934     pp.left = s;
935     else
936     pp.right = s;
937     replacement = sr;
938     }
939     else
940     replacement = (pl != null) ? pl : pr;
941     TreeNode pp = p.parent;
942     if (replacement == null) {
943     if (pp == null) {
944     root = null;
945     return;
946     }
947     replacement = p;
948 dl 1.99 }
949 dl 1.119 else {
950     replacement.parent = pp;
951     if (pp == null)
952     root = replacement;
953     else if (p == pp.left)
954     pp.left = replacement;
955     else
956     pp.right = replacement;
957     p.left = p.right = p.parent = null;
958 dl 1.4 }
959 dl 1.119 if (!p.red) { // rebalance, from CLR
960     TreeNode x = replacement;
961     while (x != null) {
962     TreeNode xp, xpl;
963     if (x.red || (xp = x.parent) == null) {
964     x.red = false;
965 dl 1.99 break;
966 dl 1.119 }
967     if (x == (xpl = xp.left)) {
968     TreeNode sib = xp.right;
969     if (sib != null && sib.red) {
970     sib.red = false;
971     xp.red = true;
972     rotateLeft(xp);
973     sib = (xp = x.parent) == null ? null : xp.right;
974     }
975     if (sib == null)
976     x = xp;
977     else {
978     TreeNode sl = sib.left, sr = sib.right;
979     if ((sr == null || !sr.red) &&
980     (sl == null || !sl.red)) {
981     sib.red = true;
982     x = xp;
983     }
984     else {
985     if (sr == null || !sr.red) {
986     if (sl != null)
987     sl.red = false;
988     sib.red = true;
989     rotateRight(sib);
990     sib = (xp = x.parent) == null ? null : xp.right;
991     }
992     if (sib != null) {
993     sib.red = (xp == null) ? false : xp.red;
994     if ((sr = sib.right) != null)
995     sr.red = false;
996     }
997     if (xp != null) {
998     xp.red = false;
999     rotateLeft(xp);
1000     }
1001     x = root;
1002     }
1003     }
1004     }
1005     else { // symmetric
1006     TreeNode sib = xpl;
1007     if (sib != null && sib.red) {
1008     sib.red = false;
1009     xp.red = true;
1010     rotateRight(xp);
1011     sib = (xp = x.parent) == null ? null : xp.left;
1012     }
1013     if (sib == null)
1014     x = xp;
1015     else {
1016     TreeNode sl = sib.left, sr = sib.right;
1017     if ((sl == null || !sl.red) &&
1018     (sr == null || !sr.red)) {
1019     sib.red = true;
1020     x = xp;
1021     }
1022     else {
1023     if (sl == null || !sl.red) {
1024     if (sr != null)
1025     sr.red = false;
1026     sib.red = true;
1027     rotateLeft(sib);
1028     sib = (xp = x.parent) == null ? null : xp.left;
1029     }
1030     if (sib != null) {
1031     sib.red = (xp == null) ? false : xp.red;
1032     if ((sl = sib.left) != null)
1033     sl.red = false;
1034     }
1035     if (xp != null) {
1036     xp.red = false;
1037     rotateRight(xp);
1038     }
1039     x = root;
1040     }
1041     }
1042     }
1043 dl 1.45 }
1044 dl 1.4 }
1045 dl 1.119 if (p == replacement && (pp = p.parent) != null) {
1046     if (p == pp.left) // detach pointers
1047     pp.left = null;
1048     else if (p == pp.right)
1049     pp.right = null;
1050     p.parent = null;
1051     }
1052 dl 1.4 }
1053 tim 1.1 }
1054    
1055 dl 1.119 /* ---------------- Collision reduction methods -------------- */
1056 tim 1.1
1057 dl 1.99 /**
1058 dl 1.119 * Spreads higher bits to lower, and also forces top 2 bits to 0.
1059     * Because the table uses power-of-two masking, sets of hashes
1060     * that vary only in bits above the current mask will always
1061     * collide. (Among known examples are sets of Float keys holding
1062     * consecutive whole numbers in small tables.) To counter this,
1063     * we apply a transform that spreads the impact of higher bits
1064     * downward. There is a tradeoff between speed, utility, and
1065     * quality of bit-spreading. Because many common sets of hashes
1066     * are already reasonably distributed across bits (so don't benefit
1067     * from spreading), and because we use trees to handle large sets
1068     * of collisions in bins, we don't need excessively high quality.
1069     */
1070     private static final int spread(int h) {
1071     h ^= (h >>> 18) ^ (h >>> 12);
1072     return (h ^ (h >>> 10)) & HASH_BITS;
1073 dl 1.99 }
1074    
1075     /**
1076 dl 1.119 * Replaces a list bin with a tree bin. Call only when locked.
1077     * Fails to replace if the given key is non-comparable or table
1078     * is, or needs, resizing.
1079     */
1080     private final void replaceWithTreeBin(Node[] tab, int index, Object key) {
1081     if ((key instanceof Comparable) &&
1082     (tab.length >= MAXIMUM_CAPACITY || counter.sum() < (long)sizeCtl)) {
1083     TreeBin t = new TreeBin();
1084     for (Node e = tabAt(tab, index); e != null; e = e.next)
1085     t.putTreeNode(e.hash & HASH_BITS, e.key, e.val);
1086     setTabAt(tab, index, new Node(MOVED, t, null, null));
1087     }
1088 dl 1.99 }
1089 tim 1.11
1090 dl 1.119 /* ---------------- Internal access and update methods -------------- */
1091    
1092     /** Implementation for get and containsKey */
1093     private final Object internalGet(Object k) {
1094     int h = spread(k.hashCode());
1095     retry: for (Node[] tab = table; tab != null;) {
1096     Node e, p; Object ek, ev; int eh; // locals to read fields once
1097     for (e = tabAt(tab, (tab.length - 1) & h); e != null; e = e.next) {
1098     if ((eh = e.hash) == MOVED) {
1099     if ((ek = e.key) instanceof TreeBin) // search TreeBin
1100     return ((TreeBin)ek).getValue(h, k);
1101     else { // restart with new table
1102     tab = (Node[])ek;
1103     continue retry;
1104     }
1105     }
1106     else if ((eh & HASH_BITS) == h && (ev = e.val) != null &&
1107     ((ek = e.key) == k || k.equals(ek)))
1108     return ev;
1109     }
1110     break;
1111     }
1112     return null;
1113 tim 1.1 }
1114    
1115     /**
1116 dl 1.119 * Implementation for the four public remove/replace methods:
1117     * Replaces node value with v, conditional upon match of cv if
1118     * non-null. If resulting value is null, delete.
1119     */
1120     private final Object internalReplace(Object k, Object v, Object cv) {
1121     int h = spread(k.hashCode());
1122     Object oldVal = null;
1123     for (Node[] tab = table;;) {
1124     Node f; int i, fh; Object fk;
1125     if (tab == null ||
1126     (f = tabAt(tab, i = (tab.length - 1) & h)) == null)
1127     break;
1128     else if ((fh = f.hash) == MOVED) {
1129     if ((fk = f.key) instanceof TreeBin) {
1130     TreeBin t = (TreeBin)fk;
1131     boolean validated = false;
1132     boolean deleted = false;
1133     t.acquire(0);
1134     try {
1135     if (tabAt(tab, i) == f) {
1136     validated = true;
1137     TreeNode p = t.getTreeNode(h, k, t.root);
1138     if (p != null) {
1139     Object pv = p.val;
1140     if (cv == null || cv == pv || cv.equals(pv)) {
1141     oldVal = pv;
1142     if ((p.val = v) == null) {
1143     deleted = true;
1144     t.deleteTreeNode(p);
1145     }
1146     }
1147     }
1148     }
1149     } finally {
1150     t.release(0);
1151     }
1152     if (validated) {
1153     if (deleted)
1154     counter.add(-1L);
1155     break;
1156     }
1157     }
1158     else
1159     tab = (Node[])fk;
1160     }
1161     else if ((fh & HASH_BITS) != h && f.next == null) // precheck
1162     break; // rules out possible existence
1163     else if ((fh & LOCKED) != 0) {
1164     checkForResize(); // try resizing if can't get lock
1165     f.tryAwaitLock(tab, i);
1166     }
1167     else if (f.casHash(fh, fh | LOCKED)) {
1168     boolean validated = false;
1169     boolean deleted = false;
1170     try {
1171     if (tabAt(tab, i) == f) {
1172     validated = true;
1173     for (Node e = f, pred = null;;) {
1174     Object ek, ev;
1175     if ((e.hash & HASH_BITS) == h &&
1176     ((ev = e.val) != null) &&
1177     ((ek = e.key) == k || k.equals(ek))) {
1178     if (cv == null || cv == ev || cv.equals(ev)) {
1179     oldVal = ev;
1180     if ((e.val = v) == null) {
1181     deleted = true;
1182     Node en = e.next;
1183     if (pred != null)
1184     pred.next = en;
1185     else
1186     setTabAt(tab, i, en);
1187     }
1188     }
1189     break;
1190     }
1191     pred = e;
1192     if ((e = e.next) == null)
1193     break;
1194     }
1195     }
1196     } finally {
1197     if (!f.casHash(fh | LOCKED, fh)) {
1198     f.hash = fh;
1199     synchronized (f) { f.notifyAll(); };
1200     }
1201     }
1202     if (validated) {
1203     if (deleted)
1204     counter.add(-1L);
1205     break;
1206     }
1207     }
1208     }
1209     return oldVal;
1210 dl 1.55 }
1211    
1212 dl 1.119 /*
1213 dl 1.126 * Internal versions of the six insertion methods, each a
1214 dl 1.119 * little more complicated than the last. All have
1215     * the same basic structure as the first (internalPut):
1216     * 1. If table uninitialized, create
1217     * 2. If bin empty, try to CAS new node
1218     * 3. If bin stale, use new table
1219     * 4. if bin converted to TreeBin, validate and relay to TreeBin methods
1220     * 5. Lock and validate; if valid, scan and add or update
1221 tim 1.1 *
1222 dl 1.119 * The others interweave other checks and/or alternative actions:
1223     * * Plain put checks for and performs resize after insertion.
1224     * * putIfAbsent prescans for mapping without lock (and fails to add
1225     * if present), which also makes pre-emptive resize checks worthwhile.
1226     * * computeIfAbsent extends form used in putIfAbsent with additional
1227     * mechanics to deal with, calls, potential exceptions and null
1228     * returns from function call.
1229     * * compute uses the same function-call mechanics, but without
1230     * the prescans
1231 dl 1.126 * * merge acts as putIfAbsent in the absent case, but invokes the
1232     * update function if present
1233 dl 1.119 * * putAll attempts to pre-allocate enough table space
1234     * and more lazily performs count updates and checks.
1235 jsr166 1.76 *
1236 dl 1.119 * Someday when details settle down a bit more, it might be worth
1237     * some factoring to reduce sprawl.
1238 tim 1.1 */
1239    
1240 dl 1.119 /** Implementation for put */
1241     private final Object internalPut(Object k, Object v) {
1242     int h = spread(k.hashCode());
1243     int count = 0;
1244     for (Node[] tab = table;;) {
1245     int i; Node f; int fh; Object fk;
1246     if (tab == null)
1247     tab = initTable();
1248     else if ((f = tabAt(tab, i = (tab.length - 1) & h)) == null) {
1249     if (casTabAt(tab, i, null, new Node(h, k, v, null)))
1250     break; // no lock when adding to empty bin
1251 dl 1.99 }
1252 dl 1.119 else if ((fh = f.hash) == MOVED) {
1253     if ((fk = f.key) instanceof TreeBin) {
1254     TreeBin t = (TreeBin)fk;
1255     Object oldVal = null;
1256     t.acquire(0);
1257     try {
1258     if (tabAt(tab, i) == f) {
1259     count = 2;
1260     TreeNode p = t.putTreeNode(h, k, v);
1261     if (p != null) {
1262     oldVal = p.val;
1263     p.val = v;
1264     }
1265     }
1266     } finally {
1267     t.release(0);
1268     }
1269     if (count != 0) {
1270     if (oldVal != null)
1271     return oldVal;
1272     break;
1273     }
1274     }
1275     else
1276     tab = (Node[])fk;
1277     }
1278     else if ((fh & LOCKED) != 0) {
1279     checkForResize();
1280     f.tryAwaitLock(tab, i);
1281     }
1282     else if (f.casHash(fh, fh | LOCKED)) {
1283     Object oldVal = null;
1284     try { // needed in case equals() throws
1285     if (tabAt(tab, i) == f) {
1286     count = 1;
1287     for (Node e = f;; ++count) {
1288     Object ek, ev;
1289     if ((e.hash & HASH_BITS) == h &&
1290     (ev = e.val) != null &&
1291     ((ek = e.key) == k || k.equals(ek))) {
1292     oldVal = ev;
1293     e.val = v;
1294     break;
1295     }
1296     Node last = e;
1297     if ((e = e.next) == null) {
1298     last.next = new Node(h, k, v, null);
1299     if (count >= TREE_THRESHOLD)
1300     replaceWithTreeBin(tab, i, k);
1301     break;
1302     }
1303     }
1304     }
1305     } finally { // unlock and signal if needed
1306     if (!f.casHash(fh | LOCKED, fh)) {
1307     f.hash = fh;
1308     synchronized (f) { f.notifyAll(); };
1309     }
1310     }
1311     if (count != 0) {
1312     if (oldVal != null)
1313     return oldVal;
1314     if (tab.length <= 64)
1315     count = 2;
1316     break;
1317 dl 1.99 }
1318 dl 1.21 }
1319     }
1320 dl 1.119 counter.add(1L);
1321     if (count > 1)
1322     checkForResize();
1323     return null;
1324 tim 1.1 }
1325    
1326 dl 1.119 /** Implementation for putIfAbsent */
1327     private final Object internalPutIfAbsent(Object k, Object v) {
1328     int h = spread(k.hashCode());
1329     int count = 0;
1330     for (Node[] tab = table;;) {
1331     int i; Node f; int fh; Object fk, fv;
1332     if (tab == null)
1333     tab = initTable();
1334     else if ((f = tabAt(tab, i = (tab.length - 1) & h)) == null) {
1335     if (casTabAt(tab, i, null, new Node(h, k, v, null)))
1336     break;
1337     }
1338     else if ((fh = f.hash) == MOVED) {
1339     if ((fk = f.key) instanceof TreeBin) {
1340     TreeBin t = (TreeBin)fk;
1341     Object oldVal = null;
1342     t.acquire(0);
1343     try {
1344     if (tabAt(tab, i) == f) {
1345     count = 2;
1346     TreeNode p = t.putTreeNode(h, k, v);
1347     if (p != null)
1348     oldVal = p.val;
1349     }
1350     } finally {
1351     t.release(0);
1352     }
1353     if (count != 0) {
1354     if (oldVal != null)
1355     return oldVal;
1356     break;
1357     }
1358     }
1359     else
1360     tab = (Node[])fk;
1361     }
1362     else if ((fh & HASH_BITS) == h && (fv = f.val) != null &&
1363     ((fk = f.key) == k || k.equals(fk)))
1364     return fv;
1365     else {
1366     Node g = f.next;
1367     if (g != null) { // at least 2 nodes -- search and maybe resize
1368     for (Node e = g;;) {
1369     Object ek, ev;
1370     if ((e.hash & HASH_BITS) == h && (ev = e.val) != null &&
1371     ((ek = e.key) == k || k.equals(ek)))
1372     return ev;
1373     if ((e = e.next) == null) {
1374     checkForResize();
1375     break;
1376     }
1377     }
1378     }
1379     if (((fh = f.hash) & LOCKED) != 0) {
1380     checkForResize();
1381     f.tryAwaitLock(tab, i);
1382     }
1383     else if (tabAt(tab, i) == f && f.casHash(fh, fh | LOCKED)) {
1384     Object oldVal = null;
1385     try {
1386     if (tabAt(tab, i) == f) {
1387     count = 1;
1388     for (Node e = f;; ++count) {
1389     Object ek, ev;
1390     if ((e.hash & HASH_BITS) == h &&
1391     (ev = e.val) != null &&
1392     ((ek = e.key) == k || k.equals(ek))) {
1393     oldVal = ev;
1394     break;
1395     }
1396     Node last = e;
1397     if ((e = e.next) == null) {
1398     last.next = new Node(h, k, v, null);
1399     if (count >= TREE_THRESHOLD)
1400     replaceWithTreeBin(tab, i, k);
1401     break;
1402     }
1403     }
1404     }
1405     } finally {
1406     if (!f.casHash(fh | LOCKED, fh)) {
1407     f.hash = fh;
1408     synchronized (f) { f.notifyAll(); };
1409     }
1410     }
1411     if (count != 0) {
1412     if (oldVal != null)
1413     return oldVal;
1414     if (tab.length <= 64)
1415     count = 2;
1416     break;
1417     }
1418     }
1419     }
1420 dl 1.45 }
1421 dl 1.119 counter.add(1L);
1422     if (count > 1)
1423     checkForResize();
1424     return null;
1425 dl 1.21 }
1426    
1427 dl 1.119 /** Implementation for computeIfAbsent */
1428     private final Object internalComputeIfAbsent(K k,
1429     Fun<? super K, ?> mf) {
1430     int h = spread(k.hashCode());
1431     Object val = null;
1432     int count = 0;
1433     for (Node[] tab = table;;) {
1434     Node f; int i, fh; Object fk, fv;
1435     if (tab == null)
1436     tab = initTable();
1437     else if ((f = tabAt(tab, i = (tab.length - 1) & h)) == null) {
1438     Node node = new Node(fh = h | LOCKED, k, null, null);
1439     if (casTabAt(tab, i, null, node)) {
1440     count = 1;
1441     try {
1442     if ((val = mf.apply(k)) != null)
1443     node.val = val;
1444     } finally {
1445     if (val == null)
1446     setTabAt(tab, i, null);
1447     if (!node.casHash(fh, h)) {
1448     node.hash = h;
1449     synchronized (node) { node.notifyAll(); };
1450     }
1451     }
1452     }
1453     if (count != 0)
1454     break;
1455     }
1456     else if ((fh = f.hash) == MOVED) {
1457     if ((fk = f.key) instanceof TreeBin) {
1458     TreeBin t = (TreeBin)fk;
1459     boolean added = false;
1460     t.acquire(0);
1461     try {
1462     if (tabAt(tab, i) == f) {
1463     count = 1;
1464     TreeNode p = t.getTreeNode(h, k, t.root);
1465     if (p != null)
1466     val = p.val;
1467     else if ((val = mf.apply(k)) != null) {
1468     added = true;
1469     count = 2;
1470     t.putTreeNode(h, k, val);
1471     }
1472     }
1473     } finally {
1474     t.release(0);
1475     }
1476     if (count != 0) {
1477     if (!added)
1478     return val;
1479     break;
1480     }
1481     }
1482     else
1483     tab = (Node[])fk;
1484     }
1485     else if ((fh & HASH_BITS) == h && (fv = f.val) != null &&
1486     ((fk = f.key) == k || k.equals(fk)))
1487     return fv;
1488     else {
1489     Node g = f.next;
1490     if (g != null) {
1491     for (Node e = g;;) {
1492     Object ek, ev;
1493     if ((e.hash & HASH_BITS) == h && (ev = e.val) != null &&
1494     ((ek = e.key) == k || k.equals(ek)))
1495     return ev;
1496     if ((e = e.next) == null) {
1497     checkForResize();
1498     break;
1499     }
1500     }
1501     }
1502     if (((fh = f.hash) & LOCKED) != 0) {
1503     checkForResize();
1504     f.tryAwaitLock(tab, i);
1505     }
1506     else if (tabAt(tab, i) == f && f.casHash(fh, fh | LOCKED)) {
1507     boolean added = false;
1508     try {
1509     if (tabAt(tab, i) == f) {
1510     count = 1;
1511     for (Node e = f;; ++count) {
1512     Object ek, ev;
1513     if ((e.hash & HASH_BITS) == h &&
1514     (ev = e.val) != null &&
1515     ((ek = e.key) == k || k.equals(ek))) {
1516     val = ev;
1517     break;
1518     }
1519     Node last = e;
1520     if ((e = e.next) == null) {
1521     if ((val = mf.apply(k)) != null) {
1522     added = true;
1523     last.next = new Node(h, k, val, null);
1524     if (count >= TREE_THRESHOLD)
1525     replaceWithTreeBin(tab, i, k);
1526     }
1527     break;
1528     }
1529     }
1530     }
1531     } finally {
1532     if (!f.casHash(fh | LOCKED, fh)) {
1533     f.hash = fh;
1534     synchronized (f) { f.notifyAll(); };
1535     }
1536     }
1537     if (count != 0) {
1538     if (!added)
1539     return val;
1540     if (tab.length <= 64)
1541     count = 2;
1542     break;
1543     }
1544     }
1545 dl 1.105 }
1546 dl 1.99 }
1547 dl 1.119 if (val != null) {
1548     counter.add(1L);
1549     if (count > 1)
1550     checkForResize();
1551     }
1552     return val;
1553 tim 1.1 }
1554    
1555 dl 1.119 /** Implementation for compute */
1556 dl 1.128 @SuppressWarnings("unchecked") private final Object internalCompute
1557     (K k, boolean onlyIfPresent, BiFun<? super K, ? super V, ? extends V> mf) {
1558 dl 1.119 int h = spread(k.hashCode());
1559     Object val = null;
1560     int delta = 0;
1561     int count = 0;
1562     for (Node[] tab = table;;) {
1563     Node f; int i, fh; Object fk;
1564     if (tab == null)
1565     tab = initTable();
1566     else if ((f = tabAt(tab, i = (tab.length - 1) & h)) == null) {
1567     if (onlyIfPresent)
1568     break;
1569     Node node = new Node(fh = h | LOCKED, k, null, null);
1570     if (casTabAt(tab, i, null, node)) {
1571     try {
1572     count = 1;
1573     if ((val = mf.apply(k, null)) != null) {
1574     node.val = val;
1575     delta = 1;
1576     }
1577     } finally {
1578     if (delta == 0)
1579     setTabAt(tab, i, null);
1580     if (!node.casHash(fh, h)) {
1581     node.hash = h;
1582     synchronized (node) { node.notifyAll(); };
1583     }
1584     }
1585     }
1586     if (count != 0)
1587     break;
1588     }
1589     else if ((fh = f.hash) == MOVED) {
1590     if ((fk = f.key) instanceof TreeBin) {
1591     TreeBin t = (TreeBin)fk;
1592     t.acquire(0);
1593     try {
1594     if (tabAt(tab, i) == f) {
1595     count = 1;
1596     TreeNode p = t.getTreeNode(h, k, t.root);
1597     Object pv = (p == null) ? null : p.val;
1598     if ((val = mf.apply(k, (V)pv)) != null) {
1599     if (p != null)
1600     p.val = val;
1601     else {
1602     count = 2;
1603     delta = 1;
1604     t.putTreeNode(h, k, val);
1605     }
1606     }
1607     else if (p != null) {
1608     delta = -1;
1609     t.deleteTreeNode(p);
1610     }
1611     }
1612     } finally {
1613     t.release(0);
1614     }
1615     if (count != 0)
1616     break;
1617     }
1618     else
1619     tab = (Node[])fk;
1620     }
1621     else if ((fh & LOCKED) != 0) {
1622     checkForResize();
1623     f.tryAwaitLock(tab, i);
1624     }
1625     else if (f.casHash(fh, fh | LOCKED)) {
1626     try {
1627     if (tabAt(tab, i) == f) {
1628     count = 1;
1629     for (Node e = f, pred = null;; ++count) {
1630     Object ek, ev;
1631     if ((e.hash & HASH_BITS) == h &&
1632     (ev = e.val) != null &&
1633     ((ek = e.key) == k || k.equals(ek))) {
1634     val = mf.apply(k, (V)ev);
1635     if (val != null)
1636     e.val = val;
1637     else {
1638     delta = -1;
1639     Node en = e.next;
1640     if (pred != null)
1641     pred.next = en;
1642     else
1643     setTabAt(tab, i, en);
1644     }
1645     break;
1646     }
1647     pred = e;
1648     if ((e = e.next) == null) {
1649     if (!onlyIfPresent && (val = mf.apply(k, null)) != null) {
1650     pred.next = new Node(h, k, val, null);
1651     delta = 1;
1652     if (count >= TREE_THRESHOLD)
1653     replaceWithTreeBin(tab, i, k);
1654     }
1655     break;
1656     }
1657     }
1658     }
1659     } finally {
1660     if (!f.casHash(fh | LOCKED, fh)) {
1661     f.hash = fh;
1662     synchronized (f) { f.notifyAll(); };
1663     }
1664     }
1665     if (count != 0) {
1666     if (tab.length <= 64)
1667     count = 2;
1668     break;
1669     }
1670     }
1671     }
1672     if (delta != 0) {
1673     counter.add((long)delta);
1674     if (count > 1)
1675     checkForResize();
1676     }
1677     return val;
1678     }
1679    
1680 dl 1.126 /** Implementation for merge */
1681 dl 1.128 @SuppressWarnings("unchecked") private final Object internalMerge
1682     (K k, V v, BiFun<? super V, ? super V, ? extends V> mf) {
1683 dl 1.119 int h = spread(k.hashCode());
1684     Object val = null;
1685     int delta = 0;
1686     int count = 0;
1687     for (Node[] tab = table;;) {
1688     int i; Node f; int fh; Object fk, fv;
1689     if (tab == null)
1690     tab = initTable();
1691     else if ((f = tabAt(tab, i = (tab.length - 1) & h)) == null) {
1692     if (casTabAt(tab, i, null, new Node(h, k, v, null))) {
1693     delta = 1;
1694     val = v;
1695     break;
1696     }
1697     }
1698     else if ((fh = f.hash) == MOVED) {
1699     if ((fk = f.key) instanceof TreeBin) {
1700     TreeBin t = (TreeBin)fk;
1701     t.acquire(0);
1702     try {
1703     if (tabAt(tab, i) == f) {
1704     count = 1;
1705     TreeNode p = t.getTreeNode(h, k, t.root);
1706     val = (p == null) ? v : mf.apply((V)p.val, v);
1707     if (val != null) {
1708     if (p != null)
1709     p.val = val;
1710     else {
1711     count = 2;
1712     delta = 1;
1713     t.putTreeNode(h, k, val);
1714     }
1715     }
1716     else if (p != null) {
1717     delta = -1;
1718     t.deleteTreeNode(p);
1719     }
1720     }
1721     } finally {
1722     t.release(0);
1723     }
1724     if (count != 0)
1725     break;
1726     }
1727     else
1728     tab = (Node[])fk;
1729     }
1730     else if ((fh & LOCKED) != 0) {
1731     checkForResize();
1732     f.tryAwaitLock(tab, i);
1733     }
1734     else if (f.casHash(fh, fh | LOCKED)) {
1735     try {
1736     if (tabAt(tab, i) == f) {
1737     count = 1;
1738     for (Node e = f, pred = null;; ++count) {
1739     Object ek, ev;
1740     if ((e.hash & HASH_BITS) == h &&
1741     (ev = e.val) != null &&
1742     ((ek = e.key) == k || k.equals(ek))) {
1743     val = mf.apply(v, (V)ev);
1744     if (val != null)
1745     e.val = val;
1746     else {
1747     delta = -1;
1748     Node en = e.next;
1749     if (pred != null)
1750     pred.next = en;
1751     else
1752     setTabAt(tab, i, en);
1753     }
1754     break;
1755     }
1756     pred = e;
1757     if ((e = e.next) == null) {
1758     val = v;
1759     pred.next = new Node(h, k, val, null);
1760     delta = 1;
1761     if (count >= TREE_THRESHOLD)
1762     replaceWithTreeBin(tab, i, k);
1763     break;
1764     }
1765     }
1766     }
1767     } finally {
1768     if (!f.casHash(fh | LOCKED, fh)) {
1769     f.hash = fh;
1770     synchronized (f) { f.notifyAll(); };
1771     }
1772     }
1773     if (count != 0) {
1774     if (tab.length <= 64)
1775     count = 2;
1776     break;
1777     }
1778 dl 1.105 }
1779 dl 1.99 }
1780 dl 1.119 if (delta != 0) {
1781     counter.add((long)delta);
1782     if (count > 1)
1783     checkForResize();
1784     }
1785     return val;
1786     }
1787    
1788     /** Implementation for putAll */
1789     private final void internalPutAll(Map<?, ?> m) {
1790     tryPresize(m.size());
1791     long delta = 0L; // number of uncommitted additions
1792     boolean npe = false; // to throw exception on exit for nulls
1793     try { // to clean up counts on other exceptions
1794     for (Map.Entry<?, ?> entry : m.entrySet()) {
1795     Object k, v;
1796     if (entry == null || (k = entry.getKey()) == null ||
1797     (v = entry.getValue()) == null) {
1798     npe = true;
1799     break;
1800     }
1801     int h = spread(k.hashCode());
1802     for (Node[] tab = table;;) {
1803     int i; Node f; int fh; Object fk;
1804     if (tab == null)
1805     tab = initTable();
1806     else if ((f = tabAt(tab, i = (tab.length - 1) & h)) == null){
1807     if (casTabAt(tab, i, null, new Node(h, k, v, null))) {
1808     ++delta;
1809     break;
1810     }
1811     }
1812     else if ((fh = f.hash) == MOVED) {
1813     if ((fk = f.key) instanceof TreeBin) {
1814     TreeBin t = (TreeBin)fk;
1815     boolean validated = false;
1816     t.acquire(0);
1817     try {
1818     if (tabAt(tab, i) == f) {
1819     validated = true;
1820     TreeNode p = t.getTreeNode(h, k, t.root);
1821     if (p != null)
1822     p.val = v;
1823     else {
1824     t.putTreeNode(h, k, v);
1825     ++delta;
1826     }
1827     }
1828     } finally {
1829     t.release(0);
1830     }
1831     if (validated)
1832     break;
1833     }
1834     else
1835     tab = (Node[])fk;
1836     }
1837     else if ((fh & LOCKED) != 0) {
1838     counter.add(delta);
1839     delta = 0L;
1840     checkForResize();
1841     f.tryAwaitLock(tab, i);
1842     }
1843     else if (f.casHash(fh, fh | LOCKED)) {
1844     int count = 0;
1845     try {
1846     if (tabAt(tab, i) == f) {
1847     count = 1;
1848     for (Node e = f;; ++count) {
1849     Object ek, ev;
1850     if ((e.hash & HASH_BITS) == h &&
1851     (ev = e.val) != null &&
1852     ((ek = e.key) == k || k.equals(ek))) {
1853     e.val = v;
1854     break;
1855     }
1856     Node last = e;
1857     if ((e = e.next) == null) {
1858     ++delta;
1859     last.next = new Node(h, k, v, null);
1860     if (count >= TREE_THRESHOLD)
1861     replaceWithTreeBin(tab, i, k);
1862     break;
1863     }
1864     }
1865     }
1866     } finally {
1867     if (!f.casHash(fh | LOCKED, fh)) {
1868     f.hash = fh;
1869     synchronized (f) { f.notifyAll(); };
1870     }
1871     }
1872     if (count != 0) {
1873     if (count > 1) {
1874     counter.add(delta);
1875     delta = 0L;
1876     checkForResize();
1877 dl 1.99 }
1878 dl 1.119 break;
1879 dl 1.99 }
1880 dl 1.21 }
1881     }
1882 dl 1.45 }
1883     } finally {
1884 dl 1.119 if (delta != 0)
1885     counter.add(delta);
1886 dl 1.45 }
1887 dl 1.119 if (npe)
1888     throw new NullPointerException();
1889 tim 1.1 }
1890 dl 1.19
1891 dl 1.119 /* ---------------- Table Initialization and Resizing -------------- */
1892    
1893 tim 1.1 /**
1894 dl 1.119 * Returns a power of two table size for the given desired capacity.
1895     * See Hackers Delight, sec 3.2
1896 tim 1.1 */
1897 dl 1.119 private static final int tableSizeFor(int c) {
1898     int n = c - 1;
1899     n |= n >>> 1;
1900     n |= n >>> 2;
1901     n |= n >>> 4;
1902     n |= n >>> 8;
1903     n |= n >>> 16;
1904     return (n < 0) ? 1 : (n >= MAXIMUM_CAPACITY) ? MAXIMUM_CAPACITY : n + 1;
1905 tim 1.1 }
1906    
1907     /**
1908 dl 1.119 * Initializes table, using the size recorded in sizeCtl.
1909     */
1910     private final Node[] initTable() {
1911     Node[] tab; int sc;
1912     while ((tab = table) == null) {
1913     if ((sc = sizeCtl) < 0)
1914     Thread.yield(); // lost initialization race; just spin
1915     else if (UNSAFE.compareAndSwapInt(this, sizeCtlOffset, sc, -1)) {
1916     try {
1917     if ((tab = table) == null) {
1918     int n = (sc > 0) ? sc : DEFAULT_CAPACITY;
1919     tab = table = new Node[n];
1920     sc = n - (n >>> 2);
1921     }
1922     } finally {
1923     sizeCtl = sc;
1924     }
1925     break;
1926     }
1927     }
1928     return tab;
1929 dl 1.4 }
1930    
1931     /**
1932 dl 1.119 * If table is too small and not already resizing, creates next
1933     * table and transfers bins. Rechecks occupancy after a transfer
1934     * to see if another resize is already needed because resizings
1935     * are lagging additions.
1936 dl 1.51 */
1937 dl 1.119 private final void checkForResize() {
1938     Node[] tab; int n, sc;
1939     while ((tab = table) != null &&
1940     (n = tab.length) < MAXIMUM_CAPACITY &&
1941     (sc = sizeCtl) >= 0 && counter.sum() >= (long)sc &&
1942     UNSAFE.compareAndSwapInt(this, sizeCtlOffset, sc, -1)) {
1943     try {
1944     if (tab == table) {
1945     table = rebuild(tab);
1946     sc = (n << 1) - (n >>> 1);
1947     }
1948     } finally {
1949     sizeCtl = sc;
1950     }
1951     }
1952 dl 1.4 }
1953    
1954     /**
1955 dl 1.119 * Tries to presize table to accommodate the given number of elements.
1956 tim 1.1 *
1957 dl 1.119 * @param size number of elements (doesn't need to be perfectly accurate)
1958 tim 1.1 */
1959 dl 1.119 private final void tryPresize(int size) {
1960     int c = (size >= (MAXIMUM_CAPACITY >>> 1)) ? MAXIMUM_CAPACITY :
1961     tableSizeFor(size + (size >>> 1) + 1);
1962     int sc;
1963     while ((sc = sizeCtl) >= 0) {
1964     Node[] tab = table; int n;
1965     if (tab == null || (n = tab.length) == 0) {
1966     n = (sc > c) ? sc : c;
1967     if (UNSAFE.compareAndSwapInt(this, sizeCtlOffset, sc, -1)) {
1968     try {
1969     if (table == tab) {
1970     table = new Node[n];
1971     sc = n - (n >>> 2);
1972     }
1973     } finally {
1974     sizeCtl = sc;
1975     }
1976     }
1977     }
1978     else if (c <= sc || n >= MAXIMUM_CAPACITY)
1979     break;
1980     else if (UNSAFE.compareAndSwapInt(this, sizeCtlOffset, sc, -1)) {
1981     try {
1982     if (table == tab) {
1983     table = rebuild(tab);
1984     sc = (n << 1) - (n >>> 1);
1985     }
1986     } finally {
1987     sizeCtl = sc;
1988     }
1989     }
1990     }
1991 dl 1.4 }
1992    
1993 dl 1.119 /*
1994     * Moves and/or copies the nodes in each bin to new table. See
1995     * above for explanation.
1996 dl 1.4 *
1997 dl 1.119 * @return the new table
1998 dl 1.4 */
1999 dl 1.119 private static final Node[] rebuild(Node[] tab) {
2000     int n = tab.length;
2001     Node[] nextTab = new Node[n << 1];
2002     Node fwd = new Node(MOVED, nextTab, null, null);
2003     int[] buffer = null; // holds bins to revisit; null until needed
2004     Node rev = null; // reverse forwarder; null until needed
2005     int nbuffered = 0; // the number of bins in buffer list
2006     int bufferIndex = 0; // buffer index of current buffered bin
2007     int bin = n - 1; // current non-buffered bin or -1 if none
2008    
2009     for (int i = bin;;) { // start upwards sweep
2010     int fh; Node f;
2011     if ((f = tabAt(tab, i)) == null) {
2012 dl 1.128 if (bin >= 0) { // Unbuffered; no lock needed (or available)
2013 dl 1.119 if (!casTabAt(tab, i, f, fwd))
2014     continue;
2015     }
2016     else { // transiently use a locked forwarding node
2017     Node g = new Node(MOVED|LOCKED, nextTab, null, null);
2018     if (!casTabAt(tab, i, f, g))
2019     continue;
2020     setTabAt(nextTab, i, null);
2021     setTabAt(nextTab, i + n, null);
2022     setTabAt(tab, i, fwd);
2023     if (!g.casHash(MOVED|LOCKED, MOVED)) {
2024     g.hash = MOVED;
2025     synchronized (g) { g.notifyAll(); }
2026     }
2027     }
2028     }
2029     else if ((fh = f.hash) == MOVED) {
2030     Object fk = f.key;
2031     if (fk instanceof TreeBin) {
2032     TreeBin t = (TreeBin)fk;
2033     boolean validated = false;
2034     t.acquire(0);
2035     try {
2036     if (tabAt(tab, i) == f) {
2037     validated = true;
2038     splitTreeBin(nextTab, i, t);
2039     setTabAt(tab, i, fwd);
2040     }
2041     } finally {
2042     t.release(0);
2043     }
2044     if (!validated)
2045     continue;
2046     }
2047     }
2048     else if ((fh & LOCKED) == 0 && f.casHash(fh, fh|LOCKED)) {
2049     boolean validated = false;
2050     try { // split to lo and hi lists; copying as needed
2051     if (tabAt(tab, i) == f) {
2052     validated = true;
2053     splitBin(nextTab, i, f);
2054     setTabAt(tab, i, fwd);
2055     }
2056     } finally {
2057     if (!f.casHash(fh | LOCKED, fh)) {
2058     f.hash = fh;
2059     synchronized (f) { f.notifyAll(); };
2060     }
2061     }
2062     if (!validated)
2063     continue;
2064     }
2065     else {
2066     if (buffer == null) // initialize buffer for revisits
2067     buffer = new int[TRANSFER_BUFFER_SIZE];
2068     if (bin < 0 && bufferIndex > 0) {
2069     int j = buffer[--bufferIndex];
2070     buffer[bufferIndex] = i;
2071     i = j; // swap with another bin
2072     continue;
2073     }
2074     if (bin < 0 || nbuffered >= TRANSFER_BUFFER_SIZE) {
2075     f.tryAwaitLock(tab, i);
2076     continue; // no other options -- block
2077     }
2078     if (rev == null) // initialize reverse-forwarder
2079     rev = new Node(MOVED, tab, null, null);
2080     if (tabAt(tab, i) != f || (f.hash & LOCKED) == 0)
2081     continue; // recheck before adding to list
2082     buffer[nbuffered++] = i;
2083     setTabAt(nextTab, i, rev); // install place-holders
2084     setTabAt(nextTab, i + n, rev);
2085     }
2086    
2087     if (bin > 0)
2088     i = --bin;
2089     else if (buffer != null && nbuffered > 0) {
2090     bin = -1;
2091     i = buffer[bufferIndex = --nbuffered];
2092     }
2093     else
2094     return nextTab;
2095     }
2096 dl 1.4 }
2097 tim 1.1
2098 dl 1.4 /**
2099 dl 1.119 * Splits a normal bin with list headed by e into lo and hi parts;
2100     * installs in given table.
2101 dl 1.4 */
2102 dl 1.119 private static void splitBin(Node[] nextTab, int i, Node e) {
2103     int bit = nextTab.length >>> 1; // bit to split on
2104     int runBit = e.hash & bit;
2105     Node lastRun = e, lo = null, hi = null;
2106     for (Node p = e.next; p != null; p = p.next) {
2107     int b = p.hash & bit;
2108     if (b != runBit) {
2109     runBit = b;
2110     lastRun = p;
2111     }
2112     }
2113     if (runBit == 0)
2114     lo = lastRun;
2115     else
2116     hi = lastRun;
2117     for (Node p = e; p != lastRun; p = p.next) {
2118     int ph = p.hash & HASH_BITS;
2119     Object pk = p.key, pv = p.val;
2120     if ((ph & bit) == 0)
2121     lo = new Node(ph, pk, pv, lo);
2122     else
2123     hi = new Node(ph, pk, pv, hi);
2124     }
2125     setTabAt(nextTab, i, lo);
2126     setTabAt(nextTab, i + bit, hi);
2127     }
2128    
2129     /**
2130     * Splits a tree bin into lo and hi parts; installs in given table.
2131     */
2132     private static void splitTreeBin(Node[] nextTab, int i, TreeBin t) {
2133     int bit = nextTab.length >>> 1;
2134     TreeBin lt = new TreeBin();
2135     TreeBin ht = new TreeBin();
2136     int lc = 0, hc = 0;
2137     for (Node e = t.first; e != null; e = e.next) {
2138     int h = e.hash & HASH_BITS;
2139     Object k = e.key, v = e.val;
2140     if ((h & bit) == 0) {
2141     ++lc;
2142     lt.putTreeNode(h, k, v);
2143     }
2144     else {
2145     ++hc;
2146     ht.putTreeNode(h, k, v);
2147     }
2148     }
2149     Node ln, hn; // throw away trees if too small
2150     if (lc <= (TREE_THRESHOLD >>> 1)) {
2151     ln = null;
2152     for (Node p = lt.first; p != null; p = p.next)
2153     ln = new Node(p.hash, p.key, p.val, ln);
2154     }
2155     else
2156     ln = new Node(MOVED, lt, null, null);
2157     setTabAt(nextTab, i, ln);
2158     if (hc <= (TREE_THRESHOLD >>> 1)) {
2159     hn = null;
2160     for (Node p = ht.first; p != null; p = p.next)
2161     hn = new Node(p.hash, p.key, p.val, hn);
2162     }
2163     else
2164     hn = new Node(MOVED, ht, null, null);
2165     setTabAt(nextTab, i + bit, hn);
2166     }
2167    
2168     /**
2169     * Implementation for clear. Steps through each bin, removing all
2170     * nodes.
2171     */
2172     private final void internalClear() {
2173     long delta = 0L; // negative number of deletions
2174     int i = 0;
2175     Node[] tab = table;
2176     while (tab != null && i < tab.length) {
2177     int fh; Object fk;
2178     Node f = tabAt(tab, i);
2179     if (f == null)
2180     ++i;
2181     else if ((fh = f.hash) == MOVED) {
2182     if ((fk = f.key) instanceof TreeBin) {
2183     TreeBin t = (TreeBin)fk;
2184     t.acquire(0);
2185     try {
2186     if (tabAt(tab, i) == f) {
2187     for (Node p = t.first; p != null; p = p.next) {
2188 dl 1.128 if (p.val != null) { // (currently always true)
2189     p.val = null;
2190     --delta;
2191     }
2192 dl 1.119 }
2193     t.first = null;
2194     t.root = null;
2195     ++i;
2196     }
2197     } finally {
2198     t.release(0);
2199     }
2200     }
2201     else
2202     tab = (Node[])fk;
2203     }
2204     else if ((fh & LOCKED) != 0) {
2205     counter.add(delta); // opportunistically update count
2206     delta = 0L;
2207     f.tryAwaitLock(tab, i);
2208     }
2209     else if (f.casHash(fh, fh | LOCKED)) {
2210     try {
2211     if (tabAt(tab, i) == f) {
2212     for (Node e = f; e != null; e = e.next) {
2213 dl 1.128 if (e.val != null) { // (currently always true)
2214     e.val = null;
2215     --delta;
2216     }
2217 dl 1.119 }
2218     setTabAt(tab, i, null);
2219     ++i;
2220     }
2221     } finally {
2222     if (!f.casHash(fh | LOCKED, fh)) {
2223     f.hash = fh;
2224     synchronized (f) { f.notifyAll(); };
2225     }
2226     }
2227     }
2228     }
2229     if (delta != 0)
2230     counter.add(delta);
2231     }
2232    
2233     /* ----------------Table Traversal -------------- */
2234    
2235     /**
2236     * Encapsulates traversal for methods such as containsValue; also
2237 dl 1.126 * serves as a base class for other iterators and bulk tasks.
2238 dl 1.119 *
2239     * At each step, the iterator snapshots the key ("nextKey") and
2240     * value ("nextVal") of a valid node (i.e., one that, at point of
2241     * snapshot, has a non-null user value). Because val fields can
2242     * change (including to null, indicating deletion), field nextVal
2243     * might not be accurate at point of use, but still maintains the
2244     * weak consistency property of holding a value that was once
2245     * valid.
2246     *
2247     * Internal traversals directly access these fields, as in:
2248     * {@code while (it.advance() != null) { process(it.nextKey); }}
2249     *
2250     * Exported iterators must track whether the iterator has advanced
2251     * (in hasNext vs next) (by setting/checking/nulling field
2252     * nextVal), and then extract key, value, or key-value pairs as
2253     * return values of next().
2254     *
2255     * The iterator visits once each still-valid node that was
2256     * reachable upon iterator construction. It might miss some that
2257     * were added to a bin after the bin was visited, which is OK wrt
2258     * consistency guarantees. Maintaining this property in the face
2259     * of possible ongoing resizes requires a fair amount of
2260     * bookkeeping state that is difficult to optimize away amidst
2261     * volatile accesses. Even so, traversal maintains reasonable
2262     * throughput.
2263     *
2264     * Normally, iteration proceeds bin-by-bin traversing lists.
2265     * However, if the table has been resized, then all future steps
2266     * must traverse both the bin at the current index as well as at
2267     * (index + baseSize); and so on for further resizings. To
2268     * paranoically cope with potential sharing by users of iterators
2269     * across threads, iteration terminates if a bounds checks fails
2270     * for a table read.
2271     *
2272     * This class extends ForkJoinTask to streamline parallel
2273     * iteration in bulk operations (see BulkTask). This adds only an
2274     * int of space overhead, which is close enough to negligible in
2275 dl 1.126 * cases where it is not needed to not worry about it. Because
2276     * ForkJoinTask is Serializable, but iterators need not be, we
2277     * need to add warning suppressions.
2278 dl 1.119 */
2279 dl 1.128 @SuppressWarnings("serial") static class Traverser<K,V,R> extends ForkJoinTask<R> {
2280 dl 1.119 final ConcurrentHashMap<K, V> map;
2281     Node next; // the next entry to use
2282     Node last; // the last entry used
2283     Object nextKey; // cached key field of next
2284     Object nextVal; // cached val field of next
2285     Node[] tab; // current table; updated if resized
2286     int index; // index of bin to use next
2287     int baseIndex; // current index of initial table
2288     int baseLimit; // index bound for initial table
2289 dl 1.130 int baseSize; // initial table size
2290 dl 1.119
2291     /** Creates iterator for all entries in the table. */
2292     Traverser(ConcurrentHashMap<K, V> map) {
2293 dl 1.130 this.map = map;
2294 dl 1.119 }
2295    
2296     /** Creates iterator for split() methods */
2297 dl 1.128 Traverser(Traverser<K,V,?> it) {
2298 dl 1.130 ConcurrentHashMap<K, V> m; Node[] t;
2299     if ((m = this.map = it.map) == null)
2300     t = null;
2301     else if ((t = it.tab) == null && // force parent tab initialization
2302     (t = it.tab = m.table) != null)
2303     it.baseLimit = it.baseSize = t.length;
2304     this.tab = t;
2305 dl 1.119 this.baseSize = it.baseSize;
2306 dl 1.128 it.baseLimit = this.index = this.baseIndex =
2307     ((this.baseLimit = it.baseLimit) + it.baseIndex + 1) >>> 1;
2308 dl 1.119 }
2309    
2310     /**
2311     * Advances next; returns nextVal or null if terminated.
2312     * See above for explanation.
2313     */
2314     final Object advance() {
2315     Node e = last = next;
2316     Object ev = null;
2317     outer: do {
2318     if (e != null) // advance past used/skipped node
2319     e = e.next;
2320     while (e == null) { // get to next non-null bin
2321 dl 1.130 ConcurrentHashMap<K, V> m;
2322 dl 1.119 Node[] t; int b, i, n; Object ek; // checks must use locals
2323 dl 1.130 if ((t = tab) != null)
2324     n = t.length;
2325     else if ((m = map) != null && (t = tab = m.table) != null)
2326     n = baseLimit = baseSize = t.length;
2327     else
2328 dl 1.119 break outer;
2329 dl 1.130 if ((b = baseIndex) >= baseLimit ||
2330     (i = index) < 0 || i >= n)
2331     break outer;
2332     if ((e = tabAt(t, i)) != null && e.hash == MOVED) {
2333 dl 1.119 if ((ek = e.key) instanceof TreeBin)
2334     e = ((TreeBin)ek).first;
2335     else {
2336     tab = (Node[])ek;
2337     continue; // restarts due to null val
2338     }
2339     } // visit upper slots if present
2340     index = (i += baseSize) < n ? i : (baseIndex = b + 1);
2341     }
2342     nextKey = e.key;
2343     } while ((ev = e.val) == null); // skip deleted or special nodes
2344     next = e;
2345     return nextVal = ev;
2346     }
2347    
2348     public final void remove() {
2349 dl 1.124 if (nextVal == null && last == null)
2350 dl 1.119 advance();
2351     Node e = last;
2352     if (e == null)
2353     throw new IllegalStateException();
2354     last = null;
2355     map.remove(e.key);
2356     }
2357    
2358     public final boolean hasNext() {
2359     return nextVal != null || advance() != null;
2360     }
2361    
2362     public final boolean hasMoreElements() { return hasNext(); }
2363     public final void setRawResult(Object x) { }
2364     public R getRawResult() { return null; }
2365     public boolean exec() { return true; }
2366     }
2367    
2368     /* ---------------- Public operations -------------- */
2369    
2370     /**
2371     * Creates a new, empty map with the default initial table size (16).
2372     */
2373     public ConcurrentHashMap() {
2374     this.counter = new LongAdder();
2375     }
2376    
2377     /**
2378     * Creates a new, empty map with an initial table size
2379     * accommodating the specified number of elements without the need
2380     * to dynamically resize.
2381     *
2382     * @param initialCapacity The implementation performs internal
2383     * sizing to accommodate this many elements.
2384     * @throws IllegalArgumentException if the initial capacity of
2385     * elements is negative
2386     */
2387     public ConcurrentHashMap(int initialCapacity) {
2388     if (initialCapacity < 0)
2389     throw new IllegalArgumentException();
2390     int cap = ((initialCapacity >= (MAXIMUM_CAPACITY >>> 1)) ?
2391     MAXIMUM_CAPACITY :
2392     tableSizeFor(initialCapacity + (initialCapacity >>> 1) + 1));
2393     this.counter = new LongAdder();
2394     this.sizeCtl = cap;
2395     }
2396    
2397     /**
2398     * Creates a new map with the same mappings as the given map.
2399     *
2400     * @param m the map
2401     */
2402     public ConcurrentHashMap(Map<? extends K, ? extends V> m) {
2403     this.counter = new LongAdder();
2404     this.sizeCtl = DEFAULT_CAPACITY;
2405     internalPutAll(m);
2406     }
2407    
2408     /**
2409     * Creates a new, empty map with an initial table size based on
2410     * the given number of elements ({@code initialCapacity}) and
2411     * initial table density ({@code loadFactor}).
2412     *
2413     * @param initialCapacity the initial capacity. The implementation
2414     * performs internal sizing to accommodate this many elements,
2415     * given the specified load factor.
2416     * @param loadFactor the load factor (table density) for
2417     * establishing the initial table size
2418     * @throws IllegalArgumentException if the initial capacity of
2419     * elements is negative or the load factor is nonpositive
2420     *
2421     * @since 1.6
2422     */
2423     public ConcurrentHashMap(int initialCapacity, float loadFactor) {
2424     this(initialCapacity, loadFactor, 1);
2425     }
2426    
2427     /**
2428     * Creates a new, empty map with an initial table size based on
2429     * the given number of elements ({@code initialCapacity}), table
2430     * density ({@code loadFactor}), and number of concurrently
2431     * updating threads ({@code concurrencyLevel}).
2432     *
2433     * @param initialCapacity the initial capacity. The implementation
2434     * performs internal sizing to accommodate this many elements,
2435     * given the specified load factor.
2436     * @param loadFactor the load factor (table density) for
2437     * establishing the initial table size
2438     * @param concurrencyLevel the estimated number of concurrently
2439     * updating threads. The implementation may use this value as
2440     * a sizing hint.
2441     * @throws IllegalArgumentException if the initial capacity is
2442     * negative or the load factor or concurrencyLevel are
2443     * nonpositive
2444     */
2445     public ConcurrentHashMap(int initialCapacity,
2446     float loadFactor, int concurrencyLevel) {
2447     if (!(loadFactor > 0.0f) || initialCapacity < 0 || concurrencyLevel <= 0)
2448     throw new IllegalArgumentException();
2449     if (initialCapacity < concurrencyLevel) // Use at least as many bins
2450     initialCapacity = concurrencyLevel; // as estimated threads
2451     long size = (long)(1.0 + (long)initialCapacity / loadFactor);
2452     int cap = (size >= (long)MAXIMUM_CAPACITY) ?
2453     MAXIMUM_CAPACITY : tableSizeFor((int)size);
2454     this.counter = new LongAdder();
2455     this.sizeCtl = cap;
2456     }
2457    
2458     /**
2459     * {@inheritDoc}
2460     */
2461     public boolean isEmpty() {
2462     return counter.sum() <= 0L; // ignore transient negative values
2463     }
2464    
2465     /**
2466     * {@inheritDoc}
2467     */
2468     public int size() {
2469     long n = counter.sum();
2470     return ((n < 0L) ? 0 :
2471     (n > (long)Integer.MAX_VALUE) ? Integer.MAX_VALUE :
2472     (int)n);
2473     }
2474    
2475     /**
2476     * Returns the number of mappings. This method should be used
2477     * instead of {@link #size} because a ConcurrentHashMap may
2478     * contain more mappings than can be represented as an int. The
2479     * value returned is a snapshot; the actual count may differ if
2480 dl 1.127 * there are ongoing concurrent insertions or removals.
2481 dl 1.119 *
2482     * @return the number of mappings
2483     */
2484     public long mappingCount() {
2485     long n = counter.sum();
2486 dl 1.126 return (n < 0L) ? 0L : n; // ignore transient negative values
2487 dl 1.119 }
2488    
2489     /**
2490     * Returns the value to which the specified key is mapped,
2491     * or {@code null} if this map contains no mapping for the key.
2492     *
2493     * <p>More formally, if this map contains a mapping from a key
2494     * {@code k} to a value {@code v} such that {@code key.equals(k)},
2495     * then this method returns {@code v}; otherwise it returns
2496     * {@code null}. (There can be at most one such mapping.)
2497     *
2498     * @throws NullPointerException if the specified key is null
2499     */
2500 dl 1.128 @SuppressWarnings("unchecked") public V get(Object key) {
2501 dl 1.119 if (key == null)
2502     throw new NullPointerException();
2503     return (V)internalGet(key);
2504     }
2505    
2506     /**
2507 dl 1.129 * Returns the value to which the specified key is mapped,
2508     * or the gieven defaultValue if this map contains no mapping for the key.
2509     *
2510     * @param key the key
2511     * @param defaultValue the value to return if this map contains
2512     * no mapping for the given key.
2513     * @return the mapping for the key, if present; else the defaultValue
2514     * @throws NullPointerException if the specified key is null
2515     */
2516     @SuppressWarnings("unchecked") public V getValueOrDefault(Object key, V defaultValue) {
2517     if (key == null)
2518     throw new NullPointerException();
2519     V v = (V) internalGet(key);
2520     return v == null ? defaultValue : v;
2521     }
2522    
2523     /**
2524 dl 1.119 * Tests if the specified object is a key in this table.
2525     *
2526     * @param key possible key
2527     * @return {@code true} if and only if the specified object
2528     * is a key in this table, as determined by the
2529     * {@code equals} method; {@code false} otherwise
2530     * @throws NullPointerException if the specified key is null
2531     */
2532     public boolean containsKey(Object key) {
2533     if (key == null)
2534     throw new NullPointerException();
2535     return internalGet(key) != null;
2536     }
2537    
2538     /**
2539     * Returns {@code true} if this map maps one or more keys to the
2540     * specified value. Note: This method may require a full traversal
2541     * of the map, and is much slower than method {@code containsKey}.
2542     *
2543     * @param value value whose presence in this map is to be tested
2544     * @return {@code true} if this map maps one or more keys to the
2545     * specified value
2546     * @throws NullPointerException if the specified value is null
2547     */
2548     public boolean containsValue(Object value) {
2549     if (value == null)
2550     throw new NullPointerException();
2551     Object v;
2552     Traverser<K,V,Object> it = new Traverser<K,V,Object>(this);
2553     while ((v = it.advance()) != null) {
2554     if (v == value || value.equals(v))
2555     return true;
2556     }
2557     return false;
2558     }
2559    
2560     /**
2561     * Legacy method testing if some key maps into the specified value
2562     * in this table. This method is identical in functionality to
2563     * {@link #containsValue}, and exists solely to ensure
2564     * full compatibility with class {@link java.util.Hashtable},
2565     * which supported this method prior to introduction of the
2566     * Java Collections framework.
2567     *
2568     * @param value a value to search for
2569     * @return {@code true} if and only if some key maps to the
2570     * {@code value} argument in this table as
2571     * determined by the {@code equals} method;
2572     * {@code false} otherwise
2573     * @throws NullPointerException if the specified value is null
2574     */
2575     public boolean contains(Object value) {
2576     return containsValue(value);
2577     }
2578    
2579     /**
2580     * Maps the specified key to the specified value in this table.
2581     * Neither the key nor the value can be null.
2582     *
2583     * <p> The value can be retrieved by calling the {@code get} method
2584     * with a key that is equal to the original key.
2585     *
2586     * @param key key with which the specified value is to be associated
2587     * @param value value to be associated with the specified key
2588     * @return the previous value associated with {@code key}, or
2589     * {@code null} if there was no mapping for {@code key}
2590     * @throws NullPointerException if the specified key or value is null
2591     */
2592 dl 1.128 @SuppressWarnings("unchecked") public V put(K key, V value) {
2593 dl 1.119 if (key == null || value == null)
2594     throw new NullPointerException();
2595     return (V)internalPut(key, value);
2596     }
2597    
2598     /**
2599     * {@inheritDoc}
2600     *
2601     * @return the previous value associated with the specified key,
2602     * or {@code null} if there was no mapping for the key
2603     * @throws NullPointerException if the specified key or value is null
2604     */
2605 dl 1.128 @SuppressWarnings("unchecked") public V putIfAbsent(K key, V value) {
2606 dl 1.119 if (key == null || value == null)
2607     throw new NullPointerException();
2608     return (V)internalPutIfAbsent(key, value);
2609     }
2610    
2611     /**
2612     * Copies all of the mappings from the specified map to this one.
2613     * These mappings replace any mappings that this map had for any of the
2614     * keys currently in the specified map.
2615     *
2616     * @param m mappings to be stored in this map
2617     */
2618     public void putAll(Map<? extends K, ? extends V> m) {
2619     internalPutAll(m);
2620     }
2621    
2622     /**
2623     * If the specified key is not already associated with a value,
2624     * computes its value using the given mappingFunction and enters
2625     * it into the map unless null. This is equivalent to
2626     * <pre> {@code
2627     * if (map.containsKey(key))
2628     * return map.get(key);
2629     * value = mappingFunction.apply(key);
2630     * if (value != null)
2631     * map.put(key, value);
2632     * return value;}</pre>
2633     *
2634     * except that the action is performed atomically. If the
2635     * function returns {@code null} no mapping is recorded. If the
2636     * function itself throws an (unchecked) exception, the exception
2637     * is rethrown to its caller, and no mapping is recorded. Some
2638     * attempted update operations on this map by other threads may be
2639     * blocked while computation is in progress, so the computation
2640     * should be short and simple, and must not attempt to update any
2641     * other mappings of this Map. The most appropriate usage is to
2642     * construct a new object serving as an initial mapped value, or
2643     * memoized result, as in:
2644     *
2645     * <pre> {@code
2646     * map.computeIfAbsent(key, new Fun<K, V>() {
2647     * public V map(K k) { return new Value(f(k)); }});}</pre>
2648     *
2649     * @param key key with which the specified value is to be associated
2650     * @param mappingFunction the function to compute a value
2651     * @return the current (existing or computed) value associated with
2652     * the specified key, or null if the computed value is null.
2653     * @throws NullPointerException if the specified key or mappingFunction
2654     * is null
2655     * @throws IllegalStateException if the computation detectably
2656     * attempts a recursive update to this map that would
2657     * otherwise never complete
2658     * @throws RuntimeException or Error if the mappingFunction does so,
2659     * in which case the mapping is left unestablished
2660     */
2661 dl 1.128 @SuppressWarnings("unchecked") public V computeIfAbsent
2662     (K key, Fun<? super K, ? extends V> mappingFunction) {
2663 dl 1.119 if (key == null || mappingFunction == null)
2664     throw new NullPointerException();
2665     return (V)internalComputeIfAbsent(key, mappingFunction);
2666     }
2667    
2668     /**
2669     * If the given key is present, computes a new mapping value given a key and
2670     * its current mapped value. This is equivalent to
2671     * <pre> {@code
2672     * if (map.containsKey(key)) {
2673     * value = remappingFunction.apply(key, map.get(key));
2674     * if (value != null)
2675     * map.put(key, value);
2676     * else
2677     * map.remove(key);
2678     * }
2679     * }</pre>
2680     *
2681     * except that the action is performed atomically. If the
2682     * function returns {@code null}, the mapping is removed. If the
2683     * function itself throws an (unchecked) exception, the exception
2684     * is rethrown to its caller, and the current mapping is left
2685     * unchanged. Some attempted update operations on this map by
2686     * other threads may be blocked while computation is in progress,
2687     * so the computation should be short and simple, and must not
2688     * attempt to update any other mappings of this Map. For example,
2689     * to either create or append new messages to a value mapping:
2690     *
2691     * @param key key with which the specified value is to be associated
2692     * @param remappingFunction the function to compute a value
2693 jsr166 1.123 * @return the new value associated with the specified key, or null if none
2694 dl 1.119 * @throws NullPointerException if the specified key or remappingFunction
2695     * is null
2696     * @throws IllegalStateException if the computation detectably
2697     * attempts a recursive update to this map that would
2698     * otherwise never complete
2699     * @throws RuntimeException or Error if the remappingFunction does so,
2700     * in which case the mapping is unchanged
2701     */
2702 dl 1.128 @SuppressWarnings("unchecked") public V computeIfPresent
2703     (K key, BiFun<? super K, ? super V, ? extends V> remappingFunction) {
2704 dl 1.119 if (key == null || remappingFunction == null)
2705     throw new NullPointerException();
2706     return (V)internalCompute(key, true, remappingFunction);
2707     }
2708    
2709     /**
2710     * Computes a new mapping value given a key and
2711     * its current mapped value (or {@code null} if there is no current
2712     * mapping). This is equivalent to
2713     * <pre> {@code
2714     * value = remappingFunction.apply(key, map.get(key));
2715     * if (value != null)
2716     * map.put(key, value);
2717     * else
2718     * map.remove(key);
2719     * }</pre>
2720     *
2721     * except that the action is performed atomically. If the
2722     * function returns {@code null}, the mapping is removed. If the
2723     * function itself throws an (unchecked) exception, the exception
2724     * is rethrown to its caller, and the current mapping is left
2725     * unchanged. Some attempted update operations on this map by
2726     * other threads may be blocked while computation is in progress,
2727     * so the computation should be short and simple, and must not
2728     * attempt to update any other mappings of this Map. For example,
2729     * to either create or append new messages to a value mapping:
2730     *
2731     * <pre> {@code
2732     * Map<Key, String> map = ...;
2733     * final String msg = ...;
2734     * map.compute(key, new BiFun<Key, String, String>() {
2735     * public String apply(Key k, String v) {
2736     * return (v == null) ? msg : v + msg;});}}</pre>
2737     *
2738     * @param key key with which the specified value is to be associated
2739     * @param remappingFunction the function to compute a value
2740 jsr166 1.123 * @return the new value associated with the specified key, or null if none
2741 dl 1.119 * @throws NullPointerException if the specified key or remappingFunction
2742     * is null
2743     * @throws IllegalStateException if the computation detectably
2744     * attempts a recursive update to this map that would
2745     * otherwise never complete
2746     * @throws RuntimeException or Error if the remappingFunction does so,
2747     * in which case the mapping is unchanged
2748     */
2749 dl 1.128 @SuppressWarnings("unchecked") public V compute
2750     (K key, BiFun<? super K, ? super V, ? extends V> remappingFunction) {
2751 dl 1.119 if (key == null || remappingFunction == null)
2752     throw new NullPointerException();
2753     return (V)internalCompute(key, false, remappingFunction);
2754     }
2755    
2756     /**
2757     * If the specified key is not already associated
2758     * with a value, associate it with the given value.
2759     * Otherwise, replace the value with the results of
2760     * the given remapping function. This is equivalent to:
2761     * <pre> {@code
2762     * if (!map.containsKey(key))
2763     * map.put(value);
2764     * else {
2765     * newValue = remappingFunction.apply(map.get(key), value);
2766     * if (value != null)
2767     * map.put(key, value);
2768     * else
2769     * map.remove(key);
2770     * }
2771     * }</pre>
2772     * except that the action is performed atomically. If the
2773     * function returns {@code null}, the mapping is removed. If the
2774     * function itself throws an (unchecked) exception, the exception
2775     * is rethrown to its caller, and the current mapping is left
2776     * unchanged. Some attempted update operations on this map by
2777     * other threads may be blocked while computation is in progress,
2778     * so the computation should be short and simple, and must not
2779     * attempt to update any other mappings of this Map.
2780     */
2781 dl 1.128 @SuppressWarnings("unchecked") public V merge
2782     (K key, V value, BiFun<? super V, ? super V, ? extends V> remappingFunction) {
2783 dl 1.119 if (key == null || value == null || remappingFunction == null)
2784     throw new NullPointerException();
2785     return (V)internalMerge(key, value, remappingFunction);
2786     }
2787    
2788     /**
2789     * Removes the key (and its corresponding value) from this map.
2790     * This method does nothing if the key is not in the map.
2791     *
2792     * @param key the key that needs to be removed
2793     * @return the previous value associated with {@code key}, or
2794     * {@code null} if there was no mapping for {@code key}
2795     * @throws NullPointerException if the specified key is null
2796     */
2797 dl 1.128 @SuppressWarnings("unchecked") public V remove(Object key) {
2798 dl 1.119 if (key == null)
2799     throw new NullPointerException();
2800     return (V)internalReplace(key, null, null);
2801     }
2802    
2803     /**
2804     * {@inheritDoc}
2805     *
2806     * @throws NullPointerException if the specified key is null
2807     */
2808     public boolean remove(Object key, Object value) {
2809     if (key == null)
2810     throw new NullPointerException();
2811     if (value == null)
2812     return false;
2813     return internalReplace(key, null, value) != null;
2814 tim 1.1 }
2815 dl 1.31
2816     /**
2817 jsr166 1.68 * {@inheritDoc}
2818     *
2819     * @throws NullPointerException if any of the arguments are null
2820 dl 1.31 */
2821     public boolean replace(K key, V oldValue, V newValue) {
2822 dl 1.119 if (key == null || oldValue == null || newValue == null)
2823 dl 1.31 throw new NullPointerException();
2824 dl 1.119 return internalReplace(key, newValue, oldValue) != null;
2825 dl 1.32 }
2826    
2827     /**
2828 jsr166 1.68 * {@inheritDoc}
2829     *
2830     * @return the previous value associated with the specified key,
2831 dl 1.119 * or {@code null} if there was no mapping for the key
2832 jsr166 1.68 * @throws NullPointerException if the specified key or value is null
2833 dl 1.32 */
2834 dl 1.128 @SuppressWarnings("unchecked") public V replace(K key, V value) {
2835 dl 1.119 if (key == null || value == null)
2836 dl 1.32 throw new NullPointerException();
2837 dl 1.119 return (V)internalReplace(key, value, null);
2838 dl 1.31 }
2839    
2840 tim 1.1 /**
2841 jsr166 1.68 * Removes all of the mappings from this map.
2842 tim 1.1 */
2843     public void clear() {
2844 dl 1.119 internalClear();
2845 tim 1.1 }
2846    
2847     /**
2848 jsr166 1.68 * Returns a {@link Set} view of the keys contained in this map.
2849     * The set is backed by the map, so changes to the map are
2850     * reflected in the set, and vice-versa. The set supports element
2851     * removal, which removes the corresponding mapping from this map,
2852 dl 1.119 * via the {@code Iterator.remove}, {@code Set.remove},
2853     * {@code removeAll}, {@code retainAll}, and {@code clear}
2854     * operations. It does not support the {@code add} or
2855     * {@code addAll} operations.
2856 jsr166 1.68 *
2857 dl 1.119 * <p>The view's {@code iterator} is a "weakly consistent" iterator
2858 jsr166 1.68 * that will never throw {@link ConcurrentModificationException},
2859 dl 1.14 * and guarantees to traverse elements as they existed upon
2860     * construction of the iterator, and may (but is not guaranteed to)
2861     * reflect any modifications subsequent to construction.
2862 tim 1.1 */
2863     public Set<K> keySet() {
2864 dl 1.119 KeySet<K,V> ks = keySet;
2865     return (ks != null) ? ks : (keySet = new KeySet<K,V>(this));
2866 tim 1.1 }
2867    
2868     /**
2869 jsr166 1.68 * Returns a {@link Collection} view of the values contained in this map.
2870     * The collection is backed by the map, so changes to the map are
2871     * reflected in the collection, and vice-versa. The collection
2872     * supports element removal, which removes the corresponding
2873 dl 1.119 * mapping from this map, via the {@code Iterator.remove},
2874     * {@code Collection.remove}, {@code removeAll},
2875     * {@code retainAll}, and {@code clear} operations. It does not
2876     * support the {@code add} or {@code addAll} operations.
2877 jsr166 1.68 *
2878 dl 1.119 * <p>The view's {@code iterator} is a "weakly consistent" iterator
2879 jsr166 1.68 * that will never throw {@link ConcurrentModificationException},
2880 dl 1.14 * and guarantees to traverse elements as they existed upon
2881     * construction of the iterator, and may (but is not guaranteed to)
2882     * reflect any modifications subsequent to construction.
2883 tim 1.1 */
2884 dl 1.119 public Collection<V> values() {
2885     Values<K,V> vs = values;
2886     return (vs != null) ? vs : (values = new Values<K,V>(this));
2887     }
2888    
2889     /**
2890     * Returns a {@link Set} view of the mappings contained in this map.
2891     * The set is backed by the map, so changes to the map are
2892     * reflected in the set, and vice-versa. The set supports element
2893     * removal, which removes the corresponding mapping from the map,
2894     * via the {@code Iterator.remove}, {@code Set.remove},
2895     * {@code removeAll}, {@code retainAll}, and {@code clear}
2896     * operations. It does not support the {@code add} or
2897     * {@code addAll} operations.
2898     *
2899     * <p>The view's {@code iterator} is a "weakly consistent" iterator
2900     * that will never throw {@link ConcurrentModificationException},
2901     * and guarantees to traverse elements as they existed upon
2902     * construction of the iterator, and may (but is not guaranteed to)
2903     * reflect any modifications subsequent to construction.
2904     */
2905     public Set<Map.Entry<K,V>> entrySet() {
2906     EntrySet<K,V> es = entrySet;
2907     return (es != null) ? es : (entrySet = new EntrySet<K,V>(this));
2908     }
2909    
2910     /**
2911     * Returns an enumeration of the keys in this table.
2912     *
2913     * @return an enumeration of the keys in this table
2914     * @see #keySet()
2915     */
2916     public Enumeration<K> keys() {
2917     return new KeyIterator<K,V>(this);
2918     }
2919    
2920     /**
2921     * Returns an enumeration of the values in this table.
2922     *
2923     * @return an enumeration of the values in this table
2924     * @see #values()
2925     */
2926     public Enumeration<V> elements() {
2927     return new ValueIterator<K,V>(this);
2928     }
2929    
2930     /**
2931 jsr166 1.122 * Returns a partitionable iterator of the keys in this map.
2932 dl 1.119 *
2933 jsr166 1.122 * @return a partitionable iterator of the keys in this map
2934 dl 1.119 */
2935     public Spliterator<K> keySpliterator() {
2936     return new KeyIterator<K,V>(this);
2937     }
2938    
2939     /**
2940 jsr166 1.122 * Returns a partitionable iterator of the values in this map.
2941 dl 1.119 *
2942 jsr166 1.122 * @return a partitionable iterator of the values in this map
2943 dl 1.119 */
2944     public Spliterator<V> valueSpliterator() {
2945     return new ValueIterator<K,V>(this);
2946     }
2947    
2948     /**
2949 jsr166 1.122 * Returns a partitionable iterator of the entries in this map.
2950 dl 1.119 *
2951 jsr166 1.122 * @return a partitionable iterator of the entries in this map
2952 dl 1.119 */
2953     public Spliterator<Map.Entry<K,V>> entrySpliterator() {
2954     return new EntryIterator<K,V>(this);
2955     }
2956    
2957     /**
2958     * Returns the hash code value for this {@link Map}, i.e.,
2959     * the sum of, for each key-value pair in the map,
2960     * {@code key.hashCode() ^ value.hashCode()}.
2961     *
2962     * @return the hash code value for this map
2963     */
2964     public int hashCode() {
2965     int h = 0;
2966     Traverser<K,V,Object> it = new Traverser<K,V,Object>(this);
2967     Object v;
2968     while ((v = it.advance()) != null) {
2969     h += it.nextKey.hashCode() ^ v.hashCode();
2970     }
2971     return h;
2972     }
2973    
2974     /**
2975     * Returns a string representation of this map. The string
2976     * representation consists of a list of key-value mappings (in no
2977     * particular order) enclosed in braces ("{@code {}}"). Adjacent
2978     * mappings are separated by the characters {@code ", "} (comma
2979     * and space). Each key-value mapping is rendered as the key
2980     * followed by an equals sign ("{@code =}") followed by the
2981     * associated value.
2982     *
2983     * @return a string representation of this map
2984     */
2985     public String toString() {
2986     Traverser<K,V,Object> it = new Traverser<K,V,Object>(this);
2987     StringBuilder sb = new StringBuilder();
2988     sb.append('{');
2989     Object v;
2990     if ((v = it.advance()) != null) {
2991     for (;;) {
2992     Object k = it.nextKey;
2993     sb.append(k == this ? "(this Map)" : k);
2994     sb.append('=');
2995     sb.append(v == this ? "(this Map)" : v);
2996     if ((v = it.advance()) == null)
2997     break;
2998     sb.append(',').append(' ');
2999     }
3000     }
3001     return sb.append('}').toString();
3002     }
3003    
3004     /**
3005     * Compares the specified object with this map for equality.
3006     * Returns {@code true} if the given object is a map with the same
3007     * mappings as this map. This operation may return misleading
3008     * results if either map is concurrently modified during execution
3009     * of this method.
3010     *
3011     * @param o object to be compared for equality with this map
3012     * @return {@code true} if the specified object is equal to this map
3013     */
3014     public boolean equals(Object o) {
3015     if (o != this) {
3016     if (!(o instanceof Map))
3017     return false;
3018     Map<?,?> m = (Map<?,?>) o;
3019     Traverser<K,V,Object> it = new Traverser<K,V,Object>(this);
3020     Object val;
3021     while ((val = it.advance()) != null) {
3022     Object v = m.get(it.nextKey);
3023     if (v == null || (v != val && !v.equals(val)))
3024     return false;
3025     }
3026     for (Map.Entry<?,?> e : m.entrySet()) {
3027     Object mk, mv, v;
3028     if ((mk = e.getKey()) == null ||
3029     (mv = e.getValue()) == null ||
3030     (v = internalGet(mk)) == null ||
3031     (mv != v && !mv.equals(v)))
3032     return false;
3033     }
3034     }
3035     return true;
3036     }
3037    
3038     /* ----------------Iterators -------------- */
3039    
3040 dl 1.128 @SuppressWarnings("serial") static final class KeyIterator<K,V> extends Traverser<K,V,Object>
3041 dl 1.119 implements Spliterator<K>, Enumeration<K> {
3042     KeyIterator(ConcurrentHashMap<K, V> map) { super(map); }
3043 dl 1.128 KeyIterator(Traverser<K,V,Object> it) {
3044     super(it);
3045 dl 1.119 }
3046     public KeyIterator<K,V> split() {
3047     if (last != null || (next != null && nextVal == null))
3048     throw new IllegalStateException();
3049 dl 1.128 return new KeyIterator<K,V>(this);
3050 dl 1.119 }
3051 dl 1.128 @SuppressWarnings("unchecked") public final K next() {
3052 dl 1.119 if (nextVal == null && advance() == null)
3053     throw new NoSuchElementException();
3054     Object k = nextKey;
3055     nextVal = null;
3056     return (K) k;
3057     }
3058    
3059     public final K nextElement() { return next(); }
3060     }
3061    
3062 dl 1.128 @SuppressWarnings("serial") static final class ValueIterator<K,V> extends Traverser<K,V,Object>
3063 dl 1.119 implements Spliterator<V>, Enumeration<V> {
3064     ValueIterator(ConcurrentHashMap<K, V> map) { super(map); }
3065 dl 1.128 ValueIterator(Traverser<K,V,Object> it) {
3066     super(it);
3067 dl 1.119 }
3068     public ValueIterator<K,V> split() {
3069     if (last != null || (next != null && nextVal == null))
3070     throw new IllegalStateException();
3071 dl 1.128 return new ValueIterator<K,V>(this);
3072 dl 1.119 }
3073    
3074 dl 1.128 @SuppressWarnings("unchecked") public final V next() {
3075 dl 1.119 Object v;
3076     if ((v = nextVal) == null && (v = advance()) == null)
3077     throw new NoSuchElementException();
3078     nextVal = null;
3079     return (V) v;
3080     }
3081    
3082     public final V nextElement() { return next(); }
3083     }
3084    
3085 dl 1.128 @SuppressWarnings("serial") static final class EntryIterator<K,V> extends Traverser<K,V,Object>
3086 dl 1.119 implements Spliterator<Map.Entry<K,V>> {
3087     EntryIterator(ConcurrentHashMap<K, V> map) { super(map); }
3088 dl 1.128 EntryIterator(Traverser<K,V,Object> it) {
3089     super(it);
3090 dl 1.119 }
3091     public EntryIterator<K,V> split() {
3092     if (last != null || (next != null && nextVal == null))
3093     throw new IllegalStateException();
3094 dl 1.128 return new EntryIterator<K,V>(this);
3095 dl 1.119 }
3096    
3097 dl 1.128 @SuppressWarnings("unchecked") public final Map.Entry<K,V> next() {
3098 dl 1.119 Object v;
3099     if ((v = nextVal) == null && (v = advance()) == null)
3100     throw new NoSuchElementException();
3101     Object k = nextKey;
3102     nextVal = null;
3103     return new MapEntry<K,V>((K)k, (V)v, map);
3104     }
3105     }
3106    
3107     /**
3108     * Exported Entry for iterators
3109     */
3110     static final class MapEntry<K,V> implements Map.Entry<K, V> {
3111     final K key; // non-null
3112     V val; // non-null
3113     final ConcurrentHashMap<K, V> map;
3114     MapEntry(K key, V val, ConcurrentHashMap<K, V> map) {
3115     this.key = key;
3116     this.val = val;
3117     this.map = map;
3118     }
3119     public final K getKey() { return key; }
3120     public final V getValue() { return val; }
3121     public final int hashCode() { return key.hashCode() ^ val.hashCode(); }
3122     public final String toString(){ return key + "=" + val; }
3123    
3124     public final boolean equals(Object o) {
3125     Object k, v; Map.Entry<?,?> e;
3126     return ((o instanceof Map.Entry) &&
3127     (k = (e = (Map.Entry<?,?>)o).getKey()) != null &&
3128     (v = e.getValue()) != null &&
3129     (k == key || k.equals(key)) &&
3130     (v == val || v.equals(val)));
3131     }
3132    
3133     /**
3134     * Sets our entry's value and writes through to the map. The
3135     * value to return is somewhat arbitrary here. Since we do not
3136     * necessarily track asynchronous changes, the most recent
3137     * "previous" value could be different from what we return (or
3138     * could even have been removed in which case the put will
3139     * re-establish). We do not and cannot guarantee more.
3140     */
3141     public final V setValue(V value) {
3142     if (value == null) throw new NullPointerException();
3143     V v = val;
3144     val = value;
3145     map.put(key, value);
3146     return v;
3147     }
3148     }
3149    
3150     /* ----------------Views -------------- */
3151    
3152     /**
3153     * Base class for views.
3154     */
3155     static abstract class CHMView<K, V> {
3156     final ConcurrentHashMap<K, V> map;
3157     CHMView(ConcurrentHashMap<K, V> map) { this.map = map; }
3158     public final int size() { return map.size(); }
3159     public final boolean isEmpty() { return map.isEmpty(); }
3160     public final void clear() { map.clear(); }
3161    
3162     // implementations below rely on concrete classes supplying these
3163     abstract public Iterator<?> iterator();
3164     abstract public boolean contains(Object o);
3165     abstract public boolean remove(Object o);
3166    
3167     private static final String oomeMsg = "Required array size too large";
3168    
3169     public final Object[] toArray() {
3170     long sz = map.mappingCount();
3171     if (sz > (long)(MAX_ARRAY_SIZE))
3172     throw new OutOfMemoryError(oomeMsg);
3173     int n = (int)sz;
3174     Object[] r = new Object[n];
3175     int i = 0;
3176     Iterator<?> it = iterator();
3177     while (it.hasNext()) {
3178     if (i == n) {
3179     if (n >= MAX_ARRAY_SIZE)
3180     throw new OutOfMemoryError(oomeMsg);
3181     if (n >= MAX_ARRAY_SIZE - (MAX_ARRAY_SIZE >>> 1) - 1)
3182     n = MAX_ARRAY_SIZE;
3183     else
3184     n += (n >>> 1) + 1;
3185     r = Arrays.copyOf(r, n);
3186     }
3187     r[i++] = it.next();
3188     }
3189     return (i == n) ? r : Arrays.copyOf(r, i);
3190     }
3191    
3192 dl 1.128 @SuppressWarnings("unchecked") public final <T> T[] toArray(T[] a) {
3193 dl 1.119 long sz = map.mappingCount();
3194     if (sz > (long)(MAX_ARRAY_SIZE))
3195     throw new OutOfMemoryError(oomeMsg);
3196     int m = (int)sz;
3197     T[] r = (a.length >= m) ? a :
3198     (T[])java.lang.reflect.Array
3199     .newInstance(a.getClass().getComponentType(), m);
3200     int n = r.length;
3201     int i = 0;
3202     Iterator<?> it = iterator();
3203     while (it.hasNext()) {
3204     if (i == n) {
3205     if (n >= MAX_ARRAY_SIZE)
3206     throw new OutOfMemoryError(oomeMsg);
3207     if (n >= MAX_ARRAY_SIZE - (MAX_ARRAY_SIZE >>> 1) - 1)
3208     n = MAX_ARRAY_SIZE;
3209     else
3210     n += (n >>> 1) + 1;
3211     r = Arrays.copyOf(r, n);
3212     }
3213     r[i++] = (T)it.next();
3214     }
3215     if (a == r && i < n) {
3216     r[i] = null; // null-terminate
3217     return r;
3218     }
3219     return (i == n) ? r : Arrays.copyOf(r, i);
3220     }
3221    
3222     public final int hashCode() {
3223     int h = 0;
3224     for (Iterator<?> it = iterator(); it.hasNext();)
3225     h += it.next().hashCode();
3226     return h;
3227     }
3228    
3229     public final String toString() {
3230     StringBuilder sb = new StringBuilder();
3231     sb.append('[');
3232     Iterator<?> it = iterator();
3233     if (it.hasNext()) {
3234     for (;;) {
3235     Object e = it.next();
3236     sb.append(e == this ? "(this Collection)" : e);
3237     if (!it.hasNext())
3238     break;
3239     sb.append(',').append(' ');
3240     }
3241     }
3242     return sb.append(']').toString();
3243     }
3244    
3245     public final boolean containsAll(Collection<?> c) {
3246     if (c != this) {
3247     for (Iterator<?> it = c.iterator(); it.hasNext();) {
3248     Object e = it.next();
3249     if (e == null || !contains(e))
3250     return false;
3251     }
3252     }
3253     return true;
3254     }
3255    
3256     public final boolean removeAll(Collection<?> c) {
3257     boolean modified = false;
3258     for (Iterator<?> it = iterator(); it.hasNext();) {
3259     if (c.contains(it.next())) {
3260     it.remove();
3261     modified = true;
3262     }
3263     }
3264     return modified;
3265     }
3266    
3267     public final boolean retainAll(Collection<?> c) {
3268     boolean modified = false;
3269     for (Iterator<?> it = iterator(); it.hasNext();) {
3270     if (!c.contains(it.next())) {
3271     it.remove();
3272     modified = true;
3273     }
3274     }
3275     return modified;
3276     }
3277    
3278     }
3279    
3280     static final class KeySet<K,V> extends CHMView<K,V> implements Set<K> {
3281     KeySet(ConcurrentHashMap<K, V> map) {
3282     super(map);
3283     }
3284     public final boolean contains(Object o) { return map.containsKey(o); }
3285     public final boolean remove(Object o) { return map.remove(o) != null; }
3286     public final Iterator<K> iterator() {
3287     return new KeyIterator<K,V>(map);
3288     }
3289     public final boolean add(K e) {
3290     throw new UnsupportedOperationException();
3291     }
3292     public final boolean addAll(Collection<? extends K> c) {
3293     throw new UnsupportedOperationException();
3294     }
3295     public boolean equals(Object o) {
3296     Set<?> c;
3297     return ((o instanceof Set) &&
3298     ((c = (Set<?>)o) == this ||
3299     (containsAll(c) && c.containsAll(this))));
3300     }
3301     }
3302    
3303    
3304     static final class Values<K,V> extends CHMView<K,V>
3305     implements Collection<V> {
3306     Values(ConcurrentHashMap<K, V> map) { super(map); }
3307     public final boolean contains(Object o) { return map.containsValue(o); }
3308     public final boolean remove(Object o) {
3309     if (o != null) {
3310     Iterator<V> it = new ValueIterator<K,V>(map);
3311     while (it.hasNext()) {
3312     if (o.equals(it.next())) {
3313     it.remove();
3314     return true;
3315     }
3316     }
3317     }
3318     return false;
3319     }
3320     public final Iterator<V> iterator() {
3321     return new ValueIterator<K,V>(map);
3322     }
3323     public final boolean add(V e) {
3324     throw new UnsupportedOperationException();
3325     }
3326     public final boolean addAll(Collection<? extends V> c) {
3327     throw new UnsupportedOperationException();
3328     }
3329    
3330     }
3331    
3332     static final class EntrySet<K,V> extends CHMView<K,V>
3333     implements Set<Map.Entry<K,V>> {
3334     EntrySet(ConcurrentHashMap<K, V> map) { super(map); }
3335     public final boolean contains(Object o) {
3336     Object k, v, r; Map.Entry<?,?> e;
3337     return ((o instanceof Map.Entry) &&
3338     (k = (e = (Map.Entry<?,?>)o).getKey()) != null &&
3339     (r = map.get(k)) != null &&
3340     (v = e.getValue()) != null &&
3341     (v == r || v.equals(r)));
3342     }
3343     public final boolean remove(Object o) {
3344     Object k, v; Map.Entry<?,?> e;
3345     return ((o instanceof Map.Entry) &&
3346     (k = (e = (Map.Entry<?,?>)o).getKey()) != null &&
3347     (v = e.getValue()) != null &&
3348     map.remove(k, v));
3349     }
3350     public final Iterator<Map.Entry<K,V>> iterator() {
3351     return new EntryIterator<K,V>(map);
3352     }
3353     public final boolean add(Entry<K,V> e) {
3354     throw new UnsupportedOperationException();
3355     }
3356     public final boolean addAll(Collection<? extends Entry<K,V>> c) {
3357     throw new UnsupportedOperationException();
3358     }
3359     public boolean equals(Object o) {
3360     Set<?> c;
3361     return ((o instanceof Set) &&
3362     ((c = (Set<?>)o) == this ||
3363     (containsAll(c) && c.containsAll(this))));
3364     }
3365     }
3366    
3367     /* ---------------- Serialization Support -------------- */
3368    
3369     /**
3370     * Stripped-down version of helper class used in previous version,
3371     * declared for the sake of serialization compatibility
3372     */
3373     static class Segment<K,V> implements Serializable {
3374     private static final long serialVersionUID = 2249069246763182397L;
3375     final float loadFactor;
3376     Segment(float lf) { this.loadFactor = lf; }
3377     }
3378    
3379     /**
3380     * Saves the state of the {@code ConcurrentHashMap} instance to a
3381     * stream (i.e., serializes it).
3382     * @param s the stream
3383     * @serialData
3384     * the key (Object) and value (Object)
3385     * for each key-value mapping, followed by a null pair.
3386     * The key-value mappings are emitted in no particular order.
3387     */
3388 dl 1.128 @SuppressWarnings("unchecked") private void writeObject(java.io.ObjectOutputStream s)
3389 dl 1.119 throws java.io.IOException {
3390     if (segments == null) { // for serialization compatibility
3391     segments = (Segment<K,V>[])
3392     new Segment<?,?>[DEFAULT_CONCURRENCY_LEVEL];
3393     for (int i = 0; i < segments.length; ++i)
3394     segments[i] = new Segment<K,V>(LOAD_FACTOR);
3395     }
3396     s.defaultWriteObject();
3397     Traverser<K,V,Object> it = new Traverser<K,V,Object>(this);
3398     Object v;
3399     while ((v = it.advance()) != null) {
3400     s.writeObject(it.nextKey);
3401     s.writeObject(v);
3402     }
3403     s.writeObject(null);
3404     s.writeObject(null);
3405     segments = null; // throw away
3406     }
3407    
3408     /**
3409     * Reconstitutes the instance from a stream (that is, deserializes it).
3410     * @param s the stream
3411     */
3412 dl 1.128 @SuppressWarnings("unchecked") private void readObject(java.io.ObjectInputStream s)
3413 dl 1.119 throws java.io.IOException, ClassNotFoundException {
3414     s.defaultReadObject();
3415     this.segments = null; // unneeded
3416     // initialize transient final field
3417     UNSAFE.putObjectVolatile(this, counterOffset, new LongAdder());
3418    
3419     // Create all nodes, then place in table once size is known
3420     long size = 0L;
3421     Node p = null;
3422     for (;;) {
3423     K k = (K) s.readObject();
3424     V v = (V) s.readObject();
3425     if (k != null && v != null) {
3426     int h = spread(k.hashCode());
3427     p = new Node(h, k, v, p);
3428     ++size;
3429     }
3430     else
3431     break;
3432     }
3433     if (p != null) {
3434     boolean init = false;
3435     int n;
3436     if (size >= (long)(MAXIMUM_CAPACITY >>> 1))
3437     n = MAXIMUM_CAPACITY;
3438     else {
3439     int sz = (int)size;
3440     n = tableSizeFor(sz + (sz >>> 1) + 1);
3441     }
3442     int sc = sizeCtl;
3443     boolean collide = false;
3444     if (n > sc &&
3445     UNSAFE.compareAndSwapInt(this, sizeCtlOffset, sc, -1)) {
3446     try {
3447     if (table == null) {
3448     init = true;
3449     Node[] tab = new Node[n];
3450     int mask = n - 1;
3451     while (p != null) {
3452     int j = p.hash & mask;
3453     Node next = p.next;
3454     Node q = p.next = tabAt(tab, j);
3455     setTabAt(tab, j, p);
3456     if (!collide && q != null && q.hash == p.hash)
3457     collide = true;
3458     p = next;
3459     }
3460     table = tab;
3461     counter.add(size);
3462     sc = n - (n >>> 2);
3463     }
3464     } finally {
3465     sizeCtl = sc;
3466     }
3467     if (collide) { // rescan and convert to TreeBins
3468     Node[] tab = table;
3469     for (int i = 0; i < tab.length; ++i) {
3470     int c = 0;
3471     for (Node e = tabAt(tab, i); e != null; e = e.next) {
3472     if (++c > TREE_THRESHOLD &&
3473     (e.key instanceof Comparable)) {
3474     replaceWithTreeBin(tab, i, e.key);
3475     break;
3476     }
3477     }
3478     }
3479     }
3480     }
3481     if (!init) { // Can only happen if unsafely published.
3482     while (p != null) {
3483     internalPut(p.key, p.val);
3484     p = p.next;
3485     }
3486     }
3487     }
3488     }
3489    
3490    
3491     // -------------------------------------------------------
3492    
3493     // Sams
3494     /** Interface describing a void action of one argument */
3495     public interface Action<A> { void apply(A a); }
3496     /** Interface describing a void action of two arguments */
3497     public interface BiAction<A,B> { void apply(A a, B b); }
3498     /** Interface describing a function of one argument */
3499     public interface Fun<A,T> { T apply(A a); }
3500     /** Interface describing a function of two arguments */
3501     public interface BiFun<A,B,T> { T apply(A a, B b); }
3502     /** Interface describing a function of no arguments */
3503     public interface Generator<T> { T apply(); }
3504     /** Interface describing a function mapping its argument to a double */
3505     public interface ObjectToDouble<A> { double apply(A a); }
3506     /** Interface describing a function mapping its argument to a long */
3507     public interface ObjectToLong<A> { long apply(A a); }
3508     /** Interface describing a function mapping its argument to an int */
3509     public interface ObjectToInt<A> {int apply(A a); }
3510     /** Interface describing a function mapping two arguments to a double */
3511     public interface ObjectByObjectToDouble<A,B> { double apply(A a, B b); }
3512     /** Interface describing a function mapping two arguments to a long */
3513     public interface ObjectByObjectToLong<A,B> { long apply(A a, B b); }
3514     /** Interface describing a function mapping two arguments to an int */
3515     public interface ObjectByObjectToInt<A,B> {int apply(A a, B b); }
3516     /** Interface describing a function mapping a double to a double */
3517     public interface DoubleToDouble { double apply(double a); }
3518     /** Interface describing a function mapping a long to a long */
3519     public interface LongToLong { long apply(long a); }
3520     /** Interface describing a function mapping an int to an int */
3521     public interface IntToInt { int apply(int a); }
3522     /** Interface describing a function mapping two doubles to a double */
3523     public interface DoubleByDoubleToDouble { double apply(double a, double b); }
3524     /** Interface describing a function mapping two longs to a long */
3525     public interface LongByLongToLong { long apply(long a, long b); }
3526     /** Interface describing a function mapping two ints to an int */
3527     public interface IntByIntToInt { int apply(int a, int b); }
3528    
3529    
3530     // -------------------------------------------------------
3531    
3532     /**
3533     * Returns an extended {@link Parallel} view of this map using the
3534     * given executor for bulk parallel operations.
3535     *
3536     * @param executor the executor
3537     * @return a parallel view
3538     */
3539     public Parallel parallel(ForkJoinPool executor) {
3540     return new Parallel(executor);
3541     }
3542    
3543     /**
3544     * An extended view of a ConcurrentHashMap supporting bulk
3545 jsr166 1.121 * parallel operations. These operations are designed to be
3546 dl 1.119 * safely, and often sensibly, applied even with maps that are
3547     * being concurrently updated by other threads; for example, when
3548     * computing a snapshot summary of the values in a shared
3549     * registry. There are three kinds of operation, each with four
3550     * forms, accepting functions with Keys, Values, Entries, and
3551     * (Key, Value) arguments and/or return values. Because the
3552     * elements of a ConcurrentHashMap are not ordered in any
3553     * particular way, and may be processed in different orders in
3554     * different parallel executions, the correctness of supplied
3555     * functions should not depend on any ordering, or on any other
3556     * objects or values that may transiently change while computation
3557     * is in progress; and except for forEach actions, should ideally
3558     * be side-effect-free.
3559     *
3560     * <ul>
3561     * <li> forEach: Perform a given action on each element.
3562     * A variant form applies a given transformation on each element
3563     * before performing the action.</li>
3564     *
3565     * <li> search: Return the first available non-null result of
3566     * applying a given function on each element; skipping further
3567     * search when a result is found.</li>
3568     *
3569     * <li> reduce: Accumulate each element. The supplied reduction
3570     * function cannot rely on ordering (more formally, it should be
3571     * both associative and commutative). There are five variants:
3572     *
3573     * <ul>
3574     *
3575     * <li> Plain reductions. (There is not a form of this method for
3576     * (key, value) function arguments since there is no corresponding
3577     * return type.)</li>
3578     *
3579     * <li> Mapped reductions that accumulate the results of a given
3580     * function applied to each element.</li>
3581     *
3582     * <li> Reductions to scalar doubles, longs, and ints, using a
3583     * given basis value.</li>
3584     *
3585     * </li>
3586     * </ul>
3587     * </ul>
3588     *
3589     * <p>The concurrency properties of the bulk operations follow
3590     * from those of ConcurrentHashMap: Any non-null result returned
3591     * from {@code get(key)} and related access methods bears a
3592     * happens-before relation with the associated insertion or
3593     * update. The result of any bulk operation reflects the
3594     * composition of these per-element relations (but is not
3595     * necessarily atomic with respect to the map as a whole unless it
3596     * is somehow known to be quiescent). Conversely, because keys
3597     * and values in the map are never null, null serves as a reliable
3598     * atomic indicator of the current lack of any result. To
3599     * maintain this property, null serves as an implicit basis for
3600     * all non-scalar reduction operations. For the double, long, and
3601     * int versions, the basis should be one that, when combined with
3602     * any other value, returns that other value (more formally, it
3603     * should be the identity element for the reduction). Most common
3604     * reductions have these properties; for example, computing a sum
3605     * with basis 0 or a minimum with basis MAX_VALUE.
3606     *
3607     * <p>Search and transformation functions provided as arguments
3608     * should similarly return null to indicate the lack of any result
3609     * (in which case it is not used). In the case of mapped
3610     * reductions, this also enables transformations to serve as
3611     * filters, returning null (or, in the case of primitive
3612     * specializations, the identity basis) if the element should not
3613     * be combined. You can create compound transformations and
3614     * filterings by composing them yourself under this "null means
3615     * there is nothing there now" rule before using them in search or
3616     * reduce operations.
3617     *
3618     * <p>Methods accepting and/or returning Entry arguments maintain
3619     * key-value associations. They may be useful for example when
3620     * finding the key for the greatest value. Note that "plain" Entry
3621     * arguments can be supplied using {@code new
3622     * AbstractMap.SimpleEntry(k,v)}.
3623     *
3624     * <p> Bulk operations may complete abruptly, throwing an
3625     * exception encountered in the application of a supplied
3626     * function. Bear in mind when handling such exceptions that other
3627     * concurrently executing functions could also have thrown
3628     * exceptions, or would have done so if the first exception had
3629     * not occurred.
3630     *
3631     * <p>Parallel speedups compared to sequential processing are
3632     * common but not guaranteed. Operations involving brief
3633     * functions on small maps may execute more slowly than sequential
3634     * loops if the underlying work to parallelize the computation is
3635     * more expensive than the computation itself. Similarly,
3636     * parallelization may not lead to much actual parallelism if all
3637     * processors are busy performing unrelated tasks.
3638     *
3639     * <p> All arguments to all task methods must be non-null.
3640     *
3641     * <p><em>jsr166e note: During transition, this class
3642     * uses nested functional interfaces with different names but the
3643     * same forms as those expected for JDK8.<em>
3644     */
3645     public class Parallel {
3646     final ForkJoinPool fjp;
3647    
3648     /**
3649     * Returns an extended view of this map using the given
3650     * executor for bulk parallel operations.
3651     *
3652     * @param executor the executor
3653     */
3654     public Parallel(ForkJoinPool executor) {
3655     this.fjp = executor;
3656     }
3657    
3658     /**
3659     * Performs the given action for each (key, value).
3660     *
3661     * @param action the action
3662     */
3663 jsr166 1.120 public void forEach(BiAction<K,V> action) {
3664 dl 1.119 fjp.invoke(ForkJoinTasks.forEach
3665     (ConcurrentHashMap.this, action));
3666     }
3667    
3668     /**
3669     * Performs the given action for each non-null transformation
3670     * of each (key, value).
3671     *
3672     * @param transformer a function returning the transformation
3673     * for an element, or null of there is no transformation (in
3674     * which case the action is not applied).
3675     * @param action the action
3676     */
3677 jsr166 1.120 public <U> void forEach(BiFun<? super K, ? super V, ? extends U> transformer,
3678 dl 1.119 Action<U> action) {
3679     fjp.invoke(ForkJoinTasks.forEach
3680     (ConcurrentHashMap.this, transformer, action));
3681     }
3682    
3683     /**
3684     * Returns a non-null result from applying the given search
3685 dl 1.126 * function on each (key, value), or null if none. Upon
3686     * success, further element processing is suppressed and the
3687     * results of any other parallel invocations of the search
3688     * function are ignored.
3689 dl 1.119 *
3690     * @param searchFunction a function returning a non-null
3691     * result on success, else null
3692     * @return a non-null result from applying the given search
3693     * function on each (key, value), or null if none
3694     */
3695     public <U> U search(BiFun<? super K, ? super V, ? extends U> searchFunction) {
3696     return fjp.invoke(ForkJoinTasks.search
3697     (ConcurrentHashMap.this, searchFunction));
3698     }
3699    
3700     /**
3701     * Returns the result of accumulating the given transformation
3702     * of all (key, value) pairs using the given reducer to
3703     * combine values, or null if none.
3704     *
3705     * @param transformer a function returning the transformation
3706     * for an element, or null of there is no transformation (in
3707     * which case it is not combined).
3708     * @param reducer a commutative associative combining function
3709     * @return the result of accumulating the given transformation
3710     * of all (key, value) pairs
3711     */
3712     public <U> U reduce(BiFun<? super K, ? super V, ? extends U> transformer,
3713     BiFun<? super U, ? super U, ? extends U> reducer) {
3714     return fjp.invoke(ForkJoinTasks.reduce
3715     (ConcurrentHashMap.this, transformer, reducer));
3716     }
3717    
3718     /**
3719     * Returns the result of accumulating the given transformation
3720     * of all (key, value) pairs using the given reducer to
3721     * combine values, and the given basis as an identity value.
3722     *
3723     * @param transformer a function returning the transformation
3724     * for an element
3725     * @param basis the identity (initial default value) for the reduction
3726     * @param reducer a commutative associative combining function
3727     * @return the result of accumulating the given transformation
3728     * of all (key, value) pairs
3729     */
3730     public double reduceToDouble(ObjectByObjectToDouble<? super K, ? super V> transformer,
3731     double basis,
3732     DoubleByDoubleToDouble reducer) {
3733     return fjp.invoke(ForkJoinTasks.reduceToDouble
3734     (ConcurrentHashMap.this, transformer, basis, reducer));
3735     }
3736    
3737     /**
3738     * Returns the result of accumulating the given transformation
3739     * of all (key, value) pairs using the given reducer to
3740     * combine values, and the given basis as an identity value.
3741     *
3742     * @param transformer a function returning the transformation
3743     * for an element
3744     * @param basis the identity (initial default value) for the reduction
3745     * @param reducer a commutative associative combining function
3746     * @return the result of accumulating the given transformation
3747 dl 1.126 * of all (key, value) pairs
3748 dl 1.119 */
3749     public long reduceToLong(ObjectByObjectToLong<? super K, ? super V> transformer,
3750     long basis,
3751     LongByLongToLong reducer) {
3752     return fjp.invoke(ForkJoinTasks.reduceToLong
3753     (ConcurrentHashMap.this, transformer, basis, reducer));
3754     }
3755    
3756     /**
3757     * Returns the result of accumulating the given transformation
3758     * of all (key, value) pairs using the given reducer to
3759     * combine values, and the given basis as an identity value.
3760     *
3761     * @param transformer a function returning the transformation
3762     * for an element
3763     * @param basis the identity (initial default value) for the reduction
3764     * @param reducer a commutative associative combining function
3765     * @return the result of accumulating the given transformation
3766     * of all (key, value) pairs
3767     */
3768     public int reduceToInt(ObjectByObjectToInt<? super K, ? super V> transformer,
3769     int basis,
3770     IntByIntToInt reducer) {
3771     return fjp.invoke(ForkJoinTasks.reduceToInt
3772     (ConcurrentHashMap.this, transformer, basis, reducer));
3773     }
3774    
3775     /**
3776 jsr166 1.123 * Performs the given action for each key.
3777 dl 1.119 *
3778     * @param action the action
3779     */
3780 jsr166 1.120 public void forEachKey(Action<K> action) {
3781 dl 1.119 fjp.invoke(ForkJoinTasks.forEachKey
3782     (ConcurrentHashMap.this, action));
3783     }
3784    
3785     /**
3786     * Performs the given action for each non-null transformation
3787 jsr166 1.123 * of each key.
3788 dl 1.119 *
3789     * @param transformer a function returning the transformation
3790     * for an element, or null of there is no transformation (in
3791     * which case the action is not applied).
3792     * @param action the action
3793     */
3794 jsr166 1.120 public <U> void forEachKey(Fun<? super K, ? extends U> transformer,
3795 dl 1.119 Action<U> action) {
3796     fjp.invoke(ForkJoinTasks.forEachKey
3797     (ConcurrentHashMap.this, transformer, action));
3798     }
3799    
3800     /**
3801     * Returns a non-null result from applying the given search
3802 dl 1.126 * function on each key, or null if none. Upon success,
3803     * further element processing is suppressed and the results of
3804     * any other parallel invocations of the search function are
3805     * ignored.
3806 dl 1.119 *
3807     * @param searchFunction a function returning a non-null
3808     * result on success, else null
3809     * @return a non-null result from applying the given search
3810     * function on each key, or null if none
3811     */
3812     public <U> U searchKeys(Fun<? super K, ? extends U> searchFunction) {
3813     return fjp.invoke(ForkJoinTasks.searchKeys
3814     (ConcurrentHashMap.this, searchFunction));
3815     }
3816    
3817     /**
3818     * Returns the result of accumulating all keys using the given
3819     * reducer to combine values, or null if none.
3820     *
3821     * @param reducer a commutative associative combining function
3822     * @return the result of accumulating all keys using the given
3823     * reducer to combine values, or null if none
3824     */
3825     public K reduceKeys(BiFun<? super K, ? super K, ? extends K> reducer) {
3826     return fjp.invoke(ForkJoinTasks.reduceKeys
3827     (ConcurrentHashMap.this, reducer));
3828     }
3829    
3830     /**
3831     * Returns the result of accumulating the given transformation
3832     * of all keys using the given reducer to combine values, or
3833     * null if none.
3834     *
3835     * @param transformer a function returning the transformation
3836     * for an element, or null of there is no transformation (in
3837     * which case it is not combined).
3838     * @param reducer a commutative associative combining function
3839     * @return the result of accumulating the given transformation
3840     * of all keys
3841     */
3842     public <U> U reduceKeys(Fun<? super K, ? extends U> transformer,
3843     BiFun<? super U, ? super U, ? extends U> reducer) {
3844     return fjp.invoke(ForkJoinTasks.reduceKeys
3845     (ConcurrentHashMap.this, transformer, reducer));
3846     }
3847    
3848     /**
3849     * Returns the result of accumulating the given transformation
3850     * of all keys using the given reducer to combine values, and
3851     * the given basis as an identity value.
3852     *
3853     * @param transformer a function returning the transformation
3854     * for an element
3855     * @param basis the identity (initial default value) for the reduction
3856     * @param reducer a commutative associative combining function
3857     * @return the result of accumulating the given transformation
3858     * of all keys
3859     */
3860     public double reduceKeysToDouble(ObjectToDouble<? super K> transformer,
3861     double basis,
3862     DoubleByDoubleToDouble reducer) {
3863     return fjp.invoke(ForkJoinTasks.reduceKeysToDouble
3864     (ConcurrentHashMap.this, transformer, basis, reducer));
3865     }
3866    
3867     /**
3868     * Returns the result of accumulating the given transformation
3869     * of all keys using the given reducer to combine values, and
3870     * the given basis as an identity value.
3871     *
3872     * @param transformer a function returning the transformation
3873     * for an element
3874     * @param basis the identity (initial default value) for the reduction
3875     * @param reducer a commutative associative combining function
3876     * @return the result of accumulating the given transformation
3877     * of all keys
3878     */
3879     public long reduceKeysToLong(ObjectToLong<? super K> transformer,
3880     long basis,
3881     LongByLongToLong reducer) {
3882     return fjp.invoke(ForkJoinTasks.reduceKeysToLong
3883     (ConcurrentHashMap.this, transformer, basis, reducer));
3884     }
3885    
3886     /**
3887     * Returns the result of accumulating the given transformation
3888     * of all keys using the given reducer to combine values, and
3889     * the given basis as an identity value.
3890     *
3891     * @param transformer a function returning the transformation
3892     * for an element
3893     * @param basis the identity (initial default value) for the reduction
3894     * @param reducer a commutative associative combining function
3895     * @return the result of accumulating the given transformation
3896     * of all keys
3897     */
3898     public int reduceKeysToInt(ObjectToInt<? super K> transformer,
3899     int basis,
3900     IntByIntToInt reducer) {
3901     return fjp.invoke(ForkJoinTasks.reduceKeysToInt
3902     (ConcurrentHashMap.this, transformer, basis, reducer));
3903     }
3904    
3905     /**
3906 jsr166 1.123 * Performs the given action for each value.
3907 dl 1.119 *
3908     * @param action the action
3909     */
3910 jsr166 1.120 public void forEachValue(Action<V> action) {
3911 dl 1.119 fjp.invoke(ForkJoinTasks.forEachValue
3912     (ConcurrentHashMap.this, action));
3913     }
3914    
3915     /**
3916     * Performs the given action for each non-null transformation
3917 jsr166 1.123 * of each value.
3918 dl 1.119 *
3919     * @param transformer a function returning the transformation
3920     * for an element, or null of there is no transformation (in
3921     * which case the action is not applied).
3922     */
3923 jsr166 1.120 public <U> void forEachValue(Fun<? super V, ? extends U> transformer,
3924 dl 1.119 Action<U> action) {
3925     fjp.invoke(ForkJoinTasks.forEachValue
3926     (ConcurrentHashMap.this, transformer, action));
3927     }
3928    
3929     /**
3930     * Returns a non-null result from applying the given search
3931 dl 1.126 * function on each value, or null if none. Upon success,
3932     * further element processing is suppressed and the results of
3933     * any other parallel invocations of the search function are
3934     * ignored.
3935 dl 1.119 *
3936     * @param searchFunction a function returning a non-null
3937     * result on success, else null
3938     * @return a non-null result from applying the given search
3939     * function on each value, or null if none
3940     *
3941     */
3942     public <U> U searchValues(Fun<? super V, ? extends U> searchFunction) {
3943     return fjp.invoke(ForkJoinTasks.searchValues
3944     (ConcurrentHashMap.this, searchFunction));
3945     }
3946    
3947     /**
3948     * Returns the result of accumulating all values using the
3949     * given reducer to combine values, or null if none.
3950     *
3951     * @param reducer a commutative associative combining function
3952     * @return the result of accumulating all values
3953     */
3954     public V reduceValues(BiFun<? super V, ? super V, ? extends V> reducer) {
3955     return fjp.invoke(ForkJoinTasks.reduceValues
3956     (ConcurrentHashMap.this, reducer));
3957     }
3958    
3959     /**
3960     * Returns the result of accumulating the given transformation
3961     * of all values using the given reducer to combine values, or
3962     * null if none.
3963     *
3964     * @param transformer a function returning the transformation
3965     * for an element, or null of there is no transformation (in
3966     * which case it is not combined).
3967     * @param reducer a commutative associative combining function
3968     * @return the result of accumulating the given transformation
3969     * of all values
3970     */
3971     public <U> U reduceValues(Fun<? super V, ? extends U> transformer,
3972     BiFun<? super U, ? super U, ? extends U> reducer) {
3973     return fjp.invoke(ForkJoinTasks.reduceValues
3974     (ConcurrentHashMap.this, transformer, reducer));
3975     }
3976    
3977     /**
3978     * Returns the result of accumulating the given transformation
3979     * of all values using the given reducer to combine values,
3980     * and the given basis as an identity value.
3981     *
3982     * @param transformer a function returning the transformation
3983     * for an element
3984     * @param basis the identity (initial default value) for the reduction
3985     * @param reducer a commutative associative combining function
3986     * @return the result of accumulating the given transformation
3987     * of all values
3988     */
3989     public double reduceValuesToDouble(ObjectToDouble<? super V> transformer,
3990     double basis,
3991     DoubleByDoubleToDouble reducer) {
3992     return fjp.invoke(ForkJoinTasks.reduceValuesToDouble
3993     (ConcurrentHashMap.this, transformer, basis, reducer));
3994     }
3995    
3996     /**
3997     * Returns the result of accumulating the given transformation
3998     * of all values using the given reducer to combine values,
3999     * and the given basis as an identity value.
4000     *
4001     * @param transformer a function returning the transformation
4002     * for an element
4003     * @param basis the identity (initial default value) for the reduction
4004     * @param reducer a commutative associative combining function
4005     * @return the result of accumulating the given transformation
4006     * of all values
4007     */
4008     public long reduceValuesToLong(ObjectToLong<? super V> transformer,
4009     long basis,
4010     LongByLongToLong reducer) {
4011     return fjp.invoke(ForkJoinTasks.reduceValuesToLong
4012     (ConcurrentHashMap.this, transformer, basis, reducer));
4013     }
4014    
4015     /**
4016     * Returns the result of accumulating the given transformation
4017     * of all values using the given reducer to combine values,
4018     * and the given basis as an identity value.
4019     *
4020     * @param transformer a function returning the transformation
4021     * for an element
4022     * @param basis the identity (initial default value) for the reduction
4023     * @param reducer a commutative associative combining function
4024     * @return the result of accumulating the given transformation
4025     * of all values
4026     */
4027     public int reduceValuesToInt(ObjectToInt<? super V> transformer,
4028     int basis,
4029     IntByIntToInt reducer) {
4030     return fjp.invoke(ForkJoinTasks.reduceValuesToInt
4031     (ConcurrentHashMap.this, transformer, basis, reducer));
4032     }
4033    
4034     /**
4035 jsr166 1.123 * Performs the given action for each entry.
4036 dl 1.119 *
4037     * @param action the action
4038     */
4039 jsr166 1.120 public void forEachEntry(Action<Map.Entry<K,V>> action) {
4040 dl 1.119 fjp.invoke(ForkJoinTasks.forEachEntry
4041     (ConcurrentHashMap.this, action));
4042     }
4043    
4044     /**
4045 jsr166 1.123 * Performs the given action for each non-null transformation
4046     * of each entry.
4047 dl 1.119 *
4048     * @param transformer a function returning the transformation
4049     * for an element, or null of there is no transformation (in
4050     * which case the action is not applied).
4051     * @param action the action
4052     */
4053 jsr166 1.120 public <U> void forEachEntry(Fun<Map.Entry<K,V>, ? extends U> transformer,
4054 dl 1.119 Action<U> action) {
4055     fjp.invoke(ForkJoinTasks.forEachEntry
4056     (ConcurrentHashMap.this, transformer, action));
4057     }
4058    
4059     /**
4060     * Returns a non-null result from applying the given search
4061 dl 1.126 * function on each entry, or null if none. Upon success,
4062     * further element processing is suppressed and the results of
4063     * any other parallel invocations of the search function are
4064     * ignored.
4065 dl 1.119 *
4066     * @param searchFunction a function returning a non-null
4067     * result on success, else null
4068     * @return a non-null result from applying the given search
4069     * function on each entry, or null if none
4070     */
4071     public <U> U searchEntries(Fun<Map.Entry<K,V>, ? extends U> searchFunction) {
4072     return fjp.invoke(ForkJoinTasks.searchEntries
4073     (ConcurrentHashMap.this, searchFunction));
4074     }
4075    
4076     /**
4077     * Returns the result of accumulating all entries using the
4078     * given reducer to combine values, or null if none.
4079     *
4080     * @param reducer a commutative associative combining function
4081     * @return the result of accumulating all entries
4082     */
4083     public Map.Entry<K,V> reduceEntries(BiFun<Map.Entry<K,V>, Map.Entry<K,V>, ? extends Map.Entry<K,V>> reducer) {
4084     return fjp.invoke(ForkJoinTasks.reduceEntries
4085     (ConcurrentHashMap.this, reducer));
4086     }
4087    
4088     /**
4089     * Returns the result of accumulating the given transformation
4090     * of all entries using the given reducer to combine values,
4091     * or null if none.
4092     *
4093     * @param transformer a function returning the transformation
4094     * for an element, or null of there is no transformation (in
4095     * which case it is not combined).
4096     * @param reducer a commutative associative combining function
4097     * @return the result of accumulating the given transformation
4098     * of all entries
4099     */
4100     public <U> U reduceEntries(Fun<Map.Entry<K,V>, ? extends U> transformer,
4101     BiFun<? super U, ? super U, ? extends U> reducer) {
4102     return fjp.invoke(ForkJoinTasks.reduceEntries
4103     (ConcurrentHashMap.this, transformer, reducer));
4104     }
4105    
4106     /**
4107     * Returns the result of accumulating the given transformation
4108     * of all entries using the given reducer to combine values,
4109     * and the given basis as an identity value.
4110     *
4111     * @param transformer a function returning the transformation
4112     * for an element
4113     * @param basis the identity (initial default value) for the reduction
4114     * @param reducer a commutative associative combining function
4115     * @return the result of accumulating the given transformation
4116     * of all entries
4117     */
4118     public double reduceEntriesToDouble(ObjectToDouble<Map.Entry<K,V>> transformer,
4119     double basis,
4120     DoubleByDoubleToDouble reducer) {
4121     return fjp.invoke(ForkJoinTasks.reduceEntriesToDouble
4122     (ConcurrentHashMap.this, transformer, basis, reducer));
4123     }
4124    
4125     /**
4126     * Returns the result of accumulating the given transformation
4127     * of all entries using the given reducer to combine values,
4128     * and the given basis as an identity value.
4129     *
4130     * @param transformer a function returning the transformation
4131     * for an element
4132     * @param basis the identity (initial default value) for the reduction
4133     * @param reducer a commutative associative combining function
4134     * @return the result of accumulating the given transformation
4135     * of all entries
4136     */
4137     public long reduceEntriesToLong(ObjectToLong<Map.Entry<K,V>> transformer,
4138     long basis,
4139     LongByLongToLong reducer) {
4140     return fjp.invoke(ForkJoinTasks.reduceEntriesToLong
4141     (ConcurrentHashMap.this, transformer, basis, reducer));
4142     }
4143    
4144     /**
4145     * Returns the result of accumulating the given transformation
4146     * of all entries using the given reducer to combine values,
4147     * and the given basis as an identity value.
4148     *
4149     * @param transformer a function returning the transformation
4150     * for an element
4151     * @param basis the identity (initial default value) for the reduction
4152     * @param reducer a commutative associative combining function
4153     * @return the result of accumulating the given transformation
4154     * of all entries
4155     */
4156     public int reduceEntriesToInt(ObjectToInt<Map.Entry<K,V>> transformer,
4157     int basis,
4158     IntByIntToInt reducer) {
4159     return fjp.invoke(ForkJoinTasks.reduceEntriesToInt
4160     (ConcurrentHashMap.this, transformer, basis, reducer));
4161     }
4162     }
4163    
4164     // ---------------------------------------------------------------------
4165    
4166     /**
4167     * Predefined tasks for performing bulk parallel operations on
4168     * ConcurrentHashMaps. These tasks follow the forms and rules used
4169     * in class {@link Parallel}. Each method has the same name, but
4170     * returns a task rather than invoking it. These methods may be
4171     * useful in custom applications such as submitting a task without
4172     * waiting for completion, or combining with other tasks.
4173     */
4174     public static class ForkJoinTasks {
4175     private ForkJoinTasks() {}
4176    
4177     /**
4178     * Returns a task that when invoked, performs the given
4179     * action for each (key, value)
4180     *
4181     * @param map the map
4182     * @param action the action
4183     * @return the task
4184     */
4185 jsr166 1.120 public static <K,V> ForkJoinTask<Void> forEach
4186 dl 1.119 (ConcurrentHashMap<K,V> map,
4187     BiAction<K,V> action) {
4188     if (action == null) throw new NullPointerException();
4189 dl 1.130 return new ForEachMappingTask<K,V>(map, null, -1, action);
4190 dl 1.119 }
4191    
4192     /**
4193     * Returns a task that when invoked, performs the given
4194     * action for each non-null transformation of each (key, value)
4195     *
4196     * @param map the map
4197     * @param transformer a function returning the transformation
4198     * for an element, or null of there is no transformation (in
4199     * which case the action is not applied).
4200     * @param action the action
4201     * @return the task
4202     */
4203 jsr166 1.120 public static <K,V,U> ForkJoinTask<Void> forEach
4204 dl 1.119 (ConcurrentHashMap<K,V> map,
4205     BiFun<? super K, ? super V, ? extends U> transformer,
4206     Action<U> action) {
4207     if (transformer == null || action == null)
4208     throw new NullPointerException();
4209     return new ForEachTransformedMappingTask<K,V,U>
4210 dl 1.130 (map, null, -1, transformer, action);
4211 dl 1.119 }
4212    
4213     /**
4214 dl 1.126 * Returns a task that when invoked, returns a non-null result
4215     * from applying the given search function on each (key,
4216     * value), or null if none. Upon success, further element
4217     * processing is suppressed and the results of any other
4218     * parallel invocations of the search function are ignored.
4219 dl 1.119 *
4220     * @param map the map
4221     * @param searchFunction a function returning a non-null
4222     * result on success, else null
4223     * @return the task
4224     */
4225     public static <K,V,U> ForkJoinTask<U> search
4226     (ConcurrentHashMap<K,V> map,
4227     BiFun<? super K, ? super V, ? extends U> searchFunction) {
4228     if (searchFunction == null) throw new NullPointerException();
4229     return new SearchMappingsTask<K,V,U>
4230 dl 1.130 (map, null, -1, searchFunction,
4231 dl 1.119 new AtomicReference<U>());
4232     }
4233    
4234     /**
4235     * Returns a task that when invoked, returns the result of
4236     * accumulating the given transformation of all (key, value) pairs
4237     * using the given reducer to combine values, or null if none.
4238     *
4239     * @param map the map
4240     * @param transformer a function returning the transformation
4241     * for an element, or null of there is no transformation (in
4242     * which case it is not combined).
4243     * @param reducer a commutative associative combining function
4244     * @return the task
4245     */
4246     public static <K,V,U> ForkJoinTask<U> reduce
4247     (ConcurrentHashMap<K,V> map,
4248     BiFun<? super K, ? super V, ? extends U> transformer,
4249     BiFun<? super U, ? super U, ? extends U> reducer) {
4250     if (transformer == null || reducer == null)
4251     throw new NullPointerException();
4252     return new MapReduceMappingsTask<K,V,U>
4253 dl 1.130 (map, null, -1, null, transformer, reducer);
4254 dl 1.119 }
4255    
4256     /**
4257     * Returns a task that when invoked, returns the result of
4258     * accumulating the given transformation of all (key, value) pairs
4259     * using the given reducer to combine values, and the given
4260     * basis as an identity value.
4261     *
4262     * @param map the map
4263     * @param transformer a function returning the transformation
4264     * for an element
4265     * @param basis the identity (initial default value) for the reduction
4266     * @param reducer a commutative associative combining function
4267     * @return the task
4268     */
4269     public static <K,V> ForkJoinTask<Double> reduceToDouble
4270     (ConcurrentHashMap<K,V> map,
4271     ObjectByObjectToDouble<? super K, ? super V> transformer,
4272     double basis,
4273     DoubleByDoubleToDouble reducer) {
4274     if (transformer == null || reducer == null)
4275     throw new NullPointerException();
4276     return new MapReduceMappingsToDoubleTask<K,V>
4277 dl 1.130 (map, null, -1, null, transformer, basis, reducer);
4278 dl 1.119 }
4279    
4280     /**
4281     * Returns a task that when invoked, returns the result of
4282     * accumulating the given transformation of all (key, value) pairs
4283     * using the given reducer to combine values, and the given
4284     * basis as an identity value.
4285     *
4286     * @param map the map
4287     * @param transformer a function returning the transformation
4288     * for an element
4289     * @param basis the identity (initial default value) for the reduction
4290     * @param reducer a commutative associative combining function
4291     * @return the task
4292     */
4293     public static <K,V> ForkJoinTask<Long> reduceToLong
4294     (ConcurrentHashMap<K,V> map,
4295     ObjectByObjectToLong<? super K, ? super V> transformer,
4296     long basis,
4297     LongByLongToLong reducer) {
4298     if (transformer == null || reducer == null)
4299     throw new NullPointerException();
4300     return new MapReduceMappingsToLongTask<K,V>
4301 dl 1.130 (map, null, -1, null, transformer, basis, reducer);
4302 dl 1.119 }
4303    
4304     /**
4305     * Returns a task that when invoked, returns the result of
4306     * accumulating the given transformation of all (key, value) pairs
4307     * using the given reducer to combine values, and the given
4308     * basis as an identity value.
4309     *
4310     * @param transformer a function returning the transformation
4311     * for an element
4312     * @param basis the identity (initial default value) for the reduction
4313     * @param reducer a commutative associative combining function
4314     * @return the task
4315     */
4316     public static <K,V> ForkJoinTask<Integer> reduceToInt
4317     (ConcurrentHashMap<K,V> map,
4318     ObjectByObjectToInt<? super K, ? super V> transformer,
4319     int basis,
4320     IntByIntToInt reducer) {
4321     if (transformer == null || reducer == null)
4322     throw new NullPointerException();
4323     return new MapReduceMappingsToIntTask<K,V>
4324 dl 1.130 (map, null, -1, null, transformer, basis, reducer);
4325 dl 1.119 }
4326    
4327     /**
4328     * Returns a task that when invoked, performs the given action
4329 jsr166 1.123 * for each key.
4330 dl 1.119 *
4331     * @param map the map
4332     * @param action the action
4333     * @return the task
4334     */
4335 jsr166 1.120 public static <K,V> ForkJoinTask<Void> forEachKey
4336 dl 1.119 (ConcurrentHashMap<K,V> map,
4337     Action<K> action) {
4338     if (action == null) throw new NullPointerException();
4339 dl 1.130 return new ForEachKeyTask<K,V>(map, null, -1, action);
4340 dl 1.119 }
4341    
4342     /**
4343     * Returns a task that when invoked, performs the given action
4344 jsr166 1.123 * for each non-null transformation of each key.
4345 dl 1.119 *
4346     * @param map the map
4347     * @param transformer a function returning the transformation
4348     * for an element, or null of there is no transformation (in
4349     * which case the action is not applied).
4350     * @param action the action
4351     * @return the task
4352     */
4353 jsr166 1.120 public static <K,V,U> ForkJoinTask<Void> forEachKey
4354 dl 1.119 (ConcurrentHashMap<K,V> map,
4355     Fun<? super K, ? extends U> transformer,
4356     Action<U> action) {
4357     if (transformer == null || action == null)
4358     throw new NullPointerException();
4359     return new ForEachTransformedKeyTask<K,V,U>
4360 dl 1.130 (map, null, -1, transformer, action);
4361 dl 1.119 }
4362    
4363     /**
4364     * Returns a task that when invoked, returns a non-null result
4365     * from applying the given search function on each key, or
4366 dl 1.126 * null if none. Upon success, further element processing is
4367     * suppressed and the results of any other parallel
4368     * invocations of the search function are ignored.
4369 dl 1.119 *
4370     * @param map the map
4371     * @param searchFunction a function returning a non-null
4372     * result on success, else null
4373     * @return the task
4374     */
4375     public static <K,V,U> ForkJoinTask<U> searchKeys
4376     (ConcurrentHashMap<K,V> map,
4377     Fun<? super K, ? extends U> searchFunction) {
4378     if (searchFunction == null) throw new NullPointerException();
4379     return new SearchKeysTask<K,V,U>
4380 dl 1.130 (map, null, -1, searchFunction,
4381 dl 1.119 new AtomicReference<U>());
4382     }
4383    
4384     /**
4385     * Returns a task that when invoked, returns the result of
4386     * accumulating all keys using the given reducer to combine
4387     * values, or null if none.
4388     *
4389     * @param map the map
4390     * @param reducer a commutative associative combining function
4391     * @return the task
4392     */
4393     public static <K,V> ForkJoinTask<K> reduceKeys
4394     (ConcurrentHashMap<K,V> map,
4395     BiFun<? super K, ? super K, ? extends K> reducer) {
4396     if (reducer == null) throw new NullPointerException();
4397     return new ReduceKeysTask<K,V>
4398 dl 1.130 (map, null, -1, null, reducer);
4399 dl 1.119 }
4400 jsr166 1.125
4401 dl 1.119 /**
4402     * Returns a task that when invoked, returns the result of
4403     * accumulating the given transformation of all keys using the given
4404     * reducer to combine values, or null if none.
4405     *
4406     * @param map the map
4407     * @param transformer a function returning the transformation
4408     * for an element, or null of there is no transformation (in
4409     * which case it is not combined).
4410     * @param reducer a commutative associative combining function
4411     * @return the task
4412     */
4413     public static <K,V,U> ForkJoinTask<U> reduceKeys
4414     (ConcurrentHashMap<K,V> map,
4415     Fun<? super K, ? extends U> transformer,
4416     BiFun<? super U, ? super U, ? extends U> reducer) {
4417     if (transformer == null || reducer == null)
4418     throw new NullPointerException();
4419     return new MapReduceKeysTask<K,V,U>
4420 dl 1.130 (map, null, -1, null, transformer, reducer);
4421 dl 1.119 }
4422    
4423     /**
4424     * Returns a task that when invoked, returns the result of
4425     * accumulating the given transformation of all keys using the given
4426     * reducer to combine values, and the given basis as an
4427     * identity value.
4428     *
4429     * @param map the map
4430     * @param transformer a function returning the transformation
4431     * for an element
4432     * @param basis the identity (initial default value) for the reduction
4433     * @param reducer a commutative associative combining function
4434     * @return the task
4435     */
4436     public static <K,V> ForkJoinTask<Double> reduceKeysToDouble
4437     (ConcurrentHashMap<K,V> map,
4438     ObjectToDouble<? super K> transformer,
4439     double basis,
4440     DoubleByDoubleToDouble reducer) {
4441     if (transformer == null || reducer == null)
4442     throw new NullPointerException();
4443     return new MapReduceKeysToDoubleTask<K,V>
4444 dl 1.130 (map, null, -1, null, transformer, basis, reducer);
4445 dl 1.119 }
4446    
4447     /**
4448     * Returns a task that when invoked, returns the result of
4449     * accumulating the given transformation of all keys using the given
4450     * reducer to combine values, and the given basis as an
4451     * identity value.
4452     *
4453     * @param map the map
4454     * @param transformer a function returning the transformation
4455     * for an element
4456     * @param basis the identity (initial default value) for the reduction
4457     * @param reducer a commutative associative combining function
4458     * @return the task
4459     */
4460     public static <K,V> ForkJoinTask<Long> reduceKeysToLong
4461     (ConcurrentHashMap<K,V> map,
4462     ObjectToLong<? super K> transformer,
4463     long basis,
4464     LongByLongToLong reducer) {
4465     if (transformer == null || reducer == null)
4466     throw new NullPointerException();
4467     return new MapReduceKeysToLongTask<K,V>
4468 dl 1.130 (map, null, -1, null, transformer, basis, reducer);
4469 dl 1.119 }
4470    
4471     /**
4472     * Returns a task that when invoked, returns the result of
4473     * accumulating the given transformation of all keys using the given
4474     * reducer to combine values, and the given basis as an
4475     * identity value.
4476     *
4477     * @param map the map
4478     * @param transformer a function returning the transformation
4479     * for an element
4480     * @param basis the identity (initial default value) for the reduction
4481     * @param reducer a commutative associative combining function
4482     * @return the task
4483     */
4484     public static <K,V> ForkJoinTask<Integer> reduceKeysToInt
4485     (ConcurrentHashMap<K,V> map,
4486     ObjectToInt<? super K> transformer,
4487     int basis,
4488     IntByIntToInt reducer) {
4489     if (transformer == null || reducer == null)
4490     throw new NullPointerException();
4491     return new MapReduceKeysToIntTask<K,V>
4492 dl 1.130 (map, null, -1, null, transformer, basis, reducer);
4493 dl 1.119 }
4494    
4495     /**
4496     * Returns a task that when invoked, performs the given action
4497 jsr166 1.123 * for each value.
4498 dl 1.119 *
4499     * @param map the map
4500     * @param action the action
4501     */
4502 jsr166 1.120 public static <K,V> ForkJoinTask<Void> forEachValue
4503 dl 1.119 (ConcurrentHashMap<K,V> map,
4504     Action<V> action) {
4505     if (action == null) throw new NullPointerException();
4506 dl 1.130 return new ForEachValueTask<K,V>(map, null, -1, action);
4507 dl 1.119 }
4508    
4509     /**
4510     * Returns a task that when invoked, performs the given action
4511 jsr166 1.123 * for each non-null transformation of each value.
4512 dl 1.119 *
4513     * @param map the map
4514     * @param transformer a function returning the transformation
4515     * for an element, or null of there is no transformation (in
4516     * which case the action is not applied).
4517     * @param action the action
4518     */
4519 jsr166 1.120 public static <K,V,U> ForkJoinTask<Void> forEachValue
4520 dl 1.119 (ConcurrentHashMap<K,V> map,
4521     Fun<? super V, ? extends U> transformer,
4522     Action<U> action) {
4523     if (transformer == null || action == null)
4524     throw new NullPointerException();
4525     return new ForEachTransformedValueTask<K,V,U>
4526 dl 1.130 (map, null, -1, transformer, action);
4527 dl 1.119 }
4528    
4529     /**
4530     * Returns a task that when invoked, returns a non-null result
4531     * from applying the given search function on each value, or
4532 dl 1.126 * null if none. Upon success, further element processing is
4533     * suppressed and the results of any other parallel
4534     * invocations of the search function are ignored.
4535 dl 1.119 *
4536     * @param map the map
4537     * @param searchFunction a function returning a non-null
4538     * result on success, else null
4539     * @return the task
4540     *
4541     */
4542     public static <K,V,U> ForkJoinTask<U> searchValues
4543     (ConcurrentHashMap<K,V> map,
4544     Fun<? super V, ? extends U> searchFunction) {
4545     if (searchFunction == null) throw new NullPointerException();
4546     return new SearchValuesTask<K,V,U>
4547 dl 1.130 (map, null, -1, searchFunction,
4548 dl 1.119 new AtomicReference<U>());
4549     }
4550    
4551     /**
4552     * Returns a task that when invoked, returns the result of
4553     * accumulating all values using the given reducer to combine
4554     * values, or null if none.
4555     *
4556     * @param map the map
4557     * @param reducer a commutative associative combining function
4558     * @return the task
4559     */
4560     public static <K,V> ForkJoinTask<V> reduceValues
4561     (ConcurrentHashMap<K,V> map,
4562     BiFun<? super V, ? super V, ? extends V> reducer) {
4563     if (reducer == null) throw new NullPointerException();
4564     return new ReduceValuesTask<K,V>
4565 dl 1.130 (map, null, -1, null, reducer);
4566 dl 1.119 }
4567    
4568     /**
4569     * Returns a task that when invoked, returns the result of
4570     * accumulating the given transformation of all values using the
4571     * given reducer to combine values, or null if none.
4572     *
4573     * @param map the map
4574     * @param transformer a function returning the transformation
4575     * for an element, or null of there is no transformation (in
4576     * which case it is not combined).
4577     * @param reducer a commutative associative combining function
4578     * @return the task
4579     */
4580     public static <K,V,U> ForkJoinTask<U> reduceValues
4581     (ConcurrentHashMap<K,V> map,
4582     Fun<? super V, ? extends U> transformer,
4583     BiFun<? super U, ? super U, ? extends U> reducer) {
4584     if (transformer == null || reducer == null)
4585     throw new NullPointerException();
4586     return new MapReduceValuesTask<K,V,U>
4587 dl 1.130 (map, null, -1, null, transformer, reducer);
4588 dl 1.119 }
4589    
4590     /**
4591     * Returns a task that when invoked, returns the result of
4592     * accumulating the given transformation of all values using the
4593     * given reducer to combine values, and the given basis as an
4594     * identity value.
4595     *
4596     * @param map the map
4597     * @param transformer a function returning the transformation
4598     * for an element
4599     * @param basis the identity (initial default value) for the reduction
4600     * @param reducer a commutative associative combining function
4601     * @return the task
4602     */
4603     public static <K,V> ForkJoinTask<Double> reduceValuesToDouble
4604     (ConcurrentHashMap<K,V> map,
4605     ObjectToDouble<? super V> transformer,
4606     double basis,
4607     DoubleByDoubleToDouble reducer) {
4608     if (transformer == null || reducer == null)
4609     throw new NullPointerException();
4610     return new MapReduceValuesToDoubleTask<K,V>
4611 dl 1.130 (map, null, -1, null, transformer, basis, reducer);
4612 dl 1.119 }
4613    
4614     /**
4615     * Returns a task that when invoked, returns the result of
4616     * accumulating the given transformation of all values using the
4617     * given reducer to combine values, and the given basis as an
4618     * identity value.
4619     *
4620     * @param map the map
4621     * @param transformer a function returning the transformation
4622     * for an element
4623     * @param basis the identity (initial default value) for the reduction
4624     * @param reducer a commutative associative combining function
4625     * @return the task
4626     */
4627     public static <K,V> ForkJoinTask<Long> reduceValuesToLong
4628     (ConcurrentHashMap<K,V> map,
4629     ObjectToLong<? super V> transformer,
4630     long basis,
4631     LongByLongToLong reducer) {
4632     if (transformer == null || reducer == null)
4633     throw new NullPointerException();
4634     return new MapReduceValuesToLongTask<K,V>
4635 dl 1.130 (map, null, -1, null, transformer, basis, reducer);
4636 dl 1.119 }
4637    
4638     /**
4639     * Returns a task that when invoked, returns the result of
4640     * accumulating the given transformation of all values using the
4641     * given reducer to combine values, and the given basis as an
4642     * identity value.
4643     *
4644     * @param map the map
4645     * @param transformer a function returning the transformation
4646     * for an element
4647     * @param basis the identity (initial default value) for the reduction
4648     * @param reducer a commutative associative combining function
4649     * @return the task
4650     */
4651     public static <K,V> ForkJoinTask<Integer> reduceValuesToInt
4652     (ConcurrentHashMap<K,V> map,
4653     ObjectToInt<? super V> transformer,
4654     int basis,
4655     IntByIntToInt reducer) {
4656     if (transformer == null || reducer == null)
4657     throw new NullPointerException();
4658     return new MapReduceValuesToIntTask<K,V>
4659 dl 1.130 (map, null, -1, null, transformer, basis, reducer);
4660 dl 1.119 }
4661    
4662     /**
4663     * Returns a task that when invoked, perform the given action
4664 jsr166 1.123 * for each entry.
4665 dl 1.119 *
4666     * @param map the map
4667     * @param action the action
4668     */
4669 jsr166 1.120 public static <K,V> ForkJoinTask<Void> forEachEntry
4670 dl 1.119 (ConcurrentHashMap<K,V> map,
4671     Action<Map.Entry<K,V>> action) {
4672     if (action == null) throw new NullPointerException();
4673 dl 1.130 return new ForEachEntryTask<K,V>(map, null, -1, action);
4674 dl 1.119 }
4675    
4676     /**
4677     * Returns a task that when invoked, perform the given action
4678 jsr166 1.123 * for each non-null transformation of each entry.
4679 dl 1.119 *
4680     * @param map the map
4681     * @param transformer a function returning the transformation
4682     * for an element, or null of there is no transformation (in
4683     * which case the action is not applied).
4684     * @param action the action
4685     */
4686 jsr166 1.120 public static <K,V,U> ForkJoinTask<Void> forEachEntry
4687 dl 1.119 (ConcurrentHashMap<K,V> map,
4688     Fun<Map.Entry<K,V>, ? extends U> transformer,
4689     Action<U> action) {
4690     if (transformer == null || action == null)
4691     throw new NullPointerException();
4692     return new ForEachTransformedEntryTask<K,V,U>
4693 dl 1.130 (map, null, -1, transformer, action);
4694 dl 1.119 }
4695    
4696     /**
4697     * Returns a task that when invoked, returns a non-null result
4698     * from applying the given search function on each entry, or
4699 dl 1.126 * null if none. Upon success, further element processing is
4700     * suppressed and the results of any other parallel
4701     * invocations of the search function are ignored.
4702 dl 1.119 *
4703     * @param map the map
4704     * @param searchFunction a function returning a non-null
4705     * result on success, else null
4706     * @return the task
4707     *
4708     */
4709     public static <K,V,U> ForkJoinTask<U> searchEntries
4710     (ConcurrentHashMap<K,V> map,
4711     Fun<Map.Entry<K,V>, ? extends U> searchFunction) {
4712     if (searchFunction == null) throw new NullPointerException();
4713     return new SearchEntriesTask<K,V,U>
4714 dl 1.130 (map, null, -1, searchFunction,
4715 dl 1.119 new AtomicReference<U>());
4716     }
4717    
4718     /**
4719     * Returns a task that when invoked, returns the result of
4720     * accumulating all entries using the given reducer to combine
4721     * values, or null if none.
4722     *
4723     * @param map the map
4724     * @param reducer a commutative associative combining function
4725     * @return the task
4726     */
4727     public static <K,V> ForkJoinTask<Map.Entry<K,V>> reduceEntries
4728     (ConcurrentHashMap<K,V> map,
4729     BiFun<Map.Entry<K,V>, Map.Entry<K,V>, ? extends Map.Entry<K,V>> reducer) {
4730     if (reducer == null) throw new NullPointerException();
4731     return new ReduceEntriesTask<K,V>
4732 dl 1.130 (map, null, -1, null, reducer);
4733 dl 1.119 }
4734    
4735     /**
4736     * Returns a task that when invoked, returns the result of
4737     * accumulating the given transformation of all entries using the
4738     * given reducer to combine values, or null if none.
4739     *
4740     * @param map the map
4741     * @param transformer a function returning the transformation
4742     * for an element, or null of there is no transformation (in
4743     * which case it is not combined).
4744     * @param reducer a commutative associative combining function
4745     * @return the task
4746     */
4747     public static <K,V,U> ForkJoinTask<U> reduceEntries
4748     (ConcurrentHashMap<K,V> map,
4749     Fun<Map.Entry<K,V>, ? extends U> transformer,
4750     BiFun<? super U, ? super U, ? extends U> reducer) {
4751     if (transformer == null || reducer == null)
4752     throw new NullPointerException();
4753     return new MapReduceEntriesTask<K,V,U>
4754 dl 1.130 (map, null, -1, null, transformer, reducer);
4755 dl 1.119 }
4756    
4757     /**
4758     * Returns a task that when invoked, returns the result of
4759     * accumulating the given transformation of all entries using the
4760     * given reducer to combine values, and the given basis as an
4761     * identity value.
4762     *
4763     * @param map the map
4764     * @param transformer a function returning the transformation
4765     * for an element
4766     * @param basis the identity (initial default value) for the reduction
4767     * @param reducer a commutative associative combining function
4768     * @return the task
4769     */
4770     public static <K,V> ForkJoinTask<Double> reduceEntriesToDouble
4771     (ConcurrentHashMap<K,V> map,
4772     ObjectToDouble<Map.Entry<K,V>> transformer,
4773     double basis,
4774     DoubleByDoubleToDouble reducer) {
4775     if (transformer == null || reducer == null)
4776     throw new NullPointerException();
4777     return new MapReduceEntriesToDoubleTask<K,V>
4778 dl 1.130 (map, null, -1, null, transformer, basis, reducer);
4779 dl 1.119 }
4780    
4781     /**
4782     * Returns a task that when invoked, returns the result of
4783     * accumulating the given transformation of all entries using the
4784     * given reducer to combine values, and the given basis as an
4785     * identity value.
4786     *
4787     * @param map the map
4788     * @param transformer a function returning the transformation
4789     * for an element
4790     * @param basis the identity (initial default value) for the reduction
4791     * @param reducer a commutative associative combining function
4792     * @return the task
4793     */
4794     public static <K,V> ForkJoinTask<Long> reduceEntriesToLong
4795     (ConcurrentHashMap<K,V> map,
4796     ObjectToLong<Map.Entry<K,V>> transformer,
4797     long basis,
4798     LongByLongToLong reducer) {
4799     if (transformer == null || reducer == null)
4800     throw new NullPointerException();
4801     return new MapReduceEntriesToLongTask<K,V>
4802 dl 1.130 (map, null, -1, null, transformer, basis, reducer);
4803 dl 1.119 }
4804    
4805     /**
4806     * Returns a task that when invoked, returns the result of
4807     * accumulating the given transformation of all entries using the
4808     * given reducer to combine values, and the given basis as an
4809     * identity value.
4810     *
4811     * @param map the map
4812     * @param transformer a function returning the transformation
4813     * for an element
4814     * @param basis the identity (initial default value) for the reduction
4815     * @param reducer a commutative associative combining function
4816     * @return the task
4817     */
4818     public static <K,V> ForkJoinTask<Integer> reduceEntriesToInt
4819     (ConcurrentHashMap<K,V> map,
4820     ObjectToInt<Map.Entry<K,V>> transformer,
4821     int basis,
4822     IntByIntToInt reducer) {
4823     if (transformer == null || reducer == null)
4824     throw new NullPointerException();
4825     return new MapReduceEntriesToIntTask<K,V>
4826 dl 1.130 (map, null, -1, null, transformer, basis, reducer);
4827 dl 1.119 }
4828     }
4829    
4830     // -------------------------------------------------------
4831    
4832     /**
4833     * Base for FJ tasks for bulk operations. This adds a variant of
4834 jsr166 1.122 * CountedCompleters and some split and merge bookkeeping to
4835 dl 1.119 * iterator functionality. The forEach and reduce methods are
4836     * similar to those illustrated in CountedCompleter documentation,
4837     * except that bottom-up reduction completions perform them within
4838     * their compute methods. The search methods are like forEach
4839     * except they continually poll for success and exit early. Also,
4840     * exceptions are handled in a simpler manner, by just trying to
4841     * complete root task exceptionally.
4842     */
4843 dl 1.128 @SuppressWarnings("serial") static abstract class BulkTask<K,V,R> extends Traverser<K,V,R> {
4844 dl 1.119 final BulkTask<K,V,?> parent; // completion target
4845 dl 1.130 int batch; // split control; -1 for unknown
4846 dl 1.119 int pending; // completion control
4847    
4848 dl 1.130 BulkTask(ConcurrentHashMap<K,V> map, BulkTask<K,V,?> parent,
4849     int batch) {
4850 dl 1.119 super(map);
4851     this.parent = parent;
4852     this.batch = batch;
4853 dl 1.130 if (parent != null && map != null) { // split parent
4854     Node[] t;
4855     if ((t = parent.tab) == null &&
4856     (t = parent.tab = map.table) != null)
4857     parent.baseLimit = parent.baseSize = t.length;
4858     this.tab = t;
4859     this.baseSize = parent.baseSize;
4860     int hi = this.baseLimit = parent.baseLimit;
4861     parent.baseLimit = this.index = this.baseIndex =
4862     (hi + parent.baseIndex + 1) >>> 1;
4863     }
4864 dl 1.119 }
4865    
4866     // FJ methods
4867    
4868     /**
4869 jsr166 1.123 * Propagates completion. Note that all reduce actions
4870 dl 1.119 * bypass this method to combine while completing.
4871     */
4872     final void tryComplete() {
4873     BulkTask<K,V,?> a = this, s = a;
4874     for (int c;;) {
4875     if ((c = a.pending) == 0) {
4876     if ((a = (s = a).parent) == null) {
4877     s.quietlyComplete();
4878     break;
4879     }
4880     }
4881     else if (U.compareAndSwapInt(a, PENDING, c, c - 1))
4882     break;
4883     }
4884     }
4885    
4886     /**
4887 dl 1.128 * Forces root task to complete.
4888     * @param ex if null, complete normally, else exceptionally
4889     * @return false to simplify use
4890 dl 1.119 */
4891 dl 1.128 final boolean tryCompleteComputation(Throwable ex) {
4892 dl 1.119 for (BulkTask<K,V,?> a = this;;) {
4893     BulkTask<K,V,?> p = a.parent;
4894     if (p == null) {
4895 dl 1.128 if (ex != null)
4896     a.completeExceptionally(ex);
4897     else
4898     a.quietlyComplete();
4899     return false;
4900 dl 1.119 }
4901     a = p;
4902     }
4903     }
4904    
4905 dl 1.128 /**
4906     * Version of tryCompleteComputation for function screening checks
4907     */
4908     final boolean abortOnNullFunction() {
4909     return tryCompleteComputation(new Error("Unexpected null function"));
4910 dl 1.119 }
4911    
4912     // utilities
4913    
4914     /** CompareAndSet pending count */
4915     final boolean casPending(int cmp, int val) {
4916     return U.compareAndSwapInt(this, PENDING, cmp, val);
4917     }
4918    
4919     /**
4920 jsr166 1.123 * Returns approx exp2 of the number of times (minus one) to
4921 dl 1.119 * split task by two before executing leaf action. This value
4922     * is faster to compute and more convenient to use as a guide
4923     * to splitting than is the depth, since it is used while
4924     * dividing by two anyway.
4925     */
4926     final int batch() {
4927 dl 1.130 ConcurrentHashMap<K, V> m; int b; Node[] t;
4928     if ((b = batch) < 0 && (m = map) != null) { // force initialization
4929     if ((t = tab) == null && (t = tab = m.table) != null)
4930     baseLimit = baseSize = t.length;
4931     if (t != null) {
4932     long n = m.counter.sum();
4933     int sp = getPool().getParallelism() << 3; // slack of 8
4934     b = batch = (n <= 0L) ? 0 : (n < (long)sp) ? (int)n : sp;
4935     }
4936 dl 1.119 }
4937     return b;
4938     }
4939    
4940     /**
4941 jsr166 1.123 * Returns exportable snapshot entry.
4942 dl 1.119 */
4943     static <K,V> AbstractMap.SimpleEntry<K,V> entryFor(K k, V v) {
4944 dl 1.126 return new AbstractMap.SimpleEntry<K,V>(k, v);
4945 dl 1.119 }
4946    
4947     // Unsafe mechanics
4948     private static final sun.misc.Unsafe U;
4949     private static final long PENDING;
4950     static {
4951     try {
4952     U = sun.misc.Unsafe.getUnsafe();
4953     PENDING = U.objectFieldOffset
4954     (BulkTask.class.getDeclaredField("pending"));
4955     } catch (Exception e) {
4956     throw new Error(e);
4957     }
4958     }
4959     }
4960    
4961     /*
4962     * Task classes. Coded in a regular but ugly format/style to
4963     * simplify checks that each variant differs in the right way from
4964     * others.
4965     */
4966    
4967 dl 1.128 @SuppressWarnings("serial") static final class ForEachKeyTask<K,V>
4968 dl 1.119 extends BulkTask<K,V,Void> {
4969     final Action<K> action;
4970     ForEachKeyTask
4971 dl 1.130 (ConcurrentHashMap<K,V> m, BulkTask<K,V,?> p, int b,
4972 dl 1.119 Action<K> action) {
4973 dl 1.130 super(m, p, b);
4974 dl 1.119 this.action = action;
4975     }
4976 dl 1.128 @SuppressWarnings("unchecked") public final boolean exec() {
4977 dl 1.119 final Action<K> action = this.action;
4978     if (action == null)
4979 dl 1.128 return abortOnNullFunction();
4980     try {
4981     int b = batch(), c;
4982     while (b > 1 && baseIndex != baseLimit) {
4983     do {} while (!casPending(c = pending, c+1));
4984 dl 1.130 new ForEachKeyTask<K,V>(map, this, b >>>= 1, action).fork();
4985 dl 1.128 }
4986     while (advance() != null)
4987     action.apply((K)nextKey);
4988     tryComplete();
4989     } catch (Throwable ex) {
4990     return tryCompleteComputation(ex);
4991 dl 1.119 }
4992 dl 1.128 return false;
4993 dl 1.119 }
4994     }
4995    
4996 dl 1.128 @SuppressWarnings("serial") static final class ForEachValueTask<K,V>
4997 dl 1.119 extends BulkTask<K,V,Void> {
4998     final Action<V> action;
4999     ForEachValueTask
5000 dl 1.130 (ConcurrentHashMap<K,V> m, BulkTask<K,V,?> p, int b,
5001 dl 1.119 Action<V> action) {
5002 dl 1.130 super(m, p, b);
5003 dl 1.119 this.action = action;
5004     }
5005 dl 1.128 @SuppressWarnings("unchecked") public final boolean exec() {
5006 dl 1.119 final Action<V> action = this.action;
5007     if (action == null)
5008 dl 1.128 return abortOnNullFunction();
5009     try {
5010     int b = batch(), c;
5011     while (b > 1 && baseIndex != baseLimit) {
5012     do {} while (!casPending(c = pending, c+1));
5013 dl 1.130 new ForEachValueTask<K,V>(map, this, b >>>= 1, action).fork();
5014 dl 1.128 }
5015     Object v;
5016     while ((v = advance()) != null)
5017     action.apply((V)v);
5018     tryComplete();
5019     } catch (Throwable ex) {
5020     return tryCompleteComputation(ex);
5021 dl 1.119 }
5022 dl 1.128 return false;
5023 dl 1.119 }
5024     }
5025    
5026 dl 1.128 @SuppressWarnings("serial") static final class ForEachEntryTask<K,V>
5027 dl 1.119 extends BulkTask<K,V,Void> {
5028     final Action<Entry<K,V>> action;
5029     ForEachEntryTask
5030 dl 1.130 (ConcurrentHashMap<K,V> m, BulkTask<K,V,?> p, int b,
5031 dl 1.119 Action<Entry<K,V>> action) {
5032 dl 1.130 super(m, p, b);
5033 dl 1.119 this.action = action;
5034     }
5035 dl 1.128 @SuppressWarnings("unchecked") public final boolean exec() {
5036 dl 1.119 final Action<Entry<K,V>> action = this.action;
5037     if (action == null)
5038 dl 1.128 return abortOnNullFunction();
5039     try {
5040     int b = batch(), c;
5041     while (b > 1 && baseIndex != baseLimit) {
5042     do {} while (!casPending(c = pending, c+1));
5043 dl 1.130 new ForEachEntryTask<K,V>(map, this, b >>>= 1, action).fork();
5044 dl 1.128 }
5045     Object v;
5046     while ((v = advance()) != null)
5047     action.apply(entryFor((K)nextKey, (V)v));
5048     tryComplete();
5049     } catch (Throwable ex) {
5050     return tryCompleteComputation(ex);
5051 dl 1.119 }
5052 dl 1.128 return false;
5053 dl 1.119 }
5054     }
5055    
5056 dl 1.128 @SuppressWarnings("serial") static final class ForEachMappingTask<K,V>
5057 dl 1.119 extends BulkTask<K,V,Void> {
5058     final BiAction<K,V> action;
5059     ForEachMappingTask
5060 dl 1.130 (ConcurrentHashMap<K,V> m, BulkTask<K,V,?> p, int b,
5061 dl 1.119 BiAction<K,V> action) {
5062 dl 1.130 super(m, p, b);
5063 dl 1.119 this.action = action;
5064     }
5065 dl 1.128 @SuppressWarnings("unchecked") public final boolean exec() {
5066 dl 1.119 final BiAction<K,V> action = this.action;
5067     if (action == null)
5068 dl 1.128 return abortOnNullFunction();
5069     try {
5070     int b = batch(), c;
5071     while (b > 1 && baseIndex != baseLimit) {
5072     do {} while (!casPending(c = pending, c+1));
5073 dl 1.130 new ForEachMappingTask<K,V>(map, this, b >>>= 1,
5074 dl 1.128 action).fork();
5075     }
5076     Object v;
5077     while ((v = advance()) != null)
5078     action.apply((K)nextKey, (V)v);
5079     tryComplete();
5080     } catch (Throwable ex) {
5081     return tryCompleteComputation(ex);
5082 dl 1.119 }
5083 dl 1.128 return false;
5084 dl 1.119 }
5085     }
5086    
5087 dl 1.128 @SuppressWarnings("serial") static final class ForEachTransformedKeyTask<K,V,U>
5088 dl 1.119 extends BulkTask<K,V,Void> {
5089     final Fun<? super K, ? extends U> transformer;
5090     final Action<U> action;
5091     ForEachTransformedKeyTask
5092 dl 1.130 (ConcurrentHashMap<K,V> m, BulkTask<K,V,?> p, int b,
5093 dl 1.119 Fun<? super K, ? extends U> transformer,
5094     Action<U> action) {
5095 dl 1.130 super(m, p, b);
5096 dl 1.119 this.transformer = transformer;
5097     this.action = action;
5098    
5099     }
5100 dl 1.128 @SuppressWarnings("unchecked") public final boolean exec() {
5101 dl 1.119 final Fun<? super K, ? extends U> transformer =
5102     this.transformer;
5103     final Action<U> action = this.action;
5104     if (transformer == null || action == null)
5105 dl 1.128 return abortOnNullFunction();
5106     try {
5107     int b = batch(), c;
5108     while (b > 1 && baseIndex != baseLimit) {
5109     do {} while (!casPending(c = pending, c+1));
5110     new ForEachTransformedKeyTask<K,V,U>
5111 dl 1.130 (map, this, b >>>= 1, transformer, action).fork();
5112 dl 1.128 }
5113     U u;
5114     while (advance() != null) {
5115     if ((u = transformer.apply((K)nextKey)) != null)
5116     action.apply(u);
5117     }
5118     tryComplete();
5119     } catch (Throwable ex) {
5120     return tryCompleteComputation(ex);
5121 dl 1.119 }
5122 dl 1.128 return false;
5123 dl 1.119 }
5124     }
5125    
5126 dl 1.128 @SuppressWarnings("serial") static final class ForEachTransformedValueTask<K,V,U>
5127 dl 1.119 extends BulkTask<K,V,Void> {
5128     final Fun<? super V, ? extends U> transformer;
5129     final Action<U> action;
5130     ForEachTransformedValueTask
5131 dl 1.130 (ConcurrentHashMap<K,V> m, BulkTask<K,V,?> p, int b,
5132 dl 1.119 Fun<? super V, ? extends U> transformer,
5133     Action<U> action) {
5134 dl 1.130 super(m, p, b);
5135 dl 1.119 this.transformer = transformer;
5136     this.action = action;
5137    
5138     }
5139 dl 1.128 @SuppressWarnings("unchecked") public final boolean exec() {
5140 dl 1.119 final Fun<? super V, ? extends U> transformer =
5141     this.transformer;
5142     final Action<U> action = this.action;
5143     if (transformer == null || action == null)
5144 dl 1.128 return abortOnNullFunction();
5145     try {
5146     int b = batch(), c;
5147     while (b > 1 && baseIndex != baseLimit) {
5148     do {} while (!casPending(c = pending, c+1));
5149     new ForEachTransformedValueTask<K,V,U>
5150 dl 1.130 (map, this, b >>>= 1, transformer, action).fork();
5151 dl 1.128 }
5152     Object v; U u;
5153     while ((v = advance()) != null) {
5154     if ((u = transformer.apply((V)v)) != null)
5155     action.apply(u);
5156     }
5157     tryComplete();
5158     } catch (Throwable ex) {
5159     return tryCompleteComputation(ex);
5160 dl 1.119 }
5161 dl 1.128 return false;
5162 dl 1.119 }
5163 tim 1.1 }
5164    
5165 dl 1.128 @SuppressWarnings("serial") static final class ForEachTransformedEntryTask<K,V,U>
5166 dl 1.119 extends BulkTask<K,V,Void> {
5167     final Fun<Map.Entry<K,V>, ? extends U> transformer;
5168     final Action<U> action;
5169     ForEachTransformedEntryTask
5170 dl 1.130 (ConcurrentHashMap<K,V> m, BulkTask<K,V,?> p, int b,
5171 dl 1.119 Fun<Map.Entry<K,V>, ? extends U> transformer,
5172     Action<U> action) {
5173 dl 1.130 super(m, p, b);
5174 dl 1.119 this.transformer = transformer;
5175     this.action = action;
5176    
5177     }
5178 dl 1.128 @SuppressWarnings("unchecked") public final boolean exec() {
5179 dl 1.119 final Fun<Map.Entry<K,V>, ? extends U> transformer =
5180     this.transformer;
5181     final Action<U> action = this.action;
5182     if (transformer == null || action == null)
5183 dl 1.128 return abortOnNullFunction();
5184     try {
5185     int b = batch(), c;
5186     while (b > 1 && baseIndex != baseLimit) {
5187     do {} while (!casPending(c = pending, c+1));
5188     new ForEachTransformedEntryTask<K,V,U>
5189 dl 1.130 (map, this, b >>>= 1, transformer, action).fork();
5190 dl 1.128 }
5191     Object v; U u;
5192     while ((v = advance()) != null) {
5193     if ((u = transformer.apply(entryFor((K)nextKey, (V)v))) != null)
5194     action.apply(u);
5195     }
5196     tryComplete();
5197     } catch (Throwable ex) {
5198     return tryCompleteComputation(ex);
5199 dl 1.119 }
5200 dl 1.128 return false;
5201 dl 1.119 }
5202 tim 1.1 }
5203    
5204 dl 1.128 @SuppressWarnings("serial") static final class ForEachTransformedMappingTask<K,V,U>
5205 dl 1.119 extends BulkTask<K,V,Void> {
5206     final BiFun<? super K, ? super V, ? extends U> transformer;
5207     final Action<U> action;
5208     ForEachTransformedMappingTask
5209 dl 1.130 (ConcurrentHashMap<K,V> m, BulkTask<K,V,?> p, int b,
5210 dl 1.119 BiFun<? super K, ? super V, ? extends U> transformer,
5211     Action<U> action) {
5212 dl 1.130 super(m, p, b);
5213 dl 1.119 this.transformer = transformer;
5214     this.action = action;
5215    
5216     }
5217 dl 1.128 @SuppressWarnings("unchecked") public final boolean exec() {
5218 dl 1.119 final BiFun<? super K, ? super V, ? extends U> transformer =
5219     this.transformer;
5220     final Action<U> action = this.action;
5221     if (transformer == null || action == null)
5222 dl 1.128 return abortOnNullFunction();
5223     try {
5224     int b = batch(), c;
5225     while (b > 1 && baseIndex != baseLimit) {
5226     do {} while (!casPending(c = pending, c+1));
5227     new ForEachTransformedMappingTask<K,V,U>
5228 dl 1.130 (map, this, b >>>= 1, transformer, action).fork();
5229 dl 1.128 }
5230     Object v; U u;
5231     while ((v = advance()) != null) {
5232     if ((u = transformer.apply((K)nextKey, (V)v)) != null)
5233     action.apply(u);
5234     }
5235     tryComplete();
5236     } catch (Throwable ex) {
5237     return tryCompleteComputation(ex);
5238 dl 1.119 }
5239 dl 1.128 return false;
5240 dl 1.119 }
5241 tim 1.1 }
5242    
5243 dl 1.128 @SuppressWarnings("serial") static final class SearchKeysTask<K,V,U>
5244 dl 1.119 extends BulkTask<K,V,U> {
5245     final Fun<? super K, ? extends U> searchFunction;
5246     final AtomicReference<U> result;
5247     SearchKeysTask
5248 dl 1.130 (ConcurrentHashMap<K,V> m, BulkTask<K,V,?> p, int b,
5249 dl 1.119 Fun<? super K, ? extends U> searchFunction,
5250     AtomicReference<U> result) {
5251 dl 1.130 super(m, p, b);
5252 dl 1.119 this.searchFunction = searchFunction; this.result = result;
5253     }
5254 dl 1.128 @SuppressWarnings("unchecked") public final boolean exec() {
5255 dl 1.119 AtomicReference<U> result = this.result;
5256     final Fun<? super K, ? extends U> searchFunction =
5257     this.searchFunction;
5258     if (searchFunction == null || result == null)
5259 dl 1.128 return abortOnNullFunction();
5260     try {
5261     int b = batch(), c;
5262     while (b > 1 && baseIndex != baseLimit && result.get() == null) {
5263     do {} while (!casPending(c = pending, c+1));
5264 dl 1.130 new SearchKeysTask<K,V,U>(map, this, b >>>= 1,
5265 dl 1.128 searchFunction, result).fork();
5266     }
5267     U u;
5268     while (result.get() == null && advance() != null) {
5269     if ((u = searchFunction.apply((K)nextKey)) != null) {
5270     if (result.compareAndSet(null, u))
5271     tryCompleteComputation(null);
5272     break;
5273 dl 1.126 }
5274 dl 1.119 }
5275 dl 1.128 tryComplete();
5276     } catch (Throwable ex) {
5277     return tryCompleteComputation(ex);
5278 dl 1.119 }
5279 dl 1.128 return false;
5280 dl 1.119 }
5281     public final U getRawResult() { return result.get(); }
5282 tim 1.1 }
5283    
5284 dl 1.128 @SuppressWarnings("serial") static final class SearchValuesTask<K,V,U>
5285 dl 1.119 extends BulkTask<K,V,U> {
5286     final Fun<? super V, ? extends U> searchFunction;
5287     final AtomicReference<U> result;
5288     SearchValuesTask
5289 dl 1.130 (ConcurrentHashMap<K,V> m, BulkTask<K,V,?> p, int b,
5290 dl 1.119 Fun<? super V, ? extends U> searchFunction,
5291     AtomicReference<U> result) {
5292 dl 1.130 super(m, p, b);
5293 dl 1.119 this.searchFunction = searchFunction; this.result = result;
5294     }
5295 dl 1.128 @SuppressWarnings("unchecked") public final boolean exec() {
5296 dl 1.119 AtomicReference<U> result = this.result;
5297     final Fun<? super V, ? extends U> searchFunction =
5298     this.searchFunction;
5299     if (searchFunction == null || result == null)
5300 dl 1.128 return abortOnNullFunction();
5301     try {
5302     int b = batch(), c;
5303     while (b > 1 && baseIndex != baseLimit && result.get() == null) {
5304     do {} while (!casPending(c = pending, c+1));
5305 dl 1.130 new SearchValuesTask<K,V,U>(map, this, b >>>= 1,
5306 dl 1.128 searchFunction, result).fork();
5307     }
5308     Object v; U u;
5309     while (result.get() == null && (v = advance()) != null) {
5310     if ((u = searchFunction.apply((V)v)) != null) {
5311     if (result.compareAndSet(null, u))
5312     tryCompleteComputation(null);
5313     break;
5314 dl 1.126 }
5315 dl 1.119 }
5316 dl 1.128 tryComplete();
5317     } catch (Throwable ex) {
5318     return tryCompleteComputation(ex);
5319 dl 1.119 }
5320 dl 1.128 return false;
5321 dl 1.119 }
5322     public final U getRawResult() { return result.get(); }
5323     }
5324 tim 1.11
5325 dl 1.128 @SuppressWarnings("serial") static final class SearchEntriesTask<K,V,U>
5326 dl 1.119 extends BulkTask<K,V,U> {
5327     final Fun<Entry<K,V>, ? extends U> searchFunction;
5328     final AtomicReference<U> result;
5329     SearchEntriesTask
5330 dl 1.130 (ConcurrentHashMap<K,V> m, BulkTask<K,V,?> p, int b,
5331 dl 1.119 Fun<Entry<K,V>, ? extends U> searchFunction,
5332     AtomicReference<U> result) {
5333 dl 1.130 super(m, p, b);
5334 dl 1.119 this.searchFunction = searchFunction; this.result = result;
5335     }
5336 dl 1.128 @SuppressWarnings("unchecked") public final boolean exec() {
5337 dl 1.119 AtomicReference<U> result = this.result;
5338     final Fun<Entry<K,V>, ? extends U> searchFunction =
5339     this.searchFunction;
5340     if (searchFunction == null || result == null)
5341 dl 1.128 return abortOnNullFunction();
5342     try {
5343     int b = batch(), c;
5344     while (b > 1 && baseIndex != baseLimit && result.get() == null) {
5345     do {} while (!casPending(c = pending, c+1));
5346 dl 1.130 new SearchEntriesTask<K,V,U>(map, this, b >>>= 1,
5347 dl 1.128 searchFunction, result).fork();
5348     }
5349     Object v; U u;
5350     while (result.get() == null && (v = advance()) != null) {
5351     if ((u = searchFunction.apply(entryFor((K)nextKey, (V)v))) != null) {
5352     if (result.compareAndSet(null, u))
5353     tryCompleteComputation(null);
5354     break;
5355 dl 1.126 }
5356 dl 1.119 }
5357 dl 1.128 tryComplete();
5358     } catch (Throwable ex) {
5359     return tryCompleteComputation(ex);
5360 dl 1.119 }
5361 dl 1.128 return false;
5362 dl 1.119 }
5363     public final U getRawResult() { return result.get(); }
5364     }
5365 tim 1.1
5366 dl 1.128 @SuppressWarnings("serial") static final class SearchMappingsTask<K,V,U>
5367 dl 1.119 extends BulkTask<K,V,U> {
5368     final BiFun<? super K, ? super V, ? extends U> searchFunction;
5369     final AtomicReference<U> result;
5370     SearchMappingsTask
5371 dl 1.130 (ConcurrentHashMap<K,V> m, BulkTask<K,V,?> p, int b,
5372 dl 1.119 BiFun<? super K, ? super V, ? extends U> searchFunction,
5373     AtomicReference<U> result) {
5374 dl 1.130 super(m, p, b);
5375 dl 1.119 this.searchFunction = searchFunction; this.result = result;
5376     }
5377 dl 1.128 @SuppressWarnings("unchecked") public final boolean exec() {
5378 dl 1.119 AtomicReference<U> result = this.result;
5379     final BiFun<? super K, ? super V, ? extends U> searchFunction =
5380     this.searchFunction;
5381     if (searchFunction == null || result == null)
5382 dl 1.128 return abortOnNullFunction();
5383     try {
5384     int b = batch(), c;
5385     while (b > 1 && baseIndex != baseLimit && result.get() == null) {
5386     do {} while (!casPending(c = pending, c+1));
5387 dl 1.130 new SearchMappingsTask<K,V,U>(map, this, b >>>= 1,
5388 dl 1.128 searchFunction, result).fork();
5389     }
5390     Object v; U u;
5391     while (result.get() == null && (v = advance()) != null) {
5392     if ((u = searchFunction.apply((K)nextKey, (V)v)) != null) {
5393     if (result.compareAndSet(null, u))
5394     tryCompleteComputation(null);
5395     break;
5396 dl 1.126 }
5397 dl 1.119 }
5398 dl 1.128 tryComplete();
5399     } catch (Throwable ex) {
5400     return tryCompleteComputation(ex);
5401 dl 1.119 }
5402 dl 1.128 return false;
5403 tim 1.1 }
5404 dl 1.119 public final U getRawResult() { return result.get(); }
5405     }
5406 tim 1.1
5407 dl 1.128 @SuppressWarnings("serial") static final class ReduceKeysTask<K,V>
5408 dl 1.119 extends BulkTask<K,V,K> {
5409     final BiFun<? super K, ? super K, ? extends K> reducer;
5410     K result;
5411 dl 1.128 ReduceKeysTask<K,V> rights, nextRight;
5412 dl 1.119 ReduceKeysTask
5413 dl 1.130 (ConcurrentHashMap<K,V> m, BulkTask<K,V,?> p, int b,
5414 dl 1.128 ReduceKeysTask<K,V> nextRight,
5415 dl 1.119 BiFun<? super K, ? super K, ? extends K> reducer) {
5416 dl 1.130 super(m, p, b); this.nextRight = nextRight;
5417 dl 1.119 this.reducer = reducer;
5418     }
5419 dl 1.128 @SuppressWarnings("unchecked") public final boolean exec() {
5420 dl 1.119 final BiFun<? super K, ? super K, ? extends K> reducer =
5421     this.reducer;
5422     if (reducer == null)
5423 dl 1.128 return abortOnNullFunction();
5424     try {
5425     for (int c, b = batch(); b > 1 && baseIndex != baseLimit;) {
5426     do {} while (!casPending(c = pending, c+1));
5427     (rights = new ReduceKeysTask<K,V>
5428 dl 1.130 (map, this, b >>>= 1, rights, reducer)).fork();
5429 dl 1.128 }
5430     K r = null;
5431     while (advance() != null) {
5432     K u = (K)nextKey;
5433     r = (r == null) ? u : reducer.apply(r, u);
5434 dl 1.99 }
5435 dl 1.128 result = r;
5436     for (ReduceKeysTask<K,V> t = this, s;;) {
5437     int c; BulkTask<K,V,?> par; K tr, sr;
5438     if ((c = t.pending) == 0) {
5439     for (s = t.rights; s != null; s = t.rights = s.nextRight) {
5440     if ((sr = s.result) != null)
5441     t.result = (tr = t.result) == null? sr : reducer.apply(tr, sr);
5442     }
5443     if ((par = t.parent) == null ||
5444     !(par instanceof ReduceKeysTask)) {
5445     t.quietlyComplete();
5446     break;
5447     }
5448     t = (ReduceKeysTask<K,V>)par;
5449     }
5450     else if (t.casPending(c, c - 1))
5451     break;
5452 tim 1.1 }
5453 dl 1.128 } catch (Throwable ex) {
5454     return tryCompleteComputation(ex);
5455 tim 1.1 }
5456 dl 1.128 return false;
5457 tim 1.1 }
5458 dl 1.119 public final K getRawResult() { return result; }
5459     }
5460 tim 1.1
5461 dl 1.128 @SuppressWarnings("serial") static final class ReduceValuesTask<K,V>
5462 dl 1.119 extends BulkTask<K,V,V> {
5463     final BiFun<? super V, ? super V, ? extends V> reducer;
5464     V result;
5465 dl 1.128 ReduceValuesTask<K,V> rights, nextRight;
5466 dl 1.119 ReduceValuesTask
5467 dl 1.130 (ConcurrentHashMap<K,V> m, BulkTask<K,V,?> p, int b,
5468 dl 1.128 ReduceValuesTask<K,V> nextRight,
5469 dl 1.119 BiFun<? super V, ? super V, ? extends V> reducer) {
5470 dl 1.130 super(m, p, b); this.nextRight = nextRight;
5471 dl 1.119 this.reducer = reducer;
5472     }
5473 dl 1.128 @SuppressWarnings("unchecked") public final boolean exec() {
5474 dl 1.119 final BiFun<? super V, ? super V, ? extends V> reducer =
5475     this.reducer;
5476     if (reducer == null)
5477 dl 1.128 return abortOnNullFunction();
5478     try {
5479     for (int c, b = batch(); b > 1 && baseIndex != baseLimit;) {
5480     do {} while (!casPending(c = pending, c+1));
5481     (rights = new ReduceValuesTask<K,V>
5482 dl 1.130 (map, this, b >>>= 1, rights, reducer)).fork();
5483 dl 1.128 }
5484     V r = null;
5485     Object v;
5486     while ((v = advance()) != null) {
5487     V u = (V)v;
5488     r = (r == null) ? u : reducer.apply(r, u);
5489 dl 1.119 }
5490 dl 1.128 result = r;
5491     for (ReduceValuesTask<K,V> t = this, s;;) {
5492     int c; BulkTask<K,V,?> par; V tr, sr;
5493     if ((c = t.pending) == 0) {
5494     for (s = t.rights; s != null; s = t.rights = s.nextRight) {
5495     if ((sr = s.result) != null)
5496     t.result = (tr = t.result) == null? sr : reducer.apply(tr, sr);
5497     }
5498     if ((par = t.parent) == null ||
5499     !(par instanceof ReduceValuesTask)) {
5500     t.quietlyComplete();
5501     break;
5502     }
5503     t = (ReduceValuesTask<K,V>)par;
5504     }
5505     else if (t.casPending(c, c - 1))
5506     break;
5507 dl 1.119 }
5508 dl 1.128 } catch (Throwable ex) {
5509     return tryCompleteComputation(ex);
5510 dl 1.119 }
5511 dl 1.128 return false;
5512 tim 1.1 }
5513 dl 1.119 public final V getRawResult() { return result; }
5514     }
5515 tim 1.1
5516 dl 1.128 @SuppressWarnings("serial") static final class ReduceEntriesTask<K,V>
5517 dl 1.119 extends BulkTask<K,V,Map.Entry<K,V>> {
5518     final BiFun<Map.Entry<K,V>, Map.Entry<K,V>, ? extends Map.Entry<K,V>> reducer;
5519     Map.Entry<K,V> result;
5520 dl 1.128 ReduceEntriesTask<K,V> rights, nextRight;
5521 dl 1.119 ReduceEntriesTask
5522 dl 1.130 (ConcurrentHashMap<K,V> m, BulkTask<K,V,?> p, int b,
5523     ReduceEntriesTask<K,V> nextRight,
5524 dl 1.119 BiFun<Entry<K,V>, Map.Entry<K,V>, ? extends Map.Entry<K,V>> reducer) {
5525 dl 1.130 super(m, p, b); this.nextRight = nextRight;
5526 dl 1.119 this.reducer = reducer;
5527     }
5528 dl 1.128 @SuppressWarnings("unchecked") public final boolean exec() {
5529 dl 1.119 final BiFun<Map.Entry<K,V>, Map.Entry<K,V>, ? extends Map.Entry<K,V>> reducer =
5530     this.reducer;
5531     if (reducer == null)
5532 dl 1.128 return abortOnNullFunction();
5533     try {
5534     for (int c, b = batch(); b > 1 && baseIndex != baseLimit;) {
5535     do {} while (!casPending(c = pending, c+1));
5536     (rights = new ReduceEntriesTask<K,V>
5537 dl 1.130 (map, this, b >>>= 1, rights, reducer)).fork();
5538 dl 1.128 }
5539     Map.Entry<K,V> r = null;
5540     Object v;
5541     while ((v = advance()) != null) {
5542     Map.Entry<K,V> u = entryFor((K)nextKey, (V)v);
5543     r = (r == null) ? u : reducer.apply(r, u);
5544 dl 1.119 }
5545 dl 1.128 result = r;
5546     for (ReduceEntriesTask<K,V> t = this, s;;) {
5547     int c; BulkTask<K,V,?> par; Map.Entry<K,V> tr, sr;
5548     if ((c = t.pending) == 0) {
5549     for (s = t.rights; s != null; s = t.rights = s.nextRight) {
5550     if ((sr = s.result) != null)
5551     t.result = (tr = t.result) == null? sr : reducer.apply(tr, sr);
5552     }
5553     if ((par = t.parent) == null ||
5554     !(par instanceof ReduceEntriesTask)) {
5555     t.quietlyComplete();
5556     break;
5557     }
5558     t = (ReduceEntriesTask<K,V>)par;
5559     }
5560     else if (t.casPending(c, c - 1))
5561     break;
5562 dl 1.119 }
5563 dl 1.128 } catch (Throwable ex) {
5564     return tryCompleteComputation(ex);
5565 dl 1.119 }
5566 dl 1.128 return false;
5567 dl 1.119 }
5568     public final Map.Entry<K,V> getRawResult() { return result; }
5569     }
5570 dl 1.99
5571 dl 1.128 @SuppressWarnings("serial") static final class MapReduceKeysTask<K,V,U>
5572 dl 1.119 extends BulkTask<K,V,U> {
5573     final Fun<? super K, ? extends U> transformer;
5574     final BiFun<? super U, ? super U, ? extends U> reducer;
5575     U result;
5576 dl 1.128 MapReduceKeysTask<K,V,U> rights, nextRight;
5577 dl 1.119 MapReduceKeysTask
5578 dl 1.130 (ConcurrentHashMap<K,V> m, BulkTask<K,V,?> p, int b,
5579 dl 1.128 MapReduceKeysTask<K,V,U> nextRight,
5580 dl 1.119 Fun<? super K, ? extends U> transformer,
5581     BiFun<? super U, ? super U, ? extends U> reducer) {
5582 dl 1.130 super(m, p, b); this.nextRight = nextRight;
5583 dl 1.119 this.transformer = transformer;
5584     this.reducer = reducer;
5585     }
5586 dl 1.128 @SuppressWarnings("unchecked") public final boolean exec() {
5587 dl 1.119 final Fun<? super K, ? extends U> transformer =
5588     this.transformer;
5589     final BiFun<? super U, ? super U, ? extends U> reducer =
5590     this.reducer;
5591     if (transformer == null || reducer == null)
5592 dl 1.128 return abortOnNullFunction();
5593     try {
5594     for (int c, b = batch(); b > 1 && baseIndex != baseLimit;) {
5595     do {} while (!casPending(c = pending, c+1));
5596     (rights = new MapReduceKeysTask<K,V,U>
5597 dl 1.130 (map, this, b >>>= 1, rights, transformer, reducer)).fork();
5598 dl 1.128 }
5599     U r = null, u;
5600     while (advance() != null) {
5601     if ((u = transformer.apply((K)nextKey)) != null)
5602     r = (r == null) ? u : reducer.apply(r, u);
5603 dl 1.119 }
5604 dl 1.128 result = r;
5605     for (MapReduceKeysTask<K,V,U> t = this, s;;) {
5606     int c; BulkTask<K,V,?> par; U tr, sr;
5607     if ((c = t.pending) == 0) {
5608     for (s = t.rights; s != null; s = t.rights = s.nextRight) {
5609     if ((sr = s.result) != null)
5610     t.result = (tr = t.result) == null? sr : reducer.apply(tr, sr);
5611     }
5612     if ((par = t.parent) == null ||
5613     !(par instanceof MapReduceKeysTask)) {
5614     t.quietlyComplete();
5615     break;
5616     }
5617     t = (MapReduceKeysTask<K,V,U>)par;
5618     }
5619     else if (t.casPending(c, c - 1))
5620     break;
5621 dl 1.119 }
5622 dl 1.128 } catch (Throwable ex) {
5623     return tryCompleteComputation(ex);
5624 dl 1.119 }
5625 dl 1.128 return false;
5626 tim 1.1 }
5627 dl 1.119 public final U getRawResult() { return result; }
5628 dl 1.4 }
5629    
5630 dl 1.128 @SuppressWarnings("serial") static final class MapReduceValuesTask<K,V,U>
5631 dl 1.119 extends BulkTask<K,V,U> {
5632     final Fun<? super V, ? extends U> transformer;
5633     final BiFun<? super U, ? super U, ? extends U> reducer;
5634     U result;
5635 dl 1.128 MapReduceValuesTask<K,V,U> rights, nextRight;
5636 dl 1.119 MapReduceValuesTask
5637 dl 1.130 (ConcurrentHashMap<K,V> m, BulkTask<K,V,?> p, int b,
5638 dl 1.128 MapReduceValuesTask<K,V,U> nextRight,
5639 dl 1.119 Fun<? super V, ? extends U> transformer,
5640     BiFun<? super U, ? super U, ? extends U> reducer) {
5641 dl 1.130 super(m, p, b); this.nextRight = nextRight;
5642 dl 1.119 this.transformer = transformer;
5643     this.reducer = reducer;
5644     }
5645 dl 1.128 @SuppressWarnings("unchecked") public final boolean exec() {
5646 dl 1.119 final Fun<? super V, ? extends U> transformer =
5647     this.transformer;
5648     final BiFun<? super U, ? super U, ? extends U> reducer =
5649     this.reducer;
5650     if (transformer == null || reducer == null)
5651 dl 1.128 return abortOnNullFunction();
5652     try {
5653     for (int c, b = batch(); b > 1 && baseIndex != baseLimit;) {
5654     do {} while (!casPending(c = pending, c+1));
5655     (rights = new MapReduceValuesTask<K,V,U>
5656 dl 1.130 (map, this, b >>>= 1, rights, transformer, reducer)).fork();
5657 dl 1.128 }
5658     U r = null, u;
5659     Object v;
5660     while ((v = advance()) != null) {
5661     if ((u = transformer.apply((V)v)) != null)
5662     r = (r == null) ? u : reducer.apply(r, u);
5663 dl 1.119 }
5664 dl 1.128 result = r;
5665     for (MapReduceValuesTask<K,V,U> t = this, s;;) {
5666     int c; BulkTask<K,V,?> par; U tr, sr;
5667     if ((c = t.pending) == 0) {
5668     for (s = t.rights; s != null; s = t.rights = s.nextRight) {
5669     if ((sr = s.result) != null)
5670     t.result = (tr = t.result) == null? sr : reducer.apply(tr, sr);
5671     }
5672     if ((par = t.parent) == null ||
5673     !(par instanceof MapReduceValuesTask)) {
5674     t.quietlyComplete();
5675     break;
5676     }
5677     t = (MapReduceValuesTask<K,V,U>)par;
5678     }
5679     else if (t.casPending(c, c - 1))
5680     break;
5681 dl 1.119 }
5682 dl 1.128 } catch (Throwable ex) {
5683     return tryCompleteComputation(ex);
5684 dl 1.119 }
5685 dl 1.128 return false;
5686 dl 1.119 }
5687     public final U getRawResult() { return result; }
5688 dl 1.4 }
5689    
5690 dl 1.128 @SuppressWarnings("serial") static final class MapReduceEntriesTask<K,V,U>
5691 dl 1.119 extends BulkTask<K,V,U> {
5692     final Fun<Map.Entry<K,V>, ? extends U> transformer;
5693     final BiFun<? super U, ? super U, ? extends U> reducer;
5694     U result;
5695 dl 1.128 MapReduceEntriesTask<K,V,U> rights, nextRight;
5696 dl 1.119 MapReduceEntriesTask
5697 dl 1.130 (ConcurrentHashMap<K,V> m, BulkTask<K,V,?> p, int b,
5698 dl 1.128 MapReduceEntriesTask<K,V,U> nextRight,
5699 dl 1.119 Fun<Map.Entry<K,V>, ? extends U> transformer,
5700     BiFun<? super U, ? super U, ? extends U> reducer) {
5701 dl 1.130 super(m, p, b); this.nextRight = nextRight;
5702 dl 1.119 this.transformer = transformer;
5703     this.reducer = reducer;
5704     }
5705 dl 1.128 @SuppressWarnings("unchecked") public final boolean exec() {
5706 dl 1.119 final Fun<Map.Entry<K,V>, ? extends U> transformer =
5707     this.transformer;
5708     final BiFun<? super U, ? super U, ? extends U> reducer =
5709     this.reducer;
5710     if (transformer == null || reducer == null)
5711 dl 1.128 return abortOnNullFunction();
5712     try {
5713     for (int c, b = batch(); b > 1 && baseIndex != baseLimit;) {
5714     do {} while (!casPending(c = pending, c+1));
5715     (rights = new MapReduceEntriesTask<K,V,U>
5716 dl 1.130 (map, this, b >>>= 1, rights, transformer, reducer)).fork();
5717 dl 1.128 }
5718     U r = null, u;
5719     Object v;
5720     while ((v = advance()) != null) {
5721     if ((u = transformer.apply(entryFor((K)nextKey, (V)v))) != null)
5722     r = (r == null) ? u : reducer.apply(r, u);
5723 dl 1.119 }
5724 dl 1.128 result = r;
5725     for (MapReduceEntriesTask<K,V,U> t = this, s;;) {
5726     int c; BulkTask<K,V,?> par; U tr, sr;
5727     if ((c = t.pending) == 0) {
5728     for (s = t.rights; s != null; s = t.rights = s.nextRight) {
5729     if ((sr = s.result) != null)
5730     t.result = (tr = t.result) == null? sr : reducer.apply(tr, sr);
5731     }
5732     if ((par = t.parent) == null ||
5733     !(par instanceof MapReduceEntriesTask)) {
5734     t.quietlyComplete();
5735     break;
5736     }
5737     t = (MapReduceEntriesTask<K,V,U>)par;
5738     }
5739     else if (t.casPending(c, c - 1))
5740     break;
5741 dl 1.119 }
5742 dl 1.128 } catch (Throwable ex) {
5743     return tryCompleteComputation(ex);
5744 dl 1.119 }
5745 dl 1.128 return false;
5746 dl 1.119 }
5747     public final U getRawResult() { return result; }
5748 dl 1.4 }
5749 tim 1.1
5750 dl 1.128 @SuppressWarnings("serial") static final class MapReduceMappingsTask<K,V,U>
5751 dl 1.119 extends BulkTask<K,V,U> {
5752     final BiFun<? super K, ? super V, ? extends U> transformer;
5753     final BiFun<? super U, ? super U, ? extends U> reducer;
5754     U result;
5755 dl 1.128 MapReduceMappingsTask<K,V,U> rights, nextRight;
5756 dl 1.119 MapReduceMappingsTask
5757 dl 1.130 (ConcurrentHashMap<K,V> m, BulkTask<K,V,?> p, int b,
5758 dl 1.128 MapReduceMappingsTask<K,V,U> nextRight,
5759 dl 1.119 BiFun<? super K, ? super V, ? extends U> transformer,
5760     BiFun<? super U, ? super U, ? extends U> reducer) {
5761 dl 1.130 super(m, p, b); this.nextRight = nextRight;
5762 dl 1.119 this.transformer = transformer;
5763     this.reducer = reducer;
5764     }
5765 dl 1.128 @SuppressWarnings("unchecked") public final boolean exec() {
5766 dl 1.119 final BiFun<? super K, ? super V, ? extends U> transformer =
5767     this.transformer;
5768     final BiFun<? super U, ? super U, ? extends U> reducer =
5769     this.reducer;
5770     if (transformer == null || reducer == null)
5771 dl 1.128 return abortOnNullFunction();
5772     try {
5773     for (int c, b = batch(); b > 1 && baseIndex != baseLimit;) {
5774     do {} while (!casPending(c = pending, c+1));
5775     (rights = new MapReduceMappingsTask<K,V,U>
5776 dl 1.130 (map, this, b >>>= 1, rights, transformer, reducer)).fork();
5777 dl 1.128 }
5778     U r = null, u;
5779     Object v;
5780     while ((v = advance()) != null) {
5781     if ((u = transformer.apply((K)nextKey, (V)v)) != null)
5782     r = (r == null) ? u : reducer.apply(r, u);
5783 dl 1.119 }
5784 dl 1.128 result = r;
5785     for (MapReduceMappingsTask<K,V,U> t = this, s;;) {
5786     int c; BulkTask<K,V,?> par; U tr, sr;
5787     if ((c = t.pending) == 0) {
5788     for (s = t.rights; s != null; s = t.rights = s.nextRight) {
5789     if ((sr = s.result) != null)
5790     t.result = (tr = t.result) == null? sr : reducer.apply(tr, sr);
5791     }
5792     if ((par = t.parent) == null ||
5793     !(par instanceof MapReduceMappingsTask)) {
5794     t.quietlyComplete();
5795     break;
5796     }
5797     t = (MapReduceMappingsTask<K,V,U>)par;
5798     }
5799     else if (t.casPending(c, c - 1))
5800     break;
5801 dl 1.119 }
5802 dl 1.128 } catch (Throwable ex) {
5803     return tryCompleteComputation(ex);
5804 dl 1.119 }
5805 dl 1.128 return false;
5806 dl 1.119 }
5807     public final U getRawResult() { return result; }
5808     }
5809 jsr166 1.114
5810 dl 1.128 @SuppressWarnings("serial") static final class MapReduceKeysToDoubleTask<K,V>
5811 dl 1.119 extends BulkTask<K,V,Double> {
5812     final ObjectToDouble<? super K> transformer;
5813     final DoubleByDoubleToDouble reducer;
5814     final double basis;
5815     double result;
5816 dl 1.128 MapReduceKeysToDoubleTask<K,V> rights, nextRight;
5817 dl 1.119 MapReduceKeysToDoubleTask
5818 dl 1.130 (ConcurrentHashMap<K,V> m, BulkTask<K,V,?> p, int b,
5819 dl 1.128 MapReduceKeysToDoubleTask<K,V> nextRight,
5820 dl 1.119 ObjectToDouble<? super K> transformer,
5821     double basis,
5822     DoubleByDoubleToDouble reducer) {
5823 dl 1.130 super(m, p, b); this.nextRight = nextRight;
5824 dl 1.119 this.transformer = transformer;
5825     this.basis = basis; this.reducer = reducer;
5826     }
5827 dl 1.128 @SuppressWarnings("unchecked") public final boolean exec() {
5828 dl 1.119 final ObjectToDouble<? super K> transformer =
5829     this.transformer;
5830     final DoubleByDoubleToDouble reducer = this.reducer;
5831     if (transformer == null || reducer == null)
5832 dl 1.128 return abortOnNullFunction();
5833     try {
5834     final double id = this.basis;
5835     for (int c, b = batch(); b > 1 && baseIndex != baseLimit;) {
5836     do {} while (!casPending(c = pending, c+1));
5837     (rights = new MapReduceKeysToDoubleTask<K,V>
5838 dl 1.130 (map, this, b >>>= 1, rights, transformer, id, reducer)).fork();
5839 dl 1.128 }
5840     double r = id;
5841     while (advance() != null)
5842     r = reducer.apply(r, transformer.apply((K)nextKey));
5843     result = r;
5844     for (MapReduceKeysToDoubleTask<K,V> t = this, s;;) {
5845     int c; BulkTask<K,V,?> par;
5846     if ((c = t.pending) == 0) {
5847     for (s = t.rights; s != null; s = t.rights = s.nextRight) {
5848     t.result = reducer.apply(t.result, s.result);
5849     }
5850     if ((par = t.parent) == null ||
5851     !(par instanceof MapReduceKeysToDoubleTask)) {
5852     t.quietlyComplete();
5853     break;
5854     }
5855     t = (MapReduceKeysToDoubleTask<K,V>)par;
5856     }
5857     else if (t.casPending(c, c - 1))
5858     break;
5859 dl 1.119 }
5860 dl 1.128 } catch (Throwable ex) {
5861     return tryCompleteComputation(ex);
5862 dl 1.119 }
5863 dl 1.128 return false;
5864 dl 1.79 }
5865 dl 1.119 public final Double getRawResult() { return result; }
5866     }
5867 dl 1.79
5868 dl 1.128 @SuppressWarnings("serial") static final class MapReduceValuesToDoubleTask<K,V>
5869 dl 1.119 extends BulkTask<K,V,Double> {
5870     final ObjectToDouble<? super V> transformer;
5871     final DoubleByDoubleToDouble reducer;
5872     final double basis;
5873     double result;
5874 dl 1.128 MapReduceValuesToDoubleTask<K,V> rights, nextRight;
5875 dl 1.119 MapReduceValuesToDoubleTask
5876 dl 1.130 (ConcurrentHashMap<K,V> m, BulkTask<K,V,?> p, int b,
5877 dl 1.128 MapReduceValuesToDoubleTask<K,V> nextRight,
5878 dl 1.119 ObjectToDouble<? super V> transformer,
5879     double basis,
5880     DoubleByDoubleToDouble reducer) {
5881 dl 1.130 super(m, p, b); this.nextRight = nextRight;
5882 dl 1.119 this.transformer = transformer;
5883     this.basis = basis; this.reducer = reducer;
5884     }
5885 dl 1.128 @SuppressWarnings("unchecked") public final boolean exec() {
5886 dl 1.119 final ObjectToDouble<? super V> transformer =
5887     this.transformer;
5888     final DoubleByDoubleToDouble reducer = this.reducer;
5889     if (transformer == null || reducer == null)
5890 dl 1.128 return abortOnNullFunction();
5891     try {
5892     final double id = this.basis;
5893     for (int c, b = batch(); b > 1 && baseIndex != baseLimit;) {
5894     do {} while (!casPending(c = pending, c+1));
5895     (rights = new MapReduceValuesToDoubleTask<K,V>
5896 dl 1.130 (map, this, b >>>= 1, rights, transformer, id, reducer)).fork();
5897 dl 1.128 }
5898     double r = id;
5899     Object v;
5900     while ((v = advance()) != null)
5901     r = reducer.apply(r, transformer.apply((V)v));
5902     result = r;
5903     for (MapReduceValuesToDoubleTask<K,V> t = this, s;;) {
5904     int c; BulkTask<K,V,?> par;
5905     if ((c = t.pending) == 0) {
5906     for (s = t.rights; s != null; s = t.rights = s.nextRight) {
5907     t.result = reducer.apply(t.result, s.result);
5908     }
5909     if ((par = t.parent) == null ||
5910     !(par instanceof MapReduceValuesToDoubleTask)) {
5911     t.quietlyComplete();
5912     break;
5913     }
5914     t = (MapReduceValuesToDoubleTask<K,V>)par;
5915     }
5916     else if (t.casPending(c, c - 1))
5917     break;
5918 dl 1.119 }
5919 dl 1.128 } catch (Throwable ex) {
5920     return tryCompleteComputation(ex);
5921 dl 1.119 }
5922 dl 1.128 return false;
5923 dl 1.30 }
5924 dl 1.119 public final Double getRawResult() { return result; }
5925 dl 1.79 }
5926 dl 1.30
5927 dl 1.128 @SuppressWarnings("serial") static final class MapReduceEntriesToDoubleTask<K,V>
5928 dl 1.119 extends BulkTask<K,V,Double> {
5929     final ObjectToDouble<Map.Entry<K,V>> transformer;
5930     final DoubleByDoubleToDouble reducer;
5931     final double basis;
5932     double result;
5933 dl 1.128 MapReduceEntriesToDoubleTask<K,V> rights, nextRight;
5934 dl 1.119 MapReduceEntriesToDoubleTask
5935 dl 1.130 (ConcurrentHashMap<K,V> m, BulkTask<K,V,?> p, int b,
5936 dl 1.128 MapReduceEntriesToDoubleTask<K,V> nextRight,
5937 dl 1.119 ObjectToDouble<Map.Entry<K,V>> transformer,
5938     double basis,
5939     DoubleByDoubleToDouble reducer) {
5940 dl 1.130 super(m, p, b); this.nextRight = nextRight;
5941 dl 1.119 this.transformer = transformer;
5942     this.basis = basis; this.reducer = reducer;
5943     }
5944 dl 1.128 @SuppressWarnings("unchecked") public final boolean exec() {
5945 dl 1.119 final ObjectToDouble<Map.Entry<K,V>> transformer =
5946     this.transformer;
5947     final DoubleByDoubleToDouble reducer = this.reducer;
5948     if (transformer == null || reducer == null)
5949 dl 1.128 return abortOnNullFunction();
5950     try {
5951     final double id = this.basis;
5952     for (int c, b = batch(); b > 1 && baseIndex != baseLimit;) {
5953     do {} while (!casPending(c = pending, c+1));
5954     (rights = new MapReduceEntriesToDoubleTask<K,V>
5955 dl 1.130 (map, this, b >>>= 1, rights, transformer, id, reducer)).fork();
5956 dl 1.128 }
5957     double r = id;
5958     Object v;
5959     while ((v = advance()) != null)
5960     r = reducer.apply(r, transformer.apply(entryFor((K)nextKey, (V)v)));
5961     result = r;
5962     for (MapReduceEntriesToDoubleTask<K,V> t = this, s;;) {
5963     int c; BulkTask<K,V,?> par;
5964     if ((c = t.pending) == 0) {
5965     for (s = t.rights; s != null; s = t.rights = s.nextRight) {
5966     t.result = reducer.apply(t.result, s.result);
5967     }
5968     if ((par = t.parent) == null ||
5969     !(par instanceof MapReduceEntriesToDoubleTask)) {
5970     t.quietlyComplete();
5971     break;
5972     }
5973     t = (MapReduceEntriesToDoubleTask<K,V>)par;
5974     }
5975     else if (t.casPending(c, c - 1))
5976     break;
5977 dl 1.119 }
5978 dl 1.128 } catch (Throwable ex) {
5979     return tryCompleteComputation(ex);
5980 dl 1.119 }
5981 dl 1.128 return false;
5982 dl 1.30 }
5983 dl 1.119 public final Double getRawResult() { return result; }
5984 tim 1.1 }
5985    
5986 dl 1.128 @SuppressWarnings("serial") static final class MapReduceMappingsToDoubleTask<K,V>
5987 dl 1.119 extends BulkTask<K,V,Double> {
5988     final ObjectByObjectToDouble<? super K, ? super V> transformer;
5989     final DoubleByDoubleToDouble reducer;
5990     final double basis;
5991     double result;
5992 dl 1.128 MapReduceMappingsToDoubleTask<K,V> rights, nextRight;
5993 dl 1.119 MapReduceMappingsToDoubleTask
5994 dl 1.130 (ConcurrentHashMap<K,V> m, BulkTask<K,V,?> p, int b,
5995 dl 1.128 MapReduceMappingsToDoubleTask<K,V> nextRight,
5996 dl 1.119 ObjectByObjectToDouble<? super K, ? super V> transformer,
5997     double basis,
5998     DoubleByDoubleToDouble reducer) {
5999 dl 1.130 super(m, p, b); this.nextRight = nextRight;
6000 dl 1.119 this.transformer = transformer;
6001     this.basis = basis; this.reducer = reducer;
6002     }
6003 dl 1.128 @SuppressWarnings("unchecked") public final boolean exec() {
6004 dl 1.119 final ObjectByObjectToDouble<? super K, ? super V> transformer =
6005     this.transformer;
6006     final DoubleByDoubleToDouble reducer = this.reducer;
6007     if (transformer == null || reducer == null)
6008 dl 1.128 return abortOnNullFunction();
6009     try {
6010     final double id = this.basis;
6011     for (int c, b = batch(); b > 1 && baseIndex != baseLimit;) {
6012     do {} while (!casPending(c = pending, c+1));
6013     (rights = new MapReduceMappingsToDoubleTask<K,V>
6014 dl 1.130 (map, this, b >>>= 1, rights, transformer, id, reducer)).fork();
6015 dl 1.128 }
6016     double r = id;
6017     Object v;
6018     while ((v = advance()) != null)
6019     r = reducer.apply(r, transformer.apply((K)nextKey, (V)v));
6020     result = r;
6021     for (MapReduceMappingsToDoubleTask<K,V> t = this, s;;) {
6022     int c; BulkTask<K,V,?> par;
6023     if ((c = t.pending) == 0) {
6024     for (s = t.rights; s != null; s = t.rights = s.nextRight) {
6025     t.result = reducer.apply(t.result, s.result);
6026     }
6027     if ((par = t.parent) == null ||
6028     !(par instanceof MapReduceMappingsToDoubleTask)) {
6029     t.quietlyComplete();
6030     break;
6031     }
6032     t = (MapReduceMappingsToDoubleTask<K,V>)par;
6033     }
6034     else if (t.casPending(c, c - 1))
6035     break;
6036 dl 1.119 }
6037 dl 1.128 } catch (Throwable ex) {
6038     return tryCompleteComputation(ex);
6039 dl 1.119 }
6040 dl 1.128 return false;
6041 dl 1.4 }
6042 dl 1.119 public final Double getRawResult() { return result; }
6043     }
6044    
6045 dl 1.128 @SuppressWarnings("serial") static final class MapReduceKeysToLongTask<K,V>
6046 dl 1.119 extends BulkTask<K,V,Long> {
6047     final ObjectToLong<? super K> transformer;
6048     final LongByLongToLong reducer;
6049     final long basis;
6050     long result;
6051 dl 1.128 MapReduceKeysToLongTask<K,V> rights, nextRight;
6052 dl 1.119 MapReduceKeysToLongTask
6053 dl 1.130 (ConcurrentHashMap<K,V> m, BulkTask<K,V,?> p, int b,
6054 dl 1.128 MapReduceKeysToLongTask<K,V> nextRight,
6055 dl 1.119 ObjectToLong<? super K> transformer,
6056     long basis,
6057     LongByLongToLong reducer) {
6058 dl 1.130 super(m, p, b); this.nextRight = nextRight;
6059 dl 1.119 this.transformer = transformer;
6060     this.basis = basis; this.reducer = reducer;
6061     }
6062 dl 1.128 @SuppressWarnings("unchecked") public final boolean exec() {
6063 dl 1.119 final ObjectToLong<? super K> transformer =
6064     this.transformer;
6065     final LongByLongToLong reducer = this.reducer;
6066     if (transformer == null || reducer == null)
6067 dl 1.128 return abortOnNullFunction();
6068     try {
6069     final long id = this.basis;
6070     for (int c, b = batch(); b > 1 && baseIndex != baseLimit;) {
6071     do {} while (!casPending(c = pending, c+1));
6072     (rights = new MapReduceKeysToLongTask<K,V>
6073 dl 1.130 (map, this, b >>>= 1, rights, transformer, id, reducer)).fork();
6074 dl 1.128 }
6075     long r = id;
6076     while (advance() != null)
6077     r = reducer.apply(r, transformer.apply((K)nextKey));
6078     result = r;
6079     for (MapReduceKeysToLongTask<K,V> t = this, s;;) {
6080     int c; BulkTask<K,V,?> par;
6081     if ((c = t.pending) == 0) {
6082     for (s = t.rights; s != null; s = t.rights = s.nextRight) {
6083     t.result = reducer.apply(t.result, s.result);
6084     }
6085     if ((par = t.parent) == null ||
6086     !(par instanceof MapReduceKeysToLongTask)) {
6087     t.quietlyComplete();
6088     break;
6089     }
6090     t = (MapReduceKeysToLongTask<K,V>)par;
6091     }
6092     else if (t.casPending(c, c - 1))
6093     break;
6094 dl 1.119 }
6095 dl 1.128 } catch (Throwable ex) {
6096     return tryCompleteComputation(ex);
6097 dl 1.119 }
6098 dl 1.128 return false;
6099 dl 1.4 }
6100 dl 1.119 public final Long getRawResult() { return result; }
6101     }
6102    
6103 dl 1.128 @SuppressWarnings("serial") static final class MapReduceValuesToLongTask<K,V>
6104 dl 1.119 extends BulkTask<K,V,Long> {
6105     final ObjectToLong<? super V> transformer;
6106     final LongByLongToLong reducer;
6107     final long basis;
6108     long result;
6109 dl 1.128 MapReduceValuesToLongTask<K,V> rights, nextRight;
6110 dl 1.119 MapReduceValuesToLongTask
6111 dl 1.130 (ConcurrentHashMap<K,V> m, BulkTask<K,V,?> p, int b,
6112 dl 1.128 MapReduceValuesToLongTask<K,V> nextRight,
6113 dl 1.119 ObjectToLong<? super V> transformer,
6114     long basis,
6115     LongByLongToLong reducer) {
6116 dl 1.130 super(m, p, b); this.nextRight = nextRight;
6117 dl 1.119 this.transformer = transformer;
6118     this.basis = basis; this.reducer = reducer;
6119     }
6120 dl 1.128 @SuppressWarnings("unchecked") public final boolean exec() {
6121 dl 1.119 final ObjectToLong<? super V> transformer =
6122     this.transformer;
6123     final LongByLongToLong reducer = this.reducer;
6124     if (transformer == null || reducer == null)
6125 dl 1.128 return abortOnNullFunction();
6126     try {
6127     final long id = this.basis;
6128     for (int c, b = batch(); b > 1 && baseIndex != baseLimit;) {
6129     do {} while (!casPending(c = pending, c+1));
6130     (rights = new MapReduceValuesToLongTask<K,V>
6131 dl 1.130 (map, this, b >>>= 1, rights, transformer, id, reducer)).fork();
6132 dl 1.128 }
6133     long r = id;
6134     Object v;
6135     while ((v = advance()) != null)
6136     r = reducer.apply(r, transformer.apply((V)v));
6137     result = r;
6138     for (MapReduceValuesToLongTask<K,V> t = this, s;;) {
6139     int c; BulkTask<K,V,?> par;
6140     if ((c = t.pending) == 0) {
6141     for (s = t.rights; s != null; s = t.rights = s.nextRight) {
6142     t.result = reducer.apply(t.result, s.result);
6143     }
6144     if ((par = t.parent) == null ||
6145     !(par instanceof MapReduceValuesToLongTask)) {
6146     t.quietlyComplete();
6147     break;
6148     }
6149     t = (MapReduceValuesToLongTask<K,V>)par;
6150     }
6151     else if (t.casPending(c, c - 1))
6152     break;
6153 dl 1.119 }
6154 dl 1.128 } catch (Throwable ex) {
6155     return tryCompleteComputation(ex);
6156 dl 1.119 }
6157 dl 1.128 return false;
6158 jsr166 1.95 }
6159 dl 1.119 public final Long getRawResult() { return result; }
6160     }
6161    
6162 dl 1.128 @SuppressWarnings("serial") static final class MapReduceEntriesToLongTask<K,V>
6163 dl 1.119 extends BulkTask<K,V,Long> {
6164     final ObjectToLong<Map.Entry<K,V>> transformer;
6165     final LongByLongToLong reducer;
6166     final long basis;
6167     long result;
6168 dl 1.128 MapReduceEntriesToLongTask<K,V> rights, nextRight;
6169 dl 1.119 MapReduceEntriesToLongTask
6170 dl 1.130 (ConcurrentHashMap<K,V> m, BulkTask<K,V,?> p, int b,
6171 dl 1.128 MapReduceEntriesToLongTask<K,V> nextRight,
6172 dl 1.119 ObjectToLong<Map.Entry<K,V>> transformer,
6173     long basis,
6174     LongByLongToLong reducer) {
6175 dl 1.130 super(m, p, b); this.nextRight = nextRight;
6176 dl 1.119 this.transformer = transformer;
6177     this.basis = basis; this.reducer = reducer;
6178     }
6179 dl 1.128 @SuppressWarnings("unchecked") public final boolean exec() {
6180 dl 1.119 final ObjectToLong<Map.Entry<K,V>> transformer =
6181     this.transformer;
6182     final LongByLongToLong reducer = this.reducer;
6183     if (transformer == null || reducer == null)
6184 dl 1.128 return abortOnNullFunction();
6185     try {
6186     final long id = this.basis;
6187     for (int c, b = batch(); b > 1 && baseIndex != baseLimit;) {
6188     do {} while (!casPending(c = pending, c+1));
6189     (rights = new MapReduceEntriesToLongTask<K,V>
6190 dl 1.130 (map, this, b >>>= 1, rights, transformer, id, reducer)).fork();
6191 dl 1.128 }
6192     long r = id;
6193     Object v;
6194     while ((v = advance()) != null)
6195     r = reducer.apply(r, transformer.apply(entryFor((K)nextKey, (V)v)));
6196     result = r;
6197     for (MapReduceEntriesToLongTask<K,V> t = this, s;;) {
6198     int c; BulkTask<K,V,?> par;
6199     if ((c = t.pending) == 0) {
6200     for (s = t.rights; s != null; s = t.rights = s.nextRight) {
6201     t.result = reducer.apply(t.result, s.result);
6202     }
6203     if ((par = t.parent) == null ||
6204     !(par instanceof MapReduceEntriesToLongTask)) {
6205     t.quietlyComplete();
6206     break;
6207     }
6208     t = (MapReduceEntriesToLongTask<K,V>)par;
6209     }
6210     else if (t.casPending(c, c - 1))
6211     break;
6212 dl 1.119 }
6213 dl 1.128 } catch (Throwable ex) {
6214     return tryCompleteComputation(ex);
6215 dl 1.119 }
6216 dl 1.128 return false;
6217 dl 1.4 }
6218 dl 1.119 public final Long getRawResult() { return result; }
6219 tim 1.1 }
6220    
6221 dl 1.128 @SuppressWarnings("serial") static final class MapReduceMappingsToLongTask<K,V>
6222 dl 1.119 extends BulkTask<K,V,Long> {
6223     final ObjectByObjectToLong<? super K, ? super V> transformer;
6224     final LongByLongToLong reducer;
6225     final long basis;
6226     long result;
6227 dl 1.128 MapReduceMappingsToLongTask<K,V> rights, nextRight;
6228 dl 1.119 MapReduceMappingsToLongTask
6229 dl 1.130 (ConcurrentHashMap<K,V> m, BulkTask<K,V,?> p, int b,
6230 dl 1.128 MapReduceMappingsToLongTask<K,V> nextRight,
6231 dl 1.119 ObjectByObjectToLong<? super K, ? super V> transformer,
6232     long basis,
6233     LongByLongToLong reducer) {
6234 dl 1.130 super(m, p, b); this.nextRight = nextRight;
6235 dl 1.119 this.transformer = transformer;
6236     this.basis = basis; this.reducer = reducer;
6237     }
6238 dl 1.128 @SuppressWarnings("unchecked") public final boolean exec() {
6239 dl 1.119 final ObjectByObjectToLong<? super K, ? super V> transformer =
6240     this.transformer;
6241     final LongByLongToLong reducer = this.reducer;
6242     if (transformer == null || reducer == null)
6243 dl 1.128 return abortOnNullFunction();
6244     try {
6245     final long id = this.basis;
6246     for (int c, b = batch(); b > 1 && baseIndex != baseLimit;) {
6247     do {} while (!casPending(c = pending, c+1));
6248     (rights = new MapReduceMappingsToLongTask<K,V>
6249 dl 1.130 (map, this, b >>>= 1, rights, transformer, id, reducer)).fork();
6250 dl 1.128 }
6251     long r = id;
6252     Object v;
6253     while ((v = advance()) != null)
6254     r = reducer.apply(r, transformer.apply((K)nextKey, (V)v));
6255     result = r;
6256     for (MapReduceMappingsToLongTask<K,V> t = this, s;;) {
6257     int c; BulkTask<K,V,?> par;
6258     if ((c = t.pending) == 0) {
6259     for (s = t.rights; s != null; s = t.rights = s.nextRight) {
6260     t.result = reducer.apply(t.result, s.result);
6261     }
6262     if ((par = t.parent) == null ||
6263     !(par instanceof MapReduceMappingsToLongTask)) {
6264     t.quietlyComplete();
6265     break;
6266     }
6267     t = (MapReduceMappingsToLongTask<K,V>)par;
6268     }
6269     else if (t.casPending(c, c - 1))
6270     break;
6271 dl 1.119 }
6272 dl 1.128 } catch (Throwable ex) {
6273     return tryCompleteComputation(ex);
6274 dl 1.119 }
6275 dl 1.128 return false;
6276 dl 1.4 }
6277 dl 1.119 public final Long getRawResult() { return result; }
6278 tim 1.1 }
6279    
6280 dl 1.128 @SuppressWarnings("serial") static final class MapReduceKeysToIntTask<K,V>
6281 dl 1.119 extends BulkTask<K,V,Integer> {
6282     final ObjectToInt<? super K> transformer;
6283     final IntByIntToInt reducer;
6284     final int basis;
6285     int result;
6286 dl 1.128 MapReduceKeysToIntTask<K,V> rights, nextRight;
6287 dl 1.119 MapReduceKeysToIntTask
6288 dl 1.130 (ConcurrentHashMap<K,V> m, BulkTask<K,V,?> p, int b,
6289 dl 1.128 MapReduceKeysToIntTask<K,V> nextRight,
6290 dl 1.119 ObjectToInt<? super K> transformer,
6291     int basis,
6292     IntByIntToInt reducer) {
6293 dl 1.130 super(m, p, b); this.nextRight = nextRight;
6294 dl 1.119 this.transformer = transformer;
6295     this.basis = basis; this.reducer = reducer;
6296     }
6297 dl 1.128 @SuppressWarnings("unchecked") public final boolean exec() {
6298 dl 1.119 final ObjectToInt<? super K> transformer =
6299     this.transformer;
6300     final IntByIntToInt reducer = this.reducer;
6301     if (transformer == null || reducer == null)
6302 dl 1.128 return abortOnNullFunction();
6303     try {
6304     final int id = this.basis;
6305     for (int c, b = batch(); b > 1 && baseIndex != baseLimit;) {
6306     do {} while (!casPending(c = pending, c+1));
6307     (rights = new MapReduceKeysToIntTask<K,V>
6308 dl 1.130 (map, this, b >>>= 1, rights, transformer, id, reducer)).fork();
6309 dl 1.128 }
6310     int r = id;
6311     while (advance() != null)
6312     r = reducer.apply(r, transformer.apply((K)nextKey));
6313     result = r;
6314     for (MapReduceKeysToIntTask<K,V> t = this, s;;) {
6315     int c; BulkTask<K,V,?> par;
6316     if ((c = t.pending) == 0) {
6317     for (s = t.rights; s != null; s = t.rights = s.nextRight) {
6318     t.result = reducer.apply(t.result, s.result);
6319     }
6320     if ((par = t.parent) == null ||
6321     !(par instanceof MapReduceKeysToIntTask)) {
6322     t.quietlyComplete();
6323     break;
6324     }
6325     t = (MapReduceKeysToIntTask<K,V>)par;
6326     }
6327     else if (t.casPending(c, c - 1))
6328     break;
6329 dl 1.119 }
6330 dl 1.128 } catch (Throwable ex) {
6331     return tryCompleteComputation(ex);
6332 dl 1.119 }
6333 dl 1.128 return false;
6334 dl 1.30 }
6335 dl 1.119 public final Integer getRawResult() { return result; }
6336 dl 1.30 }
6337    
6338 dl 1.128 @SuppressWarnings("serial") static final class MapReduceValuesToIntTask<K,V>
6339 dl 1.119 extends BulkTask<K,V,Integer> {
6340     final ObjectToInt<? super V> transformer;
6341     final IntByIntToInt reducer;
6342     final int basis;
6343     int result;
6344 dl 1.128 MapReduceValuesToIntTask<K,V> rights, nextRight;
6345 dl 1.119 MapReduceValuesToIntTask
6346 dl 1.130 (ConcurrentHashMap<K,V> m, BulkTask<K,V,?> p, int b,
6347 dl 1.128 MapReduceValuesToIntTask<K,V> nextRight,
6348 dl 1.119 ObjectToInt<? super V> transformer,
6349     int basis,
6350     IntByIntToInt reducer) {
6351 dl 1.130 super(m, p, b); this.nextRight = nextRight;
6352 dl 1.119 this.transformer = transformer;
6353     this.basis = basis; this.reducer = reducer;
6354     }
6355 dl 1.128 @SuppressWarnings("unchecked") public final boolean exec() {
6356 dl 1.119 final ObjectToInt<? super V> transformer =
6357     this.transformer;
6358     final IntByIntToInt reducer = this.reducer;
6359     if (transformer == null || reducer == null)
6360 dl 1.128 return abortOnNullFunction();
6361     try {
6362     final int id = this.basis;
6363     for (int c, b = batch(); b > 1 && baseIndex != baseLimit;) {
6364     do {} while (!casPending(c = pending, c+1));
6365     (rights = new MapReduceValuesToIntTask<K,V>
6366 dl 1.130 (map, this, b >>>= 1, rights, transformer, id, reducer)).fork();
6367 dl 1.128 }
6368     int r = id;
6369     Object v;
6370     while ((v = advance()) != null)
6371     r = reducer.apply(r, transformer.apply((V)v));
6372     result = r;
6373     for (MapReduceValuesToIntTask<K,V> t = this, s;;) {
6374     int c; BulkTask<K,V,?> par;
6375     if ((c = t.pending) == 0) {
6376     for (s = t.rights; s != null; s = t.rights = s.nextRight) {
6377     t.result = reducer.apply(t.result, s.result);
6378     }
6379     if ((par = t.parent) == null ||
6380     !(par instanceof MapReduceValuesToIntTask)) {
6381     t.quietlyComplete();
6382     break;
6383     }
6384     t = (MapReduceValuesToIntTask<K,V>)par;
6385     }
6386     else if (t.casPending(c, c - 1))
6387     break;
6388 dl 1.119 }
6389 dl 1.128 } catch (Throwable ex) {
6390     return tryCompleteComputation(ex);
6391 dl 1.2 }
6392 dl 1.128 return false;
6393 tim 1.1 }
6394 dl 1.119 public final Integer getRawResult() { return result; }
6395 tim 1.1 }
6396    
6397 dl 1.128 @SuppressWarnings("serial") static final class MapReduceEntriesToIntTask<K,V>
6398 dl 1.119 extends BulkTask<K,V,Integer> {
6399     final ObjectToInt<Map.Entry<K,V>> transformer;
6400     final IntByIntToInt reducer;
6401     final int basis;
6402     int result;
6403 dl 1.128 MapReduceEntriesToIntTask<K,V> rights, nextRight;
6404 dl 1.119 MapReduceEntriesToIntTask
6405 dl 1.130 (ConcurrentHashMap<K,V> m, BulkTask<K,V,?> p, int b,
6406 dl 1.128 MapReduceEntriesToIntTask<K,V> nextRight,
6407 dl 1.119 ObjectToInt<Map.Entry<K,V>> transformer,
6408     int basis,
6409     IntByIntToInt reducer) {
6410 dl 1.130 super(m, p, b); this.nextRight = nextRight;
6411 dl 1.119 this.transformer = transformer;
6412     this.basis = basis; this.reducer = reducer;
6413     }
6414 dl 1.128 @SuppressWarnings("unchecked") public final boolean exec() {
6415 dl 1.119 final ObjectToInt<Map.Entry<K,V>> transformer =
6416     this.transformer;
6417     final IntByIntToInt reducer = this.reducer;
6418     if (transformer == null || reducer == null)
6419 dl 1.128 return abortOnNullFunction();
6420     try {
6421     final int id = this.basis;
6422     for (int c, b = batch(); b > 1 && baseIndex != baseLimit;) {
6423     do {} while (!casPending(c = pending, c+1));
6424     (rights = new MapReduceEntriesToIntTask<K,V>
6425 dl 1.130 (map, this, b >>>= 1, rights, transformer, id, reducer)).fork();
6426 dl 1.128 }
6427     int r = id;
6428     Object v;
6429     while ((v = advance()) != null)
6430     r = reducer.apply(r, transformer.apply(entryFor((K)nextKey, (V)v)));
6431     result = r;
6432     for (MapReduceEntriesToIntTask<K,V> t = this, s;;) {
6433     int c; BulkTask<K,V,?> par;
6434     if ((c = t.pending) == 0) {
6435     for (s = t.rights; s != null; s = t.rights = s.nextRight) {
6436     t.result = reducer.apply(t.result, s.result);
6437     }
6438     if ((par = t.parent) == null ||
6439     !(par instanceof MapReduceEntriesToIntTask)) {
6440     t.quietlyComplete();
6441     break;
6442     }
6443     t = (MapReduceEntriesToIntTask<K,V>)par;
6444     }
6445     else if (t.casPending(c, c - 1))
6446     break;
6447 dl 1.119 }
6448 dl 1.128 } catch (Throwable ex) {
6449     return tryCompleteComputation(ex);
6450 dl 1.99 }
6451 dl 1.128 return false;
6452 dl 1.4 }
6453 dl 1.119 public final Integer getRawResult() { return result; }
6454     }
6455 tim 1.1
6456 dl 1.128 @SuppressWarnings("serial") static final class MapReduceMappingsToIntTask<K,V>
6457 dl 1.119 extends BulkTask<K,V,Integer> {
6458     final ObjectByObjectToInt<? super K, ? super V> transformer;
6459     final IntByIntToInt reducer;
6460     final int basis;
6461     int result;
6462 dl 1.128 MapReduceMappingsToIntTask<K,V> rights, nextRight;
6463 dl 1.119 MapReduceMappingsToIntTask
6464 dl 1.130 (ConcurrentHashMap<K,V> m, BulkTask<K,V,?> p, int b,
6465     MapReduceMappingsToIntTask<K,V> rights,
6466 dl 1.119 ObjectByObjectToInt<? super K, ? super V> transformer,
6467     int basis,
6468     IntByIntToInt reducer) {
6469 dl 1.130 super(m, p, b); this.nextRight = nextRight;
6470 dl 1.119 this.transformer = transformer;
6471     this.basis = basis; this.reducer = reducer;
6472     }
6473 dl 1.128 @SuppressWarnings("unchecked") public final boolean exec() {
6474 dl 1.119 final ObjectByObjectToInt<? super K, ? super V> transformer =
6475     this.transformer;
6476     final IntByIntToInt reducer = this.reducer;
6477     if (transformer == null || reducer == null)
6478 dl 1.128 return abortOnNullFunction();
6479     try {
6480     final int id = this.basis;
6481     for (int c, b = batch(); b > 1 && baseIndex != baseLimit;) {
6482     do {} while (!casPending(c = pending, c+1));
6483     (rights = new MapReduceMappingsToIntTask<K,V>
6484 dl 1.130 (map, this, b >>>= 1, rights, transformer, id, reducer)).fork();
6485 dl 1.128 }
6486     int r = id;
6487     Object v;
6488     while ((v = advance()) != null)
6489     r = reducer.apply(r, transformer.apply((K)nextKey, (V)v));
6490     result = r;
6491     for (MapReduceMappingsToIntTask<K,V> t = this, s;;) {
6492     int c; BulkTask<K,V,?> par;
6493     if ((c = t.pending) == 0) {
6494     for (s = t.rights; s != null; s = t.rights = s.nextRight) {
6495     t.result = reducer.apply(t.result, s.result);
6496     }
6497     if ((par = t.parent) == null ||
6498     !(par instanceof MapReduceMappingsToIntTask)) {
6499     t.quietlyComplete();
6500     break;
6501     }
6502     t = (MapReduceMappingsToIntTask<K,V>)par;
6503     }
6504     else if (t.casPending(c, c - 1))
6505     break;
6506 dl 1.119 }
6507 dl 1.128 } catch (Throwable ex) {
6508     return tryCompleteComputation(ex);
6509 dl 1.119 }
6510 dl 1.128 return false;
6511 tim 1.1 }
6512 dl 1.119 public final Integer getRawResult() { return result; }
6513 tim 1.1 }
6514 dl 1.99
6515 dl 1.130
6516 dl 1.99 // Unsafe mechanics
6517     private static final sun.misc.Unsafe UNSAFE;
6518 dl 1.119 private static final long counterOffset;
6519     private static final long sizeCtlOffset;
6520     private static final long ABASE;
6521     private static final int ASHIFT;
6522 dl 1.99
6523     static {
6524 dl 1.119 int ss;
6525 dl 1.99 try {
6526 dl 1.126 UNSAFE = sun.misc.Unsafe.getUnsafe();
6527 dl 1.119 Class<?> k = ConcurrentHashMap.class;
6528     counterOffset = UNSAFE.objectFieldOffset
6529     (k.getDeclaredField("counter"));
6530     sizeCtlOffset = UNSAFE.objectFieldOffset
6531     (k.getDeclaredField("sizeCtl"));
6532     Class<?> sc = Node[].class;
6533     ABASE = UNSAFE.arrayBaseOffset(sc);
6534 dl 1.99 ss = UNSAFE.arrayIndexScale(sc);
6535     } catch (Exception e) {
6536     throw new Error(e);
6537     }
6538 dl 1.119 if ((ss & (ss-1)) != 0)
6539 dl 1.99 throw new Error("data type scale not a power of two");
6540 dl 1.119 ASHIFT = 31 - Integer.numberOfLeadingZeros(ss);
6541 dl 1.99 }
6542 dl 1.128 }