ViewVC Help
View File | Revision Log | Show Annotations | Download File | Root Listing
root/jsr166/jsr166/src/jdk7/java/util/concurrent/ConcurrentHashMap.java
Revision: 1.16
Committed: Mon Feb 11 20:43:59 2013 UTC (11 years, 3 months ago) by jsr166
Branch: MAIN
Changes since 1.15: +1 -5 lines
Log Message:
remove redundant NPE checks

File Contents

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