ViewVC Help
View File | Revision Log | Show Annotations | Download File | Root Listing
root/jsr166/jsr166/src/jdk7/java/util/concurrent/ConcurrentHashMap.java
Revision: 1.6
Committed: Thu Jan 17 14:20:57 2013 UTC (11 years, 4 months ago) by dl
Branch: MAIN
Changes since 1.5: +3 -1 lines
Log Message:
correct putAll counts

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     /*
1976     * 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     * @param mappedValue the mapped value to use for any
2895     * additions.
2896     * @return the set view
2897     * @throws NullPointerException if the mappedValue is null
2898     */
2899     public KeySetView<K,V> keySet(V mappedValue) {
2900     if (mappedValue == null)
2901     throw new NullPointerException();
2902     return new KeySetView<K,V>(this, mappedValue);
2903     }
2904    
2905     /**
2906     * Returns a {@link Collection} view of the values contained in this map.
2907     * The collection is backed by the map, so changes to the map are
2908     * reflected in the collection, and vice-versa.
2909     */
2910     public ValuesView<K,V> values() {
2911     ValuesView<K,V> vs = values;
2912     return (vs != null) ? vs : (values = new ValuesView<K,V>(this));
2913     }
2914    
2915     /**
2916     * Returns a {@link Set} view of the mappings contained in this map.
2917     * The set is backed by the map, so changes to the map are
2918     * reflected in the set, and vice-versa. The set supports element
2919     * removal, which removes the corresponding mapping from the map,
2920     * via the {@code Iterator.remove}, {@code Set.remove},
2921     * {@code removeAll}, {@code retainAll}, and {@code clear}
2922     * operations. It does not support the {@code add} or
2923     * {@code addAll} operations.
2924     *
2925     * <p>The view's {@code iterator} is a "weakly consistent" iterator
2926     * that will never throw {@link ConcurrentModificationException},
2927     * and guarantees to traverse elements as they existed upon
2928     * construction of the iterator, and may (but is not guaranteed to)
2929     * reflect any modifications subsequent to construction.
2930     */
2931     public Set<Map.Entry<K,V>> entrySet() {
2932     EntrySetView<K,V> es = entrySet;
2933     return (es != null) ? es : (entrySet = new EntrySetView<K,V>(this));
2934     }
2935    
2936     /**
2937     * Returns an enumeration of the keys in this table.
2938     *
2939     * @return an enumeration of the keys in this table
2940     * @see #keySet()
2941     */
2942     public Enumeration<K> keys() {
2943     return new KeyIterator<K,V>(this);
2944     }
2945    
2946     /**
2947     * Returns an enumeration of the values in this table.
2948     *
2949     * @return an enumeration of the values in this table
2950     * @see #values()
2951     */
2952     public Enumeration<V> elements() {
2953     return new ValueIterator<K,V>(this);
2954     }
2955    
2956     /**
2957     * Returns a partitionable iterator of the keys in this map.
2958     *
2959     * @return a partitionable iterator of the keys in this map
2960     */
2961     public Spliterator<K> keySpliterator() {
2962     return new KeyIterator<K,V>(this);
2963     }
2964    
2965     /**
2966     * Returns a partitionable iterator of the values in this map.
2967     *
2968     * @return a partitionable iterator of the values in this map
2969     */
2970     public Spliterator<V> valueSpliterator() {
2971     return new ValueIterator<K,V>(this);
2972     }
2973    
2974     /**
2975     * Returns a partitionable iterator of the entries in this map.
2976     *
2977     * @return a partitionable iterator of the entries in this map
2978     */
2979     public Spliterator<Map.Entry<K,V>> entrySpliterator() {
2980     return new EntryIterator<K,V>(this);
2981     }
2982    
2983     /**
2984     * Returns the hash code value for this {@link Map}, i.e.,
2985     * the sum of, for each key-value pair in the map,
2986     * {@code key.hashCode() ^ value.hashCode()}.
2987     *
2988     * @return the hash code value for this map
2989     */
2990     public int hashCode() {
2991     int h = 0;
2992     Traverser<K,V,Object> it = new Traverser<K,V,Object>(this);
2993     V v;
2994     while ((v = it.advance()) != null) {
2995     h += it.nextKey.hashCode() ^ v.hashCode();
2996     }
2997     return h;
2998     }
2999    
3000     /**
3001     * Returns a string representation of this map. The string
3002     * representation consists of a list of key-value mappings (in no
3003     * particular order) enclosed in braces ("{@code {}}"). Adjacent
3004     * mappings are separated by the characters {@code ", "} (comma
3005     * and space). Each key-value mapping is rendered as the key
3006     * followed by an equals sign ("{@code =}") followed by the
3007     * associated value.
3008     *
3009     * @return a string representation of this map
3010     */
3011     public String toString() {
3012     Traverser<K,V,Object> it = new Traverser<K,V,Object>(this);
3013     StringBuilder sb = new StringBuilder();
3014     sb.append('{');
3015     V v;
3016     if ((v = it.advance()) != null) {
3017     for (;;) {
3018     Object k = it.nextKey;
3019     sb.append(k == this ? "(this Map)" : k);
3020     sb.append('=');
3021     sb.append(v == this ? "(this Map)" : v);
3022     if ((v = it.advance()) == null)
3023     break;
3024     sb.append(',').append(' ');
3025     }
3026     }
3027     return sb.append('}').toString();
3028     }
3029    
3030     /**
3031     * Compares the specified object with this map for equality.
3032     * Returns {@code true} if the given object is a map with the same
3033     * mappings as this map. This operation may return misleading
3034     * results if either map is concurrently modified during execution
3035     * of this method.
3036     *
3037     * @param o object to be compared for equality with this map
3038     * @return {@code true} if the specified object is equal to this map
3039     */
3040     public boolean equals(Object o) {
3041     if (o != this) {
3042     if (!(o instanceof Map))
3043     return false;
3044     Map<?,?> m = (Map<?,?>) o;
3045     Traverser<K,V,Object> it = new Traverser<K,V,Object>(this);
3046     V val;
3047     while ((val = it.advance()) != null) {
3048     Object v = m.get(it.nextKey);
3049     if (v == null || (v != val && !v.equals(val)))
3050     return false;
3051     }
3052     for (Map.Entry<?,?> e : m.entrySet()) {
3053     Object mk, mv, v;
3054     if ((mk = e.getKey()) == null ||
3055     (mv = e.getValue()) == null ||
3056     (v = internalGet(mk)) == null ||
3057     (mv != v && !mv.equals(v)))
3058     return false;
3059     }
3060     }
3061     return true;
3062     }
3063    
3064     /* ----------------Iterators -------------- */
3065    
3066     @SuppressWarnings("serial") static final class KeyIterator<K,V>
3067     extends Traverser<K,V,Object>
3068     implements Spliterator<K>, Enumeration<K> {
3069     KeyIterator(ConcurrentHashMap<K, V> map) { super(map); }
3070     KeyIterator(ConcurrentHashMap<K, V> map, Traverser<K,V,Object> it) {
3071     super(map, it, -1);
3072     }
3073     public KeyIterator<K,V> split() {
3074     if (nextKey != null)
3075     throw new IllegalStateException();
3076     return new KeyIterator<K,V>(map, this);
3077     }
3078     @SuppressWarnings("unchecked") public final K next() {
3079     if (nextVal == null && advance() == null)
3080     throw new NoSuchElementException();
3081     Object k = nextKey;
3082     nextVal = null;
3083     return (K) k;
3084     }
3085    
3086     public final K nextElement() { return next(); }
3087     }
3088    
3089     @SuppressWarnings("serial") static final class ValueIterator<K,V>
3090     extends Traverser<K,V,Object>
3091     implements Spliterator<V>, Enumeration<V> {
3092     ValueIterator(ConcurrentHashMap<K, V> map) { super(map); }
3093     ValueIterator(ConcurrentHashMap<K, V> map, Traverser<K,V,Object> it) {
3094     super(map, it, -1);
3095     }
3096     public ValueIterator<K,V> split() {
3097     if (nextKey != null)
3098     throw new IllegalStateException();
3099     return new ValueIterator<K,V>(map, this);
3100     }
3101    
3102     public final V next() {
3103     V v;
3104     if ((v = nextVal) == null && (v = advance()) == null)
3105     throw new NoSuchElementException();
3106     nextVal = null;
3107     return v;
3108     }
3109    
3110     public final V nextElement() { return next(); }
3111     }
3112    
3113     @SuppressWarnings("serial") static final class EntryIterator<K,V>
3114     extends Traverser<K,V,Object>
3115     implements Spliterator<Map.Entry<K,V>> {
3116     EntryIterator(ConcurrentHashMap<K, V> map) { super(map); }
3117     EntryIterator(ConcurrentHashMap<K, V> map, Traverser<K,V,Object> it) {
3118     super(map, it, -1);
3119     }
3120     public EntryIterator<K,V> split() {
3121     if (nextKey != null)
3122     throw new IllegalStateException();
3123     return new EntryIterator<K,V>(map, this);
3124     }
3125    
3126     @SuppressWarnings("unchecked") public final Map.Entry<K,V> next() {
3127     V v;
3128     if ((v = nextVal) == null && (v = advance()) == null)
3129     throw new NoSuchElementException();
3130     Object k = nextKey;
3131     nextVal = null;
3132     return new MapEntry<K,V>((K)k, v, map);
3133     }
3134     }
3135    
3136     /**
3137     * Exported Entry for iterators
3138     */
3139     static final class MapEntry<K,V> implements Map.Entry<K, V> {
3140     final K key; // non-null
3141     V val; // non-null
3142     final ConcurrentHashMap<K, V> map;
3143     MapEntry(K key, V val, ConcurrentHashMap<K, V> map) {
3144     this.key = key;
3145     this.val = val;
3146     this.map = map;
3147     }
3148     public final K getKey() { return key; }
3149     public final V getValue() { return val; }
3150     public final int hashCode() { return key.hashCode() ^ val.hashCode(); }
3151     public final String toString(){ return key + "=" + val; }
3152    
3153     public final boolean equals(Object o) {
3154     Object k, v; Map.Entry<?,?> e;
3155     return ((o instanceof Map.Entry) &&
3156     (k = (e = (Map.Entry<?,?>)o).getKey()) != null &&
3157     (v = e.getValue()) != null &&
3158     (k == key || k.equals(key)) &&
3159     (v == val || v.equals(val)));
3160     }
3161    
3162     /**
3163     * Sets our entry's value and writes through to the map. The
3164     * value to return is somewhat arbitrary here. Since we do not
3165     * necessarily track asynchronous changes, the most recent
3166     * "previous" value could be different from what we return (or
3167     * could even have been removed in which case the put will
3168     * re-establish). We do not and cannot guarantee more.
3169     */
3170     public final V setValue(V value) {
3171     if (value == null) throw new NullPointerException();
3172     V v = val;
3173     val = value;
3174     map.put(key, value);
3175     return v;
3176     }
3177     }
3178    
3179     /**
3180     * Returns exportable snapshot entry for the given key and value
3181     * when write-through can't or shouldn't be used.
3182     */
3183     static <K,V> AbstractMap.SimpleEntry<K,V> entryFor(K k, V v) {
3184     return new AbstractMap.SimpleEntry<K,V>(k, v);
3185     }
3186    
3187     /* ---------------- Serialization Support -------------- */
3188    
3189     /**
3190     * Stripped-down version of helper class used in previous version,
3191     * declared for the sake of serialization compatibility
3192     */
3193     static class Segment<K,V> implements Serializable {
3194     private static final long serialVersionUID = 2249069246763182397L;
3195     final float loadFactor;
3196     Segment(float lf) { this.loadFactor = lf; }
3197     }
3198    
3199     /**
3200     * Saves the state of the {@code ConcurrentHashMap} instance to a
3201     * stream (i.e., serializes it).
3202     * @param s the stream
3203     * @serialData
3204     * the key (Object) and value (Object)
3205     * for each key-value mapping, followed by a null pair.
3206     * The key-value mappings are emitted in no particular order.
3207     */
3208     @SuppressWarnings("unchecked") private void writeObject
3209     (java.io.ObjectOutputStream s)
3210     throws java.io.IOException {
3211     if (segments == null) { // for serialization compatibility
3212     segments = (Segment<K,V>[])
3213     new Segment<?,?>[DEFAULT_CONCURRENCY_LEVEL];
3214     for (int i = 0; i < segments.length; ++i)
3215     segments[i] = new Segment<K,V>(LOAD_FACTOR);
3216     }
3217     s.defaultWriteObject();
3218     Traverser<K,V,Object> it = new Traverser<K,V,Object>(this);
3219     V v;
3220     while ((v = it.advance()) != null) {
3221     s.writeObject(it.nextKey);
3222     s.writeObject(v);
3223     }
3224     s.writeObject(null);
3225     s.writeObject(null);
3226     segments = null; // throw away
3227     }
3228    
3229     /**
3230     * Reconstitutes the instance from a stream (that is, deserializes it).
3231     * @param s the stream
3232     */
3233     @SuppressWarnings("unchecked") private void readObject
3234     (java.io.ObjectInputStream s)
3235     throws java.io.IOException, ClassNotFoundException {
3236     s.defaultReadObject();
3237     this.segments = null; // unneeded
3238    
3239     // Create all nodes, then place in table once size is known
3240     long size = 0L;
3241     Node<V> p = null;
3242     for (;;) {
3243     K k = (K) s.readObject();
3244     V v = (V) s.readObject();
3245     if (k != null && v != null) {
3246     int h = spread(k.hashCode());
3247     p = new Node<V>(h, k, v, p);
3248     ++size;
3249     }
3250     else
3251     break;
3252     }
3253     if (p != null) {
3254     boolean init = false;
3255     int n;
3256     if (size >= (long)(MAXIMUM_CAPACITY >>> 1))
3257     n = MAXIMUM_CAPACITY;
3258     else {
3259     int sz = (int)size;
3260     n = tableSizeFor(sz + (sz >>> 1) + 1);
3261     }
3262     int sc = sizeCtl;
3263     boolean collide = false;
3264     if (n > sc &&
3265     U.compareAndSwapInt(this, SIZECTL, sc, -1)) {
3266     try {
3267     if (table == null) {
3268     init = true;
3269     @SuppressWarnings("rawtypes") Node[] rt = new Node[n];
3270     Node<V>[] tab = (Node<V>[])rt;
3271     int mask = n - 1;
3272     while (p != null) {
3273     int j = p.hash & mask;
3274     Node<V> next = p.next;
3275     Node<V> q = p.next = tabAt(tab, j);
3276     setTabAt(tab, j, p);
3277     if (!collide && q != null && q.hash == p.hash)
3278     collide = true;
3279     p = next;
3280     }
3281     table = tab;
3282     addCount(size, -1);
3283     sc = n - (n >>> 2);
3284     }
3285     } finally {
3286     sizeCtl = sc;
3287     }
3288     if (collide) { // rescan and convert to TreeBins
3289     Node<V>[] tab = table;
3290     for (int i = 0; i < tab.length; ++i) {
3291     int c = 0;
3292     for (Node<V> e = tabAt(tab, i); e != null; e = e.next) {
3293     if (++c > TREE_THRESHOLD &&
3294     (e.key instanceof Comparable)) {
3295     replaceWithTreeBin(tab, i, e.key);
3296     break;
3297     }
3298     }
3299     }
3300     }
3301     }
3302     if (!init) { // Can only happen if unsafely published.
3303     while (p != null) {
3304     internalPut((K)p.key, p.val, false);
3305     p = p.next;
3306     }
3307     }
3308     }
3309     }
3310    
3311     // -------------------------------------------------------
3312    
3313     // Sams
3314     /** Interface describing a void action of one argument */
3315     public interface Action<A> { void apply(A a); }
3316     /** Interface describing a void action of two arguments */
3317     public interface BiAction<A,B> { void apply(A a, B b); }
3318     /** Interface describing a function of one argument */
3319     public interface Fun<A,T> { T apply(A a); }
3320     /** Interface describing a function of two arguments */
3321     public interface BiFun<A,B,T> { T apply(A a, B b); }
3322     /** Interface describing a function of no arguments */
3323     public interface Generator<T> { T apply(); }
3324     /** Interface describing a function mapping its argument to a double */
3325     public interface ObjectToDouble<A> { double apply(A a); }
3326     /** Interface describing a function mapping its argument to a long */
3327     public interface ObjectToLong<A> { long apply(A a); }
3328     /** Interface describing a function mapping its argument to an int */
3329     public interface ObjectToInt<A> {int apply(A a); }
3330     /** Interface describing a function mapping two arguments to a double */
3331     public interface ObjectByObjectToDouble<A,B> { double apply(A a, B b); }
3332     /** Interface describing a function mapping two arguments to a long */
3333     public interface ObjectByObjectToLong<A,B> { long apply(A a, B b); }
3334     /** Interface describing a function mapping two arguments to an int */
3335     public interface ObjectByObjectToInt<A,B> {int apply(A a, B b); }
3336     /** Interface describing a function mapping a double to a double */
3337     public interface DoubleToDouble { double apply(double a); }
3338     /** Interface describing a function mapping a long to a long */
3339     public interface LongToLong { long apply(long a); }
3340     /** Interface describing a function mapping an int to an int */
3341     public interface IntToInt { int apply(int a); }
3342     /** Interface describing a function mapping two doubles to a double */
3343     public interface DoubleByDoubleToDouble { double apply(double a, double b); }
3344     /** Interface describing a function mapping two longs to a long */
3345     public interface LongByLongToLong { long apply(long a, long b); }
3346     /** Interface describing a function mapping two ints to an int */
3347     public interface IntByIntToInt { int apply(int a, int b); }
3348    
3349    
3350     // -------------------------------------------------------
3351    
3352     // Sequential bulk operations
3353    
3354     /**
3355     * Performs the given action for each (key, value).
3356     *
3357     * @param action the action
3358     */
3359     @SuppressWarnings("unchecked") public void forEachSequentially
3360     (BiAction<K,V> action) {
3361     if (action == null) throw new NullPointerException();
3362     Traverser<K,V,Object> it = new Traverser<K,V,Object>(this);
3363     V v;
3364     while ((v = it.advance()) != null)
3365     action.apply((K)it.nextKey, v);
3366     }
3367    
3368     /**
3369     * Performs the given action for each non-null transformation
3370     * of each (key, value).
3371     *
3372     * @param transformer a function returning the transformation
3373     * for an element, or null of there is no transformation (in
3374     * which case the action is not applied).
3375     * @param action the action
3376     */
3377     @SuppressWarnings("unchecked") public <U> void forEachSequentially
3378     (BiFun<? super K, ? super V, ? extends U> transformer,
3379     Action<U> action) {
3380     if (transformer == null || action == null)
3381     throw new NullPointerException();
3382     Traverser<K,V,Object> it = new Traverser<K,V,Object>(this);
3383     V v; U u;
3384     while ((v = it.advance()) != null) {
3385     if ((u = transformer.apply((K)it.nextKey, v)) != null)
3386     action.apply(u);
3387     }
3388     }
3389    
3390     /**
3391     * Returns a non-null result from applying the given search
3392     * function on each (key, value), or null if none.
3393     *
3394     * @param searchFunction a function returning a non-null
3395     * result on success, else null
3396     * @return a non-null result from applying the given search
3397     * function on each (key, value), or null if none
3398     */
3399     @SuppressWarnings("unchecked") public <U> U searchSequentially
3400     (BiFun<? super K, ? super V, ? extends U> searchFunction) {
3401     if (searchFunction == null) throw new NullPointerException();
3402     Traverser<K,V,Object> it = new Traverser<K,V,Object>(this);
3403     V v; U u;
3404     while ((v = it.advance()) != null) {
3405     if ((u = searchFunction.apply((K)it.nextKey, v)) != null)
3406     return u;
3407     }
3408     return null;
3409     }
3410    
3411     /**
3412     * Returns the result of accumulating the given transformation
3413     * of all (key, value) pairs using the given reducer to
3414     * combine values, or null if none.
3415     *
3416     * @param transformer a function returning the transformation
3417     * for an element, or null of there is no transformation (in
3418     * which case it is not combined).
3419     * @param reducer a commutative associative combining function
3420     * @return the result of accumulating the given transformation
3421     * of all (key, value) pairs
3422     */
3423     @SuppressWarnings("unchecked") public <U> U reduceSequentially
3424     (BiFun<? super K, ? super V, ? extends U> transformer,
3425     BiFun<? super U, ? super U, ? extends U> reducer) {
3426     if (transformer == null || reducer == null)
3427     throw new NullPointerException();
3428     Traverser<K,V,Object> it = new Traverser<K,V,Object>(this);
3429     U r = null, u; V v;
3430     while ((v = it.advance()) != null) {
3431     if ((u = transformer.apply((K)it.nextKey, v)) != null)
3432     r = (r == null) ? u : reducer.apply(r, u);
3433     }
3434     return r;
3435     }
3436    
3437     /**
3438     * Returns the result of accumulating the given transformation
3439     * of all (key, value) pairs using the given reducer to
3440     * combine values, and the given basis as an identity value.
3441     *
3442     * @param transformer a function returning the transformation
3443     * for an element
3444     * @param basis the identity (initial default value) for the reduction
3445     * @param reducer a commutative associative combining function
3446     * @return the result of accumulating the given transformation
3447     * of all (key, value) pairs
3448     */
3449     @SuppressWarnings("unchecked") public double reduceToDoubleSequentially
3450     (ObjectByObjectToDouble<? super K, ? super V> transformer,
3451     double basis,
3452     DoubleByDoubleToDouble reducer) {
3453     if (transformer == null || reducer == null)
3454     throw new NullPointerException();
3455     Traverser<K,V,Object> it = new Traverser<K,V,Object>(this);
3456     double r = basis; V v;
3457     while ((v = it.advance()) != null)
3458     r = reducer.apply(r, transformer.apply((K)it.nextKey, v));
3459     return r;
3460     }
3461    
3462     /**
3463     * Returns the result of accumulating the given transformation
3464     * of all (key, value) pairs using the given reducer to
3465     * combine values, and the given basis as an identity value.
3466     *
3467     * @param transformer a function returning the transformation
3468     * for an element
3469     * @param basis the identity (initial default value) for the reduction
3470     * @param reducer a commutative associative combining function
3471     * @return the result of accumulating the given transformation
3472     * of all (key, value) pairs
3473     */
3474     @SuppressWarnings("unchecked") public long reduceToLongSequentially
3475     (ObjectByObjectToLong<? super K, ? super V> transformer,
3476     long basis,
3477     LongByLongToLong reducer) {
3478     if (transformer == null || reducer == null)
3479     throw new NullPointerException();
3480     Traverser<K,V,Object> it = new Traverser<K,V,Object>(this);
3481     long r = basis; V v;
3482     while ((v = it.advance()) != null)
3483     r = reducer.apply(r, transformer.apply((K)it.nextKey, v));
3484     return r;
3485     }
3486    
3487     /**
3488     * Returns the result of accumulating the given transformation
3489     * of all (key, value) pairs using the given reducer to
3490     * combine values, and the given basis as an identity value.
3491     *
3492     * @param transformer a function returning the transformation
3493     * for an element
3494     * @param basis the identity (initial default value) for the reduction
3495     * @param reducer a commutative associative combining function
3496     * @return the result of accumulating the given transformation
3497     * of all (key, value) pairs
3498     */
3499     @SuppressWarnings("unchecked") public int reduceToIntSequentially
3500     (ObjectByObjectToInt<? super K, ? super V> transformer,
3501     int basis,
3502     IntByIntToInt reducer) {
3503     if (transformer == null || reducer == null)
3504     throw new NullPointerException();
3505     Traverser<K,V,Object> it = new Traverser<K,V,Object>(this);
3506     int r = basis; V v;
3507     while ((v = it.advance()) != null)
3508     r = reducer.apply(r, transformer.apply((K)it.nextKey, v));
3509     return r;
3510     }
3511    
3512     /**
3513     * Performs the given action for each key.
3514     *
3515     * @param action the action
3516     */
3517     @SuppressWarnings("unchecked") public void forEachKeySequentially
3518     (Action<K> action) {
3519     if (action == null) throw new NullPointerException();
3520     Traverser<K,V,Object> it = new Traverser<K,V,Object>(this);
3521     while (it.advance() != null)
3522     action.apply((K)it.nextKey);
3523     }
3524    
3525     /**
3526     * Performs the given action for each non-null transformation
3527     * of each key.
3528     *
3529     * @param transformer a function returning the transformation
3530     * for an element, or null of there is no transformation (in
3531     * which case the action is not applied).
3532     * @param action the action
3533     */
3534     @SuppressWarnings("unchecked") public <U> void forEachKeySequentially
3535     (Fun<? super K, ? extends U> transformer,
3536     Action<U> action) {
3537     if (transformer == null || action == null)
3538     throw new NullPointerException();
3539     Traverser<K,V,Object> it = new Traverser<K,V,Object>(this);
3540     U u;
3541     while (it.advance() != null) {
3542     if ((u = transformer.apply((K)it.nextKey)) != null)
3543     action.apply(u);
3544     }
3545     ForkJoinTasks.forEachKey
3546     (this, transformer, action).invoke();
3547     }
3548    
3549     /**
3550     * Returns a non-null result from applying the given search
3551     * function on each key, or null if none.
3552     *
3553     * @param searchFunction a function returning a non-null
3554     * result on success, else null
3555     * @return a non-null result from applying the given search
3556     * function on each key, or null if none
3557     */
3558     @SuppressWarnings("unchecked") public <U> U searchKeysSequentially
3559     (Fun<? super K, ? extends U> searchFunction) {
3560     Traverser<K,V,Object> it = new Traverser<K,V,Object>(this);
3561     U u;
3562     while (it.advance() != null) {
3563     if ((u = searchFunction.apply((K)it.nextKey)) != null)
3564     return u;
3565     }
3566     return null;
3567     }
3568    
3569     /**
3570     * Returns the result of accumulating all keys using the given
3571     * reducer to combine values, or null if none.
3572     *
3573     * @param reducer a commutative associative combining function
3574     * @return the result of accumulating all keys using the given
3575     * reducer to combine values, or null if none
3576     */
3577     @SuppressWarnings("unchecked") public K reduceKeysSequentially
3578     (BiFun<? super K, ? super K, ? extends K> reducer) {
3579     if (reducer == null) throw new NullPointerException();
3580     Traverser<K,V,Object> it = new Traverser<K,V,Object>(this);
3581     K r = null;
3582     while (it.advance() != null) {
3583     K u = (K)it.nextKey;
3584     r = (r == null) ? u : reducer.apply(r, u);
3585     }
3586     return r;
3587     }
3588    
3589     /**
3590     * Returns the result of accumulating the given transformation
3591     * of all keys using the given reducer to combine values, or
3592     * null if none.
3593     *
3594     * @param transformer a function returning the transformation
3595     * for an element, or null of there is no transformation (in
3596     * which case it is not combined).
3597     * @param reducer a commutative associative combining function
3598     * @return the result of accumulating the given transformation
3599     * of all keys
3600     */
3601     @SuppressWarnings("unchecked") public <U> U reduceKeysSequentially
3602     (Fun<? super K, ? extends U> transformer,
3603     BiFun<? super U, ? super U, ? extends U> reducer) {
3604     if (transformer == null || reducer == null)
3605     throw new NullPointerException();
3606     Traverser<K,V,Object> it = new Traverser<K,V,Object>(this);
3607     U r = null, u;
3608     while (it.advance() != null) {
3609     if ((u = transformer.apply((K)it.nextKey)) != null)
3610     r = (r == null) ? u : reducer.apply(r, u);
3611     }
3612     return r;
3613     }
3614    
3615     /**
3616     * Returns the result of accumulating the given transformation
3617     * of all keys using the given reducer to combine values, and
3618     * the given basis as an identity value.
3619     *
3620     * @param transformer a function returning the transformation
3621     * for an element
3622     * @param basis the identity (initial default value) for the reduction
3623     * @param reducer a commutative associative combining function
3624     * @return the result of accumulating the given transformation
3625     * of all keys
3626     */
3627     @SuppressWarnings("unchecked") public double reduceKeysToDoubleSequentially
3628     (ObjectToDouble<? super K> transformer,
3629     double basis,
3630     DoubleByDoubleToDouble reducer) {
3631     if (transformer == null || reducer == null)
3632     throw new NullPointerException();
3633     Traverser<K,V,Object> it = new Traverser<K,V,Object>(this);
3634     double r = basis;
3635     while (it.advance() != null)
3636     r = reducer.apply(r, transformer.apply((K)it.nextKey));
3637     return r;
3638     }
3639    
3640     /**
3641     * Returns the result of accumulating the given transformation
3642     * of all keys using the given reducer to combine values, and
3643     * the given basis as an identity value.
3644     *
3645     * @param transformer a function returning the transformation
3646     * for an element
3647     * @param basis the identity (initial default value) for the reduction
3648     * @param reducer a commutative associative combining function
3649     * @return the result of accumulating the given transformation
3650     * of all keys
3651     */
3652     @SuppressWarnings("unchecked") public long reduceKeysToLongSequentially
3653     (ObjectToLong<? super K> transformer,
3654     long basis,
3655     LongByLongToLong reducer) {
3656     if (transformer == null || reducer == null)
3657     throw new NullPointerException();
3658     Traverser<K,V,Object> it = new Traverser<K,V,Object>(this);
3659     long r = basis;
3660     while (it.advance() != null)
3661     r = reducer.apply(r, transformer.apply((K)it.nextKey));
3662     return r;
3663     }
3664    
3665     /**
3666     * Returns the result of accumulating the given transformation
3667     * of all keys using the given reducer to combine values, and
3668     * the given basis as an identity value.
3669     *
3670     * @param transformer a function returning the transformation
3671     * for an element
3672     * @param basis the identity (initial default value) for the reduction
3673     * @param reducer a commutative associative combining function
3674     * @return the result of accumulating the given transformation
3675     * of all keys
3676     */
3677     @SuppressWarnings("unchecked") public int reduceKeysToIntSequentially
3678     (ObjectToInt<? super K> transformer,
3679     int basis,
3680     IntByIntToInt reducer) {
3681     if (transformer == null || reducer == null)
3682     throw new NullPointerException();
3683     Traverser<K,V,Object> it = new Traverser<K,V,Object>(this);
3684     int r = basis;
3685     while (it.advance() != null)
3686     r = reducer.apply(r, transformer.apply((K)it.nextKey));
3687     return r;
3688     }
3689    
3690     /**
3691     * Performs the given action for each value.
3692     *
3693     * @param action the action
3694     */
3695     public void forEachValueSequentially(Action<V> action) {
3696     if (action == null) throw new NullPointerException();
3697     Traverser<K,V,Object> it = new Traverser<K,V,Object>(this);
3698     V v;
3699     while ((v = it.advance()) != null)
3700     action.apply(v);
3701     }
3702    
3703     /**
3704     * Performs the given action for each non-null transformation
3705     * of each value.
3706     *
3707     * @param transformer a function returning the transformation
3708     * for an element, or null of there is no transformation (in
3709     * which case the action is not applied).
3710     */
3711     public <U> void forEachValueSequentially
3712     (Fun<? super V, ? extends U> transformer,
3713     Action<U> action) {
3714     if (transformer == null || action == null)
3715     throw new NullPointerException();
3716     Traverser<K,V,Object> it = new Traverser<K,V,Object>(this);
3717     V v; U u;
3718     while ((v = it.advance()) != null) {
3719     if ((u = transformer.apply(v)) != null)
3720     action.apply(u);
3721     }
3722     }
3723    
3724     /**
3725     * Returns a non-null result from applying the given search
3726     * function on each value, or null if none.
3727     *
3728     * @param searchFunction a function returning a non-null
3729     * result on success, else null
3730     * @return a non-null result from applying the given search
3731     * function on each value, or null if none
3732     */
3733     public <U> U searchValuesSequentially
3734     (Fun<? super V, ? extends U> searchFunction) {
3735     if (searchFunction == null) throw new NullPointerException();
3736     Traverser<K,V,Object> it = new Traverser<K,V,Object>(this);
3737     V v; U u;
3738     while ((v = it.advance()) != null) {
3739     if ((u = searchFunction.apply(v)) != null)
3740     return u;
3741     }
3742     return null;
3743     }
3744    
3745     /**
3746     * Returns the result of accumulating all values using the
3747     * given reducer to combine values, or null if none.
3748     *
3749     * @param reducer a commutative associative combining function
3750     * @return the result of accumulating all values
3751     */
3752     public V reduceValuesSequentially
3753     (BiFun<? super V, ? super V, ? extends V> reducer) {
3754     if (reducer == null) throw new NullPointerException();
3755     Traverser<K,V,Object> it = new Traverser<K,V,Object>(this);
3756     V r = null; V v;
3757     while ((v = it.advance()) != null)
3758     r = (r == null) ? v : reducer.apply(r, v);
3759     return r;
3760     }
3761    
3762     /**
3763     * Returns the result of accumulating the given transformation
3764     * of all values using the given reducer to combine values, or
3765     * null if none.
3766     *
3767     * @param transformer a function returning the transformation
3768     * for an element, or null of there is no transformation (in
3769     * which case it is not combined).
3770     * @param reducer a commutative associative combining function
3771     * @return the result of accumulating the given transformation
3772     * of all values
3773     */
3774     public <U> U reduceValuesSequentially
3775     (Fun<? super V, ? extends U> transformer,
3776     BiFun<? super U, ? super U, ? extends U> reducer) {
3777     if (transformer == null || reducer == null)
3778     throw new NullPointerException();
3779     Traverser<K,V,Object> it = new Traverser<K,V,Object>(this);
3780     U r = null, u; V v;
3781     while ((v = it.advance()) != null) {
3782     if ((u = transformer.apply(v)) != null)
3783     r = (r == null) ? u : reducer.apply(r, u);
3784     }
3785     return r;
3786     }
3787    
3788     /**
3789     * Returns the result of accumulating the given transformation
3790     * of all values using the given reducer to combine values,
3791     * and the given basis as an identity value.
3792     *
3793     * @param transformer a function returning the transformation
3794     * for an element
3795     * @param basis the identity (initial default value) for the reduction
3796     * @param reducer a commutative associative combining function
3797     * @return the result of accumulating the given transformation
3798     * of all values
3799     */
3800     public double reduceValuesToDoubleSequentially
3801     (ObjectToDouble<? super V> transformer,
3802     double basis,
3803     DoubleByDoubleToDouble reducer) {
3804     if (transformer == null || reducer == null)
3805     throw new NullPointerException();
3806     Traverser<K,V,Object> it = new Traverser<K,V,Object>(this);
3807     double r = basis; V v;
3808     while ((v = it.advance()) != null)
3809     r = reducer.apply(r, transformer.apply(v));
3810     return r;
3811     }
3812    
3813     /**
3814     * Returns the result of accumulating the given transformation
3815     * of all values using the given reducer to combine values,
3816     * and the given basis as an identity value.
3817     *
3818     * @param transformer a function returning the transformation
3819     * for an element
3820     * @param basis the identity (initial default value) for the reduction
3821     * @param reducer a commutative associative combining function
3822     * @return the result of accumulating the given transformation
3823     * of all values
3824     */
3825     public long reduceValuesToLongSequentially
3826     (ObjectToLong<? super V> transformer,
3827     long basis,
3828     LongByLongToLong reducer) {
3829     if (transformer == null || reducer == null)
3830     throw new NullPointerException();
3831     Traverser<K,V,Object> it = new Traverser<K,V,Object>(this);
3832     long r = basis; V v;
3833     while ((v = it.advance()) != null)
3834     r = reducer.apply(r, transformer.apply(v));
3835     return r;
3836     }
3837    
3838     /**
3839     * Returns the result of accumulating the given transformation
3840     * of all values using the given reducer to combine values,
3841     * and the given basis as an identity value.
3842     *
3843     * @param transformer a function returning the transformation
3844     * for an element
3845     * @param basis the identity (initial default value) for the reduction
3846     * @param reducer a commutative associative combining function
3847     * @return the result of accumulating the given transformation
3848     * of all values
3849     */
3850     public int reduceValuesToIntSequentially
3851     (ObjectToInt<? super V> transformer,
3852     int basis,
3853     IntByIntToInt reducer) {
3854     if (transformer == null || reducer == null)
3855     throw new NullPointerException();
3856     Traverser<K,V,Object> it = new Traverser<K,V,Object>(this);
3857     int r = basis; V v;
3858     while ((v = it.advance()) != null)
3859     r = reducer.apply(r, transformer.apply(v));
3860     return r;
3861     }
3862    
3863     /**
3864     * Performs the given action for each entry.
3865     *
3866     * @param action the action
3867     */
3868     @SuppressWarnings("unchecked") public void forEachEntrySequentially
3869     (Action<Map.Entry<K,V>> action) {
3870     if (action == null) throw new NullPointerException();
3871     Traverser<K,V,Object> it = new Traverser<K,V,Object>(this);
3872     V v;
3873     while ((v = it.advance()) != null)
3874     action.apply(entryFor((K)it.nextKey, v));
3875     }
3876    
3877     /**
3878     * Performs the given action for each non-null transformation
3879     * of each entry.
3880     *
3881     * @param transformer a function returning the transformation
3882     * for an element, or null of there is no transformation (in
3883     * which case the action is not applied).
3884     * @param action the action
3885     */
3886     @SuppressWarnings("unchecked") public <U> void forEachEntrySequentially
3887     (Fun<Map.Entry<K,V>, ? extends U> transformer,
3888     Action<U> action) {
3889     if (transformer == null || action == null)
3890     throw new NullPointerException();
3891     Traverser<K,V,Object> it = new Traverser<K,V,Object>(this);
3892     V v; U u;
3893     while ((v = it.advance()) != null) {
3894     if ((u = transformer.apply(entryFor((K)it.nextKey, v))) != null)
3895     action.apply(u);
3896     }
3897     }
3898    
3899     /**
3900     * Returns a non-null result from applying the given search
3901     * function on each entry, or null if none.
3902     *
3903     * @param searchFunction a function returning a non-null
3904     * result on success, else null
3905     * @return a non-null result from applying the given search
3906     * function on each entry, or null if none
3907     */
3908     @SuppressWarnings("unchecked") public <U> U searchEntriesSequentially
3909     (Fun<Map.Entry<K,V>, ? extends U> searchFunction) {
3910     if (searchFunction == null) throw new NullPointerException();
3911     Traverser<K,V,Object> it = new Traverser<K,V,Object>(this);
3912     V v; U u;
3913     while ((v = it.advance()) != null) {
3914     if ((u = searchFunction.apply(entryFor((K)it.nextKey, v))) != null)
3915     return u;
3916     }
3917     return null;
3918     }
3919    
3920     /**
3921     * Returns the result of accumulating all entries using the
3922     * given reducer to combine values, or null if none.
3923     *
3924     * @param reducer a commutative associative combining function
3925     * @return the result of accumulating all entries
3926     */
3927     @SuppressWarnings("unchecked") public Map.Entry<K,V> reduceEntriesSequentially
3928     (BiFun<Map.Entry<K,V>, Map.Entry<K,V>, ? extends Map.Entry<K,V>> reducer) {
3929     if (reducer == null) throw new NullPointerException();
3930     Traverser<K,V,Object> it = new Traverser<K,V,Object>(this);
3931     Map.Entry<K,V> r = null; V v;
3932     while ((v = it.advance()) != null) {
3933     Map.Entry<K,V> u = entryFor((K)it.nextKey, v);
3934     r = (r == null) ? u : reducer.apply(r, u);
3935     }
3936     return r;
3937     }
3938    
3939     /**
3940     * Returns the result of accumulating the given transformation
3941     * of all entries using the given reducer to combine values,
3942     * or null if none.
3943     *
3944     * @param transformer a function returning the transformation
3945     * for an element, or null of there is no transformation (in
3946     * which case it is not combined).
3947     * @param reducer a commutative associative combining function
3948     * @return the result of accumulating the given transformation
3949     * of all entries
3950     */
3951     @SuppressWarnings("unchecked") public <U> U reduceEntriesSequentially
3952     (Fun<Map.Entry<K,V>, ? extends U> transformer,
3953     BiFun<? super U, ? super U, ? extends U> reducer) {
3954     if (transformer == null || reducer == null)
3955     throw new NullPointerException();
3956     Traverser<K,V,Object> it = new Traverser<K,V,Object>(this);
3957     U r = null, u; V v;
3958     while ((v = it.advance()) != null) {
3959     if ((u = transformer.apply(entryFor((K)it.nextKey, v))) != null)
3960     r = (r == null) ? u : reducer.apply(r, u);
3961     }
3962     return r;
3963     }
3964    
3965     /**
3966     * Returns the result of accumulating the given transformation
3967     * of all entries using the given reducer to combine values,
3968     * and the given basis as an identity value.
3969     *
3970     * @param transformer a function returning the transformation
3971     * for an element
3972     * @param basis the identity (initial default value) for the reduction
3973     * @param reducer a commutative associative combining function
3974     * @return the result of accumulating the given transformation
3975     * of all entries
3976     */
3977     @SuppressWarnings("unchecked") public double reduceEntriesToDoubleSequentially
3978     (ObjectToDouble<Map.Entry<K,V>> transformer,
3979     double basis,
3980     DoubleByDoubleToDouble reducer) {
3981     if (transformer == null || reducer == null)
3982     throw new NullPointerException();
3983     Traverser<K,V,Object> it = new Traverser<K,V,Object>(this);
3984     double r = basis; V v;
3985     while ((v = it.advance()) != null)
3986     r = reducer.apply(r, transformer.apply(entryFor((K)it.nextKey, v)));
3987     return r;
3988     }
3989    
3990     /**
3991     * Returns the result of accumulating the given transformation
3992     * of all entries using the given reducer to combine values,
3993     * and the given basis as an identity value.
3994     *
3995     * @param transformer a function returning the transformation
3996     * for an element
3997     * @param basis the identity (initial default value) for the reduction
3998     * @param reducer a commutative associative combining function
3999     * @return the result of accumulating the given transformation
4000     * of all entries
4001     */
4002     @SuppressWarnings("unchecked") public long reduceEntriesToLongSequentially
4003     (ObjectToLong<Map.Entry<K,V>> transformer,
4004     long basis,
4005     LongByLongToLong reducer) {
4006     if (transformer == null || reducer == null)
4007     throw new NullPointerException();
4008     Traverser<K,V,Object> it = new Traverser<K,V,Object>(this);
4009     long r = basis; V v;
4010     while ((v = it.advance()) != null)
4011     r = reducer.apply(r, transformer.apply(entryFor((K)it.nextKey, v)));
4012     return r;
4013     }
4014    
4015     /**
4016     * Returns the result of accumulating the given transformation
4017     * of all entries using the given reducer to combine values,
4018     * and the given basis as an identity value.
4019     *
4020     * @param transformer a function returning the transformation
4021     * for an element
4022     * @param basis the identity (initial default value) for the reduction
4023     * @param reducer a commutative associative combining function
4024     * @return the result of accumulating the given transformation
4025     * of all entries
4026     */
4027     @SuppressWarnings("unchecked") public int reduceEntriesToIntSequentially
4028     (ObjectToInt<Map.Entry<K,V>> transformer,
4029     int basis,
4030     IntByIntToInt reducer) {
4031     if (transformer == null || reducer == null)
4032     throw new NullPointerException();
4033     Traverser<K,V,Object> it = new Traverser<K,V,Object>(this);
4034     int r = basis; V v;
4035     while ((v = it.advance()) != null)
4036     r = reducer.apply(r, transformer.apply(entryFor((K)it.nextKey, v)));
4037     return r;
4038     }
4039    
4040     // Parallel bulk operations
4041    
4042     /**
4043     * Performs the given action for each (key, value).
4044     *
4045     * @param action the action
4046     */
4047     public void forEachInParallel(BiAction<K,V> action) {
4048     ForkJoinTasks.forEach
4049     (this, action).invoke();
4050     }
4051    
4052     /**
4053     * Performs the given action for each non-null transformation
4054     * of each (key, value).
4055     *
4056     * @param transformer a function returning the transformation
4057     * for an element, or null of there is no transformation (in
4058     * which case the action is not applied).
4059     * @param action the action
4060     */
4061     public <U> void forEachInParallel
4062     (BiFun<? super K, ? super V, ? extends U> transformer,
4063     Action<U> action) {
4064     ForkJoinTasks.forEach
4065     (this, transformer, action).invoke();
4066     }
4067    
4068     /**
4069     * Returns a non-null result from applying the given search
4070     * function on each (key, value), or null if none. Upon
4071     * success, further element processing is suppressed and the
4072     * results of any other parallel invocations of the search
4073     * function are ignored.
4074     *
4075     * @param searchFunction a function returning a non-null
4076     * result on success, else null
4077     * @return a non-null result from applying the given search
4078     * function on each (key, value), or null if none
4079     */
4080     public <U> U searchInParallel
4081     (BiFun<? super K, ? super V, ? extends U> searchFunction) {
4082     return ForkJoinTasks.search
4083     (this, searchFunction).invoke();
4084     }
4085    
4086     /**
4087     * Returns the result of accumulating the given transformation
4088     * of all (key, value) pairs using the given reducer to
4089     * combine values, or null if none.
4090     *
4091     * @param transformer a function returning the transformation
4092     * for an element, or null of there is no transformation (in
4093     * which case it is not combined).
4094     * @param reducer a commutative associative combining function
4095     * @return the result of accumulating the given transformation
4096     * of all (key, value) pairs
4097     */
4098     public <U> U reduceInParallel
4099     (BiFun<? super K, ? super V, ? extends U> transformer,
4100     BiFun<? super U, ? super U, ? extends U> reducer) {
4101     return ForkJoinTasks.reduce
4102     (this, transformer, reducer).invoke();
4103     }
4104    
4105     /**
4106     * Returns the result of accumulating the given transformation
4107     * of all (key, value) pairs using the given reducer to
4108     * combine values, and the given basis as an identity value.
4109     *
4110     * @param transformer a function returning the transformation
4111     * for an element
4112     * @param basis the identity (initial default value) for the reduction
4113     * @param reducer a commutative associative combining function
4114     * @return the result of accumulating the given transformation
4115     * of all (key, value) pairs
4116     */
4117     public double reduceToDoubleInParallel
4118     (ObjectByObjectToDouble<? super K, ? super V> transformer,
4119     double basis,
4120     DoubleByDoubleToDouble reducer) {
4121     return ForkJoinTasks.reduceToDouble
4122     (this, transformer, basis, reducer).invoke();
4123     }
4124    
4125     /**
4126     * Returns the result of accumulating the given transformation
4127     * of all (key, value) pairs using the given reducer to
4128     * combine values, and the given basis as an identity value.
4129     *
4130     * @param transformer a function returning the transformation
4131     * for an element
4132     * @param basis the identity (initial default value) for the reduction
4133     * @param reducer a commutative associative combining function
4134     * @return the result of accumulating the given transformation
4135     * of all (key, value) pairs
4136     */
4137     public long reduceToLongInParallel
4138     (ObjectByObjectToLong<? super K, ? super V> transformer,
4139     long basis,
4140     LongByLongToLong reducer) {
4141     return ForkJoinTasks.reduceToLong
4142     (this, transformer, basis, reducer).invoke();
4143     }
4144    
4145     /**
4146     * Returns the result of accumulating the given transformation
4147     * of all (key, value) pairs using the given reducer to
4148     * combine values, and the given basis as an identity value.
4149     *
4150     * @param transformer a function returning the transformation
4151     * for an element
4152     * @param basis the identity (initial default value) for the reduction
4153     * @param reducer a commutative associative combining function
4154     * @return the result of accumulating the given transformation
4155     * of all (key, value) pairs
4156     */
4157     public int reduceToIntInParallel
4158     (ObjectByObjectToInt<? super K, ? super V> transformer,
4159     int basis,
4160     IntByIntToInt reducer) {
4161     return ForkJoinTasks.reduceToInt
4162     (this, transformer, basis, reducer).invoke();
4163     }
4164    
4165     /**
4166     * Performs the given action for each key.
4167     *
4168     * @param action the action
4169     */
4170     public void forEachKeyInParallel(Action<K> action) {
4171     ForkJoinTasks.forEachKey
4172     (this, action).invoke();
4173     }
4174    
4175     /**
4176     * Performs the given action for each non-null transformation
4177     * of each key.
4178     *
4179     * @param transformer a function returning the transformation
4180     * for an element, or null of there is no transformation (in
4181     * which case the action is not applied).
4182     * @param action the action
4183     */
4184     public <U> void forEachKeyInParallel
4185     (Fun<? super K, ? extends U> transformer,
4186     Action<U> action) {
4187     ForkJoinTasks.forEachKey
4188     (this, transformer, action).invoke();
4189     }
4190    
4191     /**
4192     * Returns a non-null result from applying the given search
4193     * function on each key, or null if none. Upon success,
4194     * further element processing is suppressed and the results of
4195     * any other parallel invocations of the search function are
4196     * ignored.
4197     *
4198     * @param searchFunction a function returning a non-null
4199     * result on success, else null
4200     * @return a non-null result from applying the given search
4201     * function on each key, or null if none
4202     */
4203     public <U> U searchKeysInParallel
4204     (Fun<? super K, ? extends U> searchFunction) {
4205     return ForkJoinTasks.searchKeys
4206     (this, searchFunction).invoke();
4207     }
4208    
4209     /**
4210     * Returns the result of accumulating all keys using the given
4211     * reducer to combine values, or null if none.
4212     *
4213     * @param reducer a commutative associative combining function
4214     * @return the result of accumulating all keys using the given
4215     * reducer to combine values, or null if none
4216     */
4217     public K reduceKeysInParallel
4218     (BiFun<? super K, ? super K, ? extends K> reducer) {
4219     return ForkJoinTasks.reduceKeys
4220     (this, reducer).invoke();
4221     }
4222    
4223     /**
4224     * Returns the result of accumulating the given transformation
4225     * of all keys using the given reducer to combine values, or
4226     * null if none.
4227     *
4228     * @param transformer a function returning the transformation
4229     * for an element, or null of there is no transformation (in
4230     * which case it is not combined).
4231     * @param reducer a commutative associative combining function
4232     * @return the result of accumulating the given transformation
4233     * of all keys
4234     */
4235     public <U> U reduceKeysInParallel
4236     (Fun<? super K, ? extends U> transformer,
4237     BiFun<? super U, ? super U, ? extends U> reducer) {
4238     return ForkJoinTasks.reduceKeys
4239     (this, transformer, reducer).invoke();
4240     }
4241    
4242     /**
4243     * Returns the result of accumulating the given transformation
4244     * of all keys using the given reducer to combine values, and
4245     * the given basis as an identity value.
4246     *
4247     * @param transformer a function returning the transformation
4248     * for an element
4249     * @param basis the identity (initial default value) for the reduction
4250     * @param reducer a commutative associative combining function
4251     * @return the result of accumulating the given transformation
4252     * of all keys
4253     */
4254     public double reduceKeysToDoubleInParallel
4255     (ObjectToDouble<? super K> transformer,
4256     double basis,
4257     DoubleByDoubleToDouble reducer) {
4258     return ForkJoinTasks.reduceKeysToDouble
4259     (this, transformer, basis, reducer).invoke();
4260     }
4261    
4262     /**
4263     * Returns the result of accumulating the given transformation
4264     * of all keys using the given reducer to combine values, and
4265     * the given basis as an identity value.
4266     *
4267     * @param transformer a function returning the transformation
4268     * for an element
4269     * @param basis the identity (initial default value) for the reduction
4270     * @param reducer a commutative associative combining function
4271     * @return the result of accumulating the given transformation
4272     * of all keys
4273     */
4274     public long reduceKeysToLongInParallel
4275     (ObjectToLong<? super K> transformer,
4276     long basis,
4277     LongByLongToLong reducer) {
4278     return ForkJoinTasks.reduceKeysToLong
4279     (this, transformer, basis, reducer).invoke();
4280     }
4281    
4282     /**
4283     * Returns the result of accumulating the given transformation
4284     * of all keys using the given reducer to combine values, and
4285     * the given basis as an identity value.
4286     *
4287     * @param transformer a function returning the transformation
4288     * for an element
4289     * @param basis the identity (initial default value) for the reduction
4290     * @param reducer a commutative associative combining function
4291     * @return the result of accumulating the given transformation
4292     * of all keys
4293     */
4294     public int reduceKeysToIntInParallel
4295     (ObjectToInt<? super K> transformer,
4296     int basis,
4297     IntByIntToInt reducer) {
4298     return ForkJoinTasks.reduceKeysToInt
4299     (this, transformer, basis, reducer).invoke();
4300     }
4301    
4302     /**
4303     * Performs the given action for each value.
4304     *
4305     * @param action the action
4306     */
4307     public void forEachValueInParallel(Action<V> action) {
4308     ForkJoinTasks.forEachValue
4309     (this, action).invoke();
4310     }
4311    
4312     /**
4313     * Performs the given action for each non-null transformation
4314     * of each value.
4315     *
4316     * @param transformer a function returning the transformation
4317     * for an element, or null of there is no transformation (in
4318     * which case the action is not applied).
4319     */
4320     public <U> void forEachValueInParallel
4321     (Fun<? super V, ? extends U> transformer,
4322     Action<U> action) {
4323     ForkJoinTasks.forEachValue
4324     (this, transformer, action).invoke();
4325     }
4326    
4327     /**
4328     * Returns a non-null result from applying the given search
4329     * function on each value, or null if none. Upon success,
4330     * further element processing is suppressed and the results of
4331     * any other parallel invocations of the search function are
4332     * ignored.
4333     *
4334     * @param searchFunction a function returning a non-null
4335     * result on success, else null
4336     * @return a non-null result from applying the given search
4337     * function on each value, or null if none
4338     */
4339     public <U> U searchValuesInParallel
4340     (Fun<? super V, ? extends U> searchFunction) {
4341     return ForkJoinTasks.searchValues
4342     (this, searchFunction).invoke();
4343     }
4344    
4345     /**
4346     * Returns the result of accumulating all values using the
4347     * given reducer to combine values, or null if none.
4348     *
4349     * @param reducer a commutative associative combining function
4350     * @return the result of accumulating all values
4351     */
4352     public V reduceValuesInParallel
4353     (BiFun<? super V, ? super V, ? extends V> reducer) {
4354     return ForkJoinTasks.reduceValues
4355     (this, reducer).invoke();
4356     }
4357    
4358     /**
4359     * Returns the result of accumulating the given transformation
4360     * of all values using the given reducer to combine values, or
4361     * null if none.
4362     *
4363     * @param transformer a function returning the transformation
4364     * for an element, or null of there is no transformation (in
4365     * which case it is not combined).
4366     * @param reducer a commutative associative combining function
4367     * @return the result of accumulating the given transformation
4368     * of all values
4369     */
4370     public <U> U reduceValuesInParallel
4371     (Fun<? super V, ? extends U> transformer,
4372     BiFun<? super U, ? super U, ? extends U> reducer) {
4373     return ForkJoinTasks.reduceValues
4374     (this, transformer, reducer).invoke();
4375     }
4376    
4377     /**
4378     * Returns the result of accumulating the given transformation
4379     * of all values using the given reducer to combine values,
4380     * and the given basis as an identity value.
4381     *
4382     * @param transformer a function returning the transformation
4383     * for an element
4384     * @param basis the identity (initial default value) for the reduction
4385     * @param reducer a commutative associative combining function
4386     * @return the result of accumulating the given transformation
4387     * of all values
4388     */
4389     public double reduceValuesToDoubleInParallel
4390     (ObjectToDouble<? super V> transformer,
4391     double basis,
4392     DoubleByDoubleToDouble reducer) {
4393     return ForkJoinTasks.reduceValuesToDouble
4394     (this, transformer, basis, reducer).invoke();
4395     }
4396    
4397     /**
4398     * Returns the result of accumulating the given transformation
4399     * of all values using the given reducer to combine values,
4400     * and the given basis as an identity value.
4401     *
4402     * @param transformer a function returning the transformation
4403     * for an element
4404     * @param basis the identity (initial default value) for the reduction
4405     * @param reducer a commutative associative combining function
4406     * @return the result of accumulating the given transformation
4407     * of all values
4408     */
4409     public long reduceValuesToLongInParallel
4410     (ObjectToLong<? super V> transformer,
4411     long basis,
4412     LongByLongToLong reducer) {
4413     return ForkJoinTasks.reduceValuesToLong
4414     (this, transformer, basis, reducer).invoke();
4415     }
4416    
4417     /**
4418     * Returns the result of accumulating the given transformation
4419     * of all values using the given reducer to combine values,
4420     * and the given basis as an identity value.
4421     *
4422     * @param transformer a function returning the transformation
4423     * for an element
4424     * @param basis the identity (initial default value) for the reduction
4425     * @param reducer a commutative associative combining function
4426     * @return the result of accumulating the given transformation
4427     * of all values
4428     */
4429     public int reduceValuesToIntInParallel
4430     (ObjectToInt<? super V> transformer,
4431     int basis,
4432     IntByIntToInt reducer) {
4433     return ForkJoinTasks.reduceValuesToInt
4434     (this, transformer, basis, reducer).invoke();
4435     }
4436    
4437     /**
4438     * Performs the given action for each entry.
4439     *
4440     * @param action the action
4441     */
4442     public void forEachEntryInParallel(Action<Map.Entry<K,V>> action) {
4443     ForkJoinTasks.forEachEntry
4444     (this, action).invoke();
4445     }
4446    
4447     /**
4448     * Performs the given action for each non-null transformation
4449     * of each entry.
4450     *
4451     * @param transformer a function returning the transformation
4452     * for an element, or null of there is no transformation (in
4453     * which case the action is not applied).
4454     * @param action the action
4455     */
4456     public <U> void forEachEntryInParallel
4457     (Fun<Map.Entry<K,V>, ? extends U> transformer,
4458     Action<U> action) {
4459     ForkJoinTasks.forEachEntry
4460     (this, transformer, action).invoke();
4461     }
4462    
4463     /**
4464     * Returns a non-null result from applying the given search
4465     * function on each entry, or null if none. Upon success,
4466     * further element processing is suppressed and the results of
4467     * any other parallel invocations of the search function are
4468     * ignored.
4469     *
4470     * @param searchFunction a function returning a non-null
4471     * result on success, else null
4472     * @return a non-null result from applying the given search
4473     * function on each entry, or null if none
4474     */
4475     public <U> U searchEntriesInParallel
4476     (Fun<Map.Entry<K,V>, ? extends U> searchFunction) {
4477     return ForkJoinTasks.searchEntries
4478     (this, searchFunction).invoke();
4479     }
4480    
4481     /**
4482     * Returns the result of accumulating all entries using the
4483     * given reducer to combine values, or null if none.
4484     *
4485     * @param reducer a commutative associative combining function
4486     * @return the result of accumulating all entries
4487     */
4488     public Map.Entry<K,V> reduceEntriesInParallel
4489     (BiFun<Map.Entry<K,V>, Map.Entry<K,V>, ? extends Map.Entry<K,V>> reducer) {
4490     return ForkJoinTasks.reduceEntries
4491     (this, reducer).invoke();
4492     }
4493    
4494     /**
4495     * Returns the result of accumulating the given transformation
4496     * of all entries using the given reducer to combine values,
4497     * or null if none.
4498     *
4499     * @param transformer a function returning the transformation
4500     * for an element, or null of there is no transformation (in
4501     * which case it is not combined).
4502     * @param reducer a commutative associative combining function
4503     * @return the result of accumulating the given transformation
4504     * of all entries
4505     */
4506     public <U> U reduceEntriesInParallel
4507     (Fun<Map.Entry<K,V>, ? extends U> transformer,
4508     BiFun<? super U, ? super U, ? extends U> reducer) {
4509     return ForkJoinTasks.reduceEntries
4510     (this, transformer, reducer).invoke();
4511     }
4512    
4513     /**
4514     * Returns the result of accumulating the given transformation
4515     * of all entries using the given reducer to combine values,
4516     * and the given basis as an identity value.
4517     *
4518     * @param transformer a function returning the transformation
4519     * for an element
4520     * @param basis the identity (initial default value) for the reduction
4521     * @param reducer a commutative associative combining function
4522     * @return the result of accumulating the given transformation
4523     * of all entries
4524     */
4525     public double reduceEntriesToDoubleInParallel
4526     (ObjectToDouble<Map.Entry<K,V>> transformer,
4527     double basis,
4528     DoubleByDoubleToDouble reducer) {
4529     return ForkJoinTasks.reduceEntriesToDouble
4530     (this, transformer, basis, reducer).invoke();
4531     }
4532    
4533     /**
4534     * Returns the result of accumulating the given transformation
4535     * of all entries using the given reducer to combine values,
4536     * and the given basis as an identity value.
4537     *
4538     * @param transformer a function returning the transformation
4539     * for an element
4540     * @param basis the identity (initial default value) for the reduction
4541     * @param reducer a commutative associative combining function
4542     * @return the result of accumulating the given transformation
4543     * of all entries
4544     */
4545     public long reduceEntriesToLongInParallel
4546     (ObjectToLong<Map.Entry<K,V>> transformer,
4547     long basis,
4548     LongByLongToLong reducer) {
4549     return ForkJoinTasks.reduceEntriesToLong
4550     (this, transformer, basis, reducer).invoke();
4551     }
4552    
4553     /**
4554     * Returns the result of accumulating the given transformation
4555     * of all entries using the given reducer to combine values,
4556     * and the given basis as an identity value.
4557     *
4558     * @param transformer a function returning the transformation
4559     * for an element
4560     * @param basis the identity (initial default value) for the reduction
4561     * @param reducer a commutative associative combining function
4562     * @return the result of accumulating the given transformation
4563     * of all entries
4564     */
4565     public int reduceEntriesToIntInParallel
4566     (ObjectToInt<Map.Entry<K,V>> transformer,
4567     int basis,
4568     IntByIntToInt reducer) {
4569     return ForkJoinTasks.reduceEntriesToInt
4570     (this, transformer, basis, reducer).invoke();
4571     }
4572    
4573    
4574     /* ----------------Views -------------- */
4575    
4576     /**
4577     * Base class for views.
4578     */
4579 dl 1.5 static abstract class CHMView<K, V> implements java.io.Serializable {
4580     private static final long serialVersionUID = 7249069246763182397L;
4581 dl 1.1 final ConcurrentHashMap<K, V> map;
4582     CHMView(ConcurrentHashMap<K, V> map) { this.map = map; }
4583    
4584     /**
4585     * Returns the map backing this view.
4586     *
4587     * @return the map backing this view
4588     */
4589     public ConcurrentHashMap<K,V> getMap() { return map; }
4590    
4591     public final int size() { return map.size(); }
4592     public final boolean isEmpty() { return map.isEmpty(); }
4593     public final void clear() { map.clear(); }
4594    
4595     // implementations below rely on concrete classes supplying these
4596     abstract public Iterator<?> iterator();
4597     abstract public boolean contains(Object o);
4598     abstract public boolean remove(Object o);
4599    
4600     private static final String oomeMsg = "Required array size too large";
4601    
4602     public final Object[] toArray() {
4603     long sz = map.mappingCount();
4604     if (sz > (long)(MAX_ARRAY_SIZE))
4605     throw new OutOfMemoryError(oomeMsg);
4606     int n = (int)sz;
4607     Object[] r = new Object[n];
4608     int i = 0;
4609     Iterator<?> it = iterator();
4610     while (it.hasNext()) {
4611     if (i == n) {
4612     if (n >= MAX_ARRAY_SIZE)
4613     throw new OutOfMemoryError(oomeMsg);
4614     if (n >= MAX_ARRAY_SIZE - (MAX_ARRAY_SIZE >>> 1) - 1)
4615     n = MAX_ARRAY_SIZE;
4616     else
4617     n += (n >>> 1) + 1;
4618     r = Arrays.copyOf(r, n);
4619     }
4620     r[i++] = it.next();
4621     }
4622     return (i == n) ? r : Arrays.copyOf(r, i);
4623     }
4624    
4625     @SuppressWarnings("unchecked") public final <T> T[] toArray(T[] a) {
4626     long sz = map.mappingCount();
4627     if (sz > (long)(MAX_ARRAY_SIZE))
4628     throw new OutOfMemoryError(oomeMsg);
4629     int m = (int)sz;
4630     T[] r = (a.length >= m) ? a :
4631     (T[])java.lang.reflect.Array
4632     .newInstance(a.getClass().getComponentType(), m);
4633     int n = r.length;
4634     int i = 0;
4635     Iterator<?> it = iterator();
4636     while (it.hasNext()) {
4637     if (i == n) {
4638     if (n >= MAX_ARRAY_SIZE)
4639     throw new OutOfMemoryError(oomeMsg);
4640     if (n >= MAX_ARRAY_SIZE - (MAX_ARRAY_SIZE >>> 1) - 1)
4641     n = MAX_ARRAY_SIZE;
4642     else
4643     n += (n >>> 1) + 1;
4644     r = Arrays.copyOf(r, n);
4645     }
4646     r[i++] = (T)it.next();
4647     }
4648     if (a == r && i < n) {
4649     r[i] = null; // null-terminate
4650     return r;
4651     }
4652     return (i == n) ? r : Arrays.copyOf(r, i);
4653     }
4654    
4655     public final int hashCode() {
4656     int h = 0;
4657     for (Iterator<?> it = iterator(); it.hasNext();)
4658     h += it.next().hashCode();
4659     return h;
4660     }
4661    
4662     public final String toString() {
4663     StringBuilder sb = new StringBuilder();
4664     sb.append('[');
4665     Iterator<?> it = iterator();
4666     if (it.hasNext()) {
4667     for (;;) {
4668     Object e = it.next();
4669     sb.append(e == this ? "(this Collection)" : e);
4670     if (!it.hasNext())
4671     break;
4672     sb.append(',').append(' ');
4673     }
4674     }
4675     return sb.append(']').toString();
4676     }
4677    
4678     public final boolean containsAll(Collection<?> c) {
4679     if (c != this) {
4680     for (Iterator<?> it = c.iterator(); it.hasNext();) {
4681     Object e = it.next();
4682     if (e == null || !contains(e))
4683     return false;
4684     }
4685     }
4686     return true;
4687     }
4688    
4689     public final boolean removeAll(Collection<?> c) {
4690     boolean modified = false;
4691     for (Iterator<?> it = iterator(); it.hasNext();) {
4692     if (c.contains(it.next())) {
4693     it.remove();
4694     modified = true;
4695     }
4696     }
4697     return modified;
4698     }
4699    
4700     public final boolean retainAll(Collection<?> c) {
4701     boolean modified = false;
4702     for (Iterator<?> it = iterator(); it.hasNext();) {
4703     if (!c.contains(it.next())) {
4704     it.remove();
4705     modified = true;
4706     }
4707     }
4708     return modified;
4709     }
4710    
4711     }
4712    
4713     /**
4714     * A view of a ConcurrentHashMap as a {@link Set} of keys, in
4715     * which additions may optionally be enabled by mapping to a
4716     * common value. This class cannot be directly instantiated. See
4717     * {@link #keySet}, {@link #keySet(Object)}, {@link #newKeySet()},
4718     * {@link #newKeySet(int)}.
4719     */
4720     public static class KeySetView<K,V> extends CHMView<K,V>
4721     implements Set<K>, java.io.Serializable {
4722     private static final long serialVersionUID = 7249069246763182397L;
4723     private final V value;
4724     KeySetView(ConcurrentHashMap<K, V> map, V value) { // non-public
4725     super(map);
4726     this.value = value;
4727     }
4728    
4729     /**
4730     * Returns the default mapped value for additions,
4731     * or {@code null} if additions are not supported.
4732     *
4733     * @return the default mapped value for additions, or {@code null}
4734     * if not supported.
4735     */
4736     public V getMappedValue() { return value; }
4737    
4738     // implement Set API
4739    
4740     public boolean contains(Object o) { return map.containsKey(o); }
4741     public boolean remove(Object o) { return map.remove(o) != null; }
4742    
4743     /**
4744     * Returns a "weakly consistent" iterator that will never
4745     * throw {@link ConcurrentModificationException}, and
4746     * guarantees to traverse elements as they existed upon
4747     * construction of the iterator, and may (but is not
4748     * guaranteed to) reflect any modifications subsequent to
4749     * construction.
4750     *
4751     * @return an iterator over the keys of this map
4752     */
4753     public Iterator<K> iterator() { return new KeyIterator<K,V>(map); }
4754     public boolean add(K e) {
4755     V v;
4756     if ((v = value) == null)
4757     throw new UnsupportedOperationException();
4758     if (e == null)
4759     throw new NullPointerException();
4760     return map.internalPut(e, v, true) == null;
4761     }
4762     public boolean addAll(Collection<? extends K> c) {
4763     boolean added = false;
4764     V v;
4765     if ((v = value) == null)
4766     throw new UnsupportedOperationException();
4767     for (K e : c) {
4768     if (e == null)
4769     throw new NullPointerException();
4770     if (map.internalPut(e, v, true) == null)
4771     added = true;
4772     }
4773     return added;
4774     }
4775     public boolean equals(Object o) {
4776     Set<?> c;
4777     return ((o instanceof Set) &&
4778     ((c = (Set<?>)o) == this ||
4779     (containsAll(c) && c.containsAll(this))));
4780     }
4781     }
4782    
4783     /**
4784     * A view of a ConcurrentHashMap as a {@link Collection} of
4785     * values, in which additions are disabled. This class cannot be
4786     * directly instantiated. See {@link #values},
4787     *
4788     * <p>The view's {@code iterator} is a "weakly consistent" iterator
4789     * that will never throw {@link ConcurrentModificationException},
4790     * and guarantees to traverse elements as they existed upon
4791     * construction of the iterator, and may (but is not guaranteed to)
4792     * reflect any modifications subsequent to construction.
4793     */
4794     public static final class ValuesView<K,V> extends CHMView<K,V>
4795     implements Collection<V> {
4796     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     * {@link #entrySet}.
4837     */
4838     public static final class EntrySetView<K,V> extends CHMView<K,V>
4839     implements Set<Map.Entry<K,V>> {
4840     EntrySetView(ConcurrentHashMap<K, V> map) { super(map); }
4841     public final boolean contains(Object o) {
4842     Object k, v, r; Map.Entry<?,?> e;
4843     return ((o instanceof Map.Entry) &&
4844     (k = (e = (Map.Entry<?,?>)o).getKey()) != null &&
4845     (r = map.get(k)) != null &&
4846     (v = e.getValue()) != null &&
4847     (v == r || v.equals(r)));
4848     }
4849     public final boolean remove(Object o) {
4850     Object k, v; Map.Entry<?,?> e;
4851     return ((o instanceof Map.Entry) &&
4852     (k = (e = (Map.Entry<?,?>)o).getKey()) != null &&
4853     (v = e.getValue()) != null &&
4854     map.remove(k, v));
4855     }
4856    
4857     /**
4858     * Returns a "weakly consistent" iterator that will never
4859     * throw {@link ConcurrentModificationException}, and
4860     * guarantees to traverse elements as they existed upon
4861     * construction of the iterator, and may (but is not
4862     * guaranteed to) reflect any modifications subsequent to
4863     * construction.
4864     *
4865     * @return an iterator over the entries of this map
4866     */
4867     public final Iterator<Map.Entry<K,V>> iterator() {
4868     return new EntryIterator<K,V>(map);
4869     }
4870    
4871     public final boolean add(Entry<K,V> e) {
4872     K key = e.getKey();
4873     V value = e.getValue();
4874     if (key == null || value == null)
4875     throw new NullPointerException();
4876     return map.internalPut(key, value, false) == null;
4877     }
4878     public final boolean addAll(Collection<? extends Entry<K,V>> c) {
4879     boolean added = false;
4880     for (Entry<K,V> e : c) {
4881     if (add(e))
4882     added = true;
4883     }
4884     return added;
4885     }
4886     public boolean equals(Object o) {
4887     Set<?> c;
4888     return ((o instanceof Set) &&
4889     ((c = (Set<?>)o) == this ||
4890     (containsAll(c) && c.containsAll(this))));
4891     }
4892     }
4893    
4894     // ---------------------------------------------------------------------
4895    
4896     /**
4897     * Predefined tasks for performing bulk parallel operations on
4898     * ConcurrentHashMaps. These tasks follow the forms and rules used
4899     * for bulk operations. Each method has the same name, but returns
4900     * a task rather than invoking it. These methods may be useful in
4901     * custom applications such as submitting a task without waiting
4902     * for completion, using a custom pool, or combining with other
4903     * tasks.
4904     */
4905     public static class ForkJoinTasks {
4906     private ForkJoinTasks() {}
4907    
4908     /**
4909     * Returns a task that when invoked, performs the given
4910     * action for each (key, value)
4911     *
4912     * @param map the map
4913     * @param action the action
4914     * @return the task
4915     */
4916     public static <K,V> ForkJoinTask<Void> forEach
4917     (ConcurrentHashMap<K,V> map,
4918     BiAction<K,V> action) {
4919     if (action == null) throw new NullPointerException();
4920     return new ForEachMappingTask<K,V>(map, null, -1, action);
4921     }
4922    
4923     /**
4924     * Returns a task that when invoked, performs the given
4925     * action for each non-null transformation of each (key, value)
4926     *
4927     * @param map the map
4928     * @param transformer a function returning the transformation
4929     * for an element, or null if there is no transformation (in
4930     * which case the action is not applied)
4931     * @param action the action
4932     * @return the task
4933     */
4934     public static <K,V,U> ForkJoinTask<Void> forEach
4935     (ConcurrentHashMap<K,V> map,
4936     BiFun<? super K, ? super V, ? extends U> transformer,
4937     Action<U> action) {
4938     if (transformer == null || action == null)
4939     throw new NullPointerException();
4940     return new ForEachTransformedMappingTask<K,V,U>
4941     (map, null, -1, transformer, action);
4942     }
4943    
4944     /**
4945     * Returns a task that when invoked, returns a non-null result
4946     * from applying the given search function on each (key,
4947     * value), or null if none. Upon success, further element
4948     * processing is suppressed and the results of any other
4949     * parallel invocations of the search function are ignored.
4950     *
4951     * @param map the map
4952     * @param searchFunction a function returning a non-null
4953     * result on success, else null
4954     * @return the task
4955     */
4956     public static <K,V,U> ForkJoinTask<U> search
4957     (ConcurrentHashMap<K,V> map,
4958     BiFun<? super K, ? super V, ? extends U> searchFunction) {
4959     if (searchFunction == null) throw new NullPointerException();
4960     return new SearchMappingsTask<K,V,U>
4961     (map, null, -1, searchFunction,
4962     new AtomicReference<U>());
4963     }
4964    
4965     /**
4966     * Returns a task that when invoked, returns the result of
4967     * accumulating the given transformation of all (key, value) pairs
4968     * using the given reducer to combine values, or null if none.
4969     *
4970     * @param map the map
4971     * @param transformer a function returning the transformation
4972     * for an element, or null if there is no transformation (in
4973     * which case it is not combined).
4974     * @param reducer a commutative associative combining function
4975     * @return the task
4976     */
4977     public static <K,V,U> ForkJoinTask<U> reduce
4978     (ConcurrentHashMap<K,V> map,
4979     BiFun<? super K, ? super V, ? extends U> transformer,
4980     BiFun<? super U, ? super U, ? extends U> reducer) {
4981     if (transformer == null || reducer == null)
4982     throw new NullPointerException();
4983     return new MapReduceMappingsTask<K,V,U>
4984     (map, null, -1, null, transformer, reducer);
4985     }
4986    
4987     /**
4988     * Returns a task that when invoked, returns the result of
4989     * accumulating the given transformation of all (key, value) pairs
4990     * using the given reducer to combine values, and the given
4991     * basis as an identity value.
4992     *
4993     * @param map the map
4994     * @param transformer a function returning the transformation
4995     * for an element
4996     * @param basis the identity (initial default value) for the reduction
4997     * @param reducer a commutative associative combining function
4998     * @return the task
4999     */
5000     public static <K,V> ForkJoinTask<Double> reduceToDouble
5001     (ConcurrentHashMap<K,V> map,
5002     ObjectByObjectToDouble<? super K, ? super V> transformer,
5003     double basis,
5004     DoubleByDoubleToDouble reducer) {
5005     if (transformer == null || reducer == null)
5006     throw new NullPointerException();
5007     return new MapReduceMappingsToDoubleTask<K,V>
5008     (map, null, -1, null, transformer, basis, reducer);
5009     }
5010    
5011     /**
5012     * Returns a task that when invoked, returns the result of
5013     * accumulating the given transformation of all (key, value) pairs
5014     * using the given reducer to combine values, and the given
5015     * basis as an identity value.
5016     *
5017     * @param map the map
5018     * @param transformer a function returning the transformation
5019     * for an element
5020     * @param basis the identity (initial default value) for the reduction
5021     * @param reducer a commutative associative combining function
5022     * @return the task
5023     */
5024     public static <K,V> ForkJoinTask<Long> reduceToLong
5025     (ConcurrentHashMap<K,V> map,
5026     ObjectByObjectToLong<? super K, ? super V> transformer,
5027     long basis,
5028     LongByLongToLong reducer) {
5029     if (transformer == null || reducer == null)
5030     throw new NullPointerException();
5031     return new MapReduceMappingsToLongTask<K,V>
5032     (map, null, -1, null, transformer, basis, reducer);
5033     }
5034    
5035     /**
5036     * Returns a task that when invoked, returns the result of
5037     * accumulating the given transformation of all (key, value) pairs
5038     * using the given reducer to combine values, and the given
5039     * basis as an identity value.
5040     *
5041     * @param transformer a function returning the transformation
5042     * for an element
5043     * @param basis the identity (initial default value) for the reduction
5044     * @param reducer a commutative associative combining function
5045     * @return the task
5046     */
5047     public static <K,V> ForkJoinTask<Integer> reduceToInt
5048     (ConcurrentHashMap<K,V> map,
5049     ObjectByObjectToInt<? super K, ? super V> transformer,
5050     int basis,
5051     IntByIntToInt reducer) {
5052     if (transformer == null || reducer == null)
5053     throw new NullPointerException();
5054     return new MapReduceMappingsToIntTask<K,V>
5055     (map, null, -1, null, transformer, basis, reducer);
5056     }
5057    
5058     /**
5059     * Returns a task that when invoked, performs the given action
5060     * for each key.
5061     *
5062     * @param map the map
5063     * @param action the action
5064     * @return the task
5065     */
5066     public static <K,V> ForkJoinTask<Void> forEachKey
5067     (ConcurrentHashMap<K,V> map,
5068     Action<K> action) {
5069     if (action == null) throw new NullPointerException();
5070     return new ForEachKeyTask<K,V>(map, null, -1, action);
5071     }
5072    
5073     /**
5074     * Returns a task that when invoked, performs the given action
5075     * for each non-null transformation of each key.
5076     *
5077     * @param map the map
5078     * @param transformer a function returning the transformation
5079     * for an element, or null if there is no transformation (in
5080     * which case the action is not applied)
5081     * @param action the action
5082     * @return the task
5083     */
5084     public static <K,V,U> ForkJoinTask<Void> forEachKey
5085     (ConcurrentHashMap<K,V> map,
5086     Fun<? super K, ? extends U> transformer,
5087     Action<U> action) {
5088     if (transformer == null || action == null)
5089     throw new NullPointerException();
5090     return new ForEachTransformedKeyTask<K,V,U>
5091     (map, null, -1, transformer, action);
5092     }
5093    
5094     /**
5095     * Returns a task that when invoked, returns a non-null result
5096     * from applying the given search function on each key, or
5097     * null if none. Upon success, further element processing is
5098     * suppressed and the results of any other parallel
5099     * invocations of the search function are ignored.
5100     *
5101     * @param map the map
5102     * @param searchFunction a function returning a non-null
5103     * result on success, else null
5104     * @return the task
5105     */
5106     public static <K,V,U> ForkJoinTask<U> searchKeys
5107     (ConcurrentHashMap<K,V> map,
5108     Fun<? super K, ? extends U> searchFunction) {
5109     if (searchFunction == null) throw new NullPointerException();
5110     return new SearchKeysTask<K,V,U>
5111     (map, null, -1, searchFunction,
5112     new AtomicReference<U>());
5113     }
5114    
5115     /**
5116     * Returns a task that when invoked, returns the result of
5117     * accumulating all keys using the given reducer to combine
5118     * values, or null if none.
5119     *
5120     * @param map the map
5121     * @param reducer a commutative associative combining function
5122     * @return the task
5123     */
5124     public static <K,V> ForkJoinTask<K> reduceKeys
5125     (ConcurrentHashMap<K,V> map,
5126     BiFun<? super K, ? super K, ? extends K> reducer) {
5127     if (reducer == null) throw new NullPointerException();
5128     return new ReduceKeysTask<K,V>
5129     (map, null, -1, null, reducer);
5130     }
5131    
5132     /**
5133     * Returns a task that when invoked, returns the result of
5134     * accumulating the given transformation of all keys using the given
5135     * reducer to combine values, or null if none.
5136     *
5137     * @param map the map
5138     * @param transformer a function returning the transformation
5139     * for an element, or null if there is no transformation (in
5140     * which case it is not combined).
5141     * @param reducer a commutative associative combining function
5142     * @return the task
5143     */
5144     public static <K,V,U> ForkJoinTask<U> reduceKeys
5145     (ConcurrentHashMap<K,V> map,
5146     Fun<? super K, ? extends U> transformer,
5147     BiFun<? super U, ? super U, ? extends U> reducer) {
5148     if (transformer == null || reducer == null)
5149     throw new NullPointerException();
5150     return new MapReduceKeysTask<K,V,U>
5151     (map, null, -1, null, transformer, reducer);
5152     }
5153    
5154     /**
5155     * Returns a task that when invoked, returns the result of
5156     * accumulating the given transformation of all keys using the given
5157     * reducer to combine values, and the given basis as an
5158     * identity value.
5159     *
5160     * @param map the map
5161     * @param transformer a function returning the transformation
5162     * for an element
5163     * @param basis the identity (initial default value) for the reduction
5164     * @param reducer a commutative associative combining function
5165     * @return the task
5166     */
5167     public static <K,V> ForkJoinTask<Double> reduceKeysToDouble
5168     (ConcurrentHashMap<K,V> map,
5169     ObjectToDouble<? super K> transformer,
5170     double basis,
5171     DoubleByDoubleToDouble reducer) {
5172     if (transformer == null || reducer == null)
5173     throw new NullPointerException();
5174     return new MapReduceKeysToDoubleTask<K,V>
5175     (map, null, -1, null, transformer, basis, reducer);
5176     }
5177    
5178     /**
5179     * Returns a task that when invoked, returns the result of
5180     * accumulating the given transformation of all keys using the given
5181     * reducer to combine values, and the given basis as an
5182     * identity value.
5183     *
5184     * @param map the map
5185     * @param transformer a function returning the transformation
5186     * for an element
5187     * @param basis the identity (initial default value) for the reduction
5188     * @param reducer a commutative associative combining function
5189     * @return the task
5190     */
5191     public static <K,V> ForkJoinTask<Long> reduceKeysToLong
5192     (ConcurrentHashMap<K,V> map,
5193     ObjectToLong<? super K> transformer,
5194     long basis,
5195     LongByLongToLong reducer) {
5196     if (transformer == null || reducer == null)
5197     throw new NullPointerException();
5198     return new MapReduceKeysToLongTask<K,V>
5199     (map, null, -1, null, transformer, basis, reducer);
5200     }
5201    
5202     /**
5203     * Returns a task that when invoked, returns the result of
5204     * accumulating the given transformation of all keys using the given
5205     * reducer to combine values, and the given basis as an
5206     * identity value.
5207     *
5208     * @param map the map
5209     * @param transformer a function returning the transformation
5210     * for an element
5211     * @param basis the identity (initial default value) for the reduction
5212     * @param reducer a commutative associative combining function
5213     * @return the task
5214     */
5215     public static <K,V> ForkJoinTask<Integer> reduceKeysToInt
5216     (ConcurrentHashMap<K,V> map,
5217     ObjectToInt<? super K> transformer,
5218     int basis,
5219     IntByIntToInt reducer) {
5220     if (transformer == null || reducer == null)
5221     throw new NullPointerException();
5222     return new MapReduceKeysToIntTask<K,V>
5223     (map, null, -1, null, transformer, basis, reducer);
5224     }
5225    
5226     /**
5227     * Returns a task that when invoked, performs the given action
5228     * for each value.
5229     *
5230     * @param map the map
5231     * @param action the action
5232     */
5233     public static <K,V> ForkJoinTask<Void> forEachValue
5234     (ConcurrentHashMap<K,V> map,
5235     Action<V> action) {
5236     if (action == null) throw new NullPointerException();
5237     return new ForEachValueTask<K,V>(map, null, -1, action);
5238     }
5239    
5240     /**
5241     * Returns a task that when invoked, performs the given action
5242     * for each non-null transformation of each value.
5243     *
5244     * @param map the map
5245     * @param transformer a function returning the transformation
5246     * for an element, or null if there is no transformation (in
5247     * which case the action is not applied)
5248     * @param action the action
5249     */
5250     public static <K,V,U> ForkJoinTask<Void> forEachValue
5251     (ConcurrentHashMap<K,V> map,
5252     Fun<? super V, ? extends U> transformer,
5253     Action<U> action) {
5254     if (transformer == null || action == null)
5255     throw new NullPointerException();
5256     return new ForEachTransformedValueTask<K,V,U>
5257     (map, null, -1, transformer, action);
5258     }
5259    
5260     /**
5261     * Returns a task that when invoked, returns a non-null result
5262     * from applying the given search function on each value, or
5263     * null if none. Upon success, further element processing is
5264     * suppressed and the results of any other parallel
5265     * invocations of the search function are ignored.
5266     *
5267     * @param map the map
5268     * @param searchFunction a function returning a non-null
5269     * result on success, else null
5270     * @return the task
5271     */
5272     public static <K,V,U> ForkJoinTask<U> searchValues
5273     (ConcurrentHashMap<K,V> map,
5274     Fun<? super V, ? extends U> searchFunction) {
5275     if (searchFunction == null) throw new NullPointerException();
5276     return new SearchValuesTask<K,V,U>
5277     (map, null, -1, searchFunction,
5278     new AtomicReference<U>());
5279     }
5280    
5281     /**
5282     * Returns a task that when invoked, returns the result of
5283     * accumulating all values using the given reducer to combine
5284     * values, or null if none.
5285     *
5286     * @param map the map
5287     * @param reducer a commutative associative combining function
5288     * @return the task
5289     */
5290     public static <K,V> ForkJoinTask<V> reduceValues
5291     (ConcurrentHashMap<K,V> map,
5292     BiFun<? super V, ? super V, ? extends V> reducer) {
5293     if (reducer == null) throw new NullPointerException();
5294     return new ReduceValuesTask<K,V>
5295     (map, null, -1, null, reducer);
5296     }
5297    
5298     /**
5299     * Returns a task that when invoked, returns the result of
5300     * accumulating the given transformation of all values using the
5301     * given reducer to combine values, or null if none.
5302     *
5303     * @param map the map
5304     * @param transformer a function returning the transformation
5305     * for an element, or null if there is no transformation (in
5306     * which case it is not combined).
5307     * @param reducer a commutative associative combining function
5308     * @return the task
5309     */
5310     public static <K,V,U> ForkJoinTask<U> reduceValues
5311     (ConcurrentHashMap<K,V> map,
5312     Fun<? super V, ? extends U> transformer,
5313     BiFun<? super U, ? super U, ? extends U> reducer) {
5314     if (transformer == null || reducer == null)
5315     throw new NullPointerException();
5316     return new MapReduceValuesTask<K,V,U>
5317     (map, null, -1, null, transformer, reducer);
5318     }
5319    
5320     /**
5321     * Returns a task that when invoked, returns the result of
5322     * accumulating the given transformation of all values using the
5323     * given reducer to combine values, and the given basis as an
5324     * identity value.
5325     *
5326     * @param map the map
5327     * @param transformer a function returning the transformation
5328     * for an element
5329     * @param basis the identity (initial default value) for the reduction
5330     * @param reducer a commutative associative combining function
5331     * @return the task
5332     */
5333     public static <K,V> ForkJoinTask<Double> reduceValuesToDouble
5334     (ConcurrentHashMap<K,V> map,
5335     ObjectToDouble<? super V> transformer,
5336     double basis,
5337     DoubleByDoubleToDouble reducer) {
5338     if (transformer == null || reducer == null)
5339     throw new NullPointerException();
5340     return new MapReduceValuesToDoubleTask<K,V>
5341     (map, null, -1, null, transformer, basis, reducer);
5342     }
5343    
5344     /**
5345     * Returns a task that when invoked, returns the result of
5346     * accumulating the given transformation of all values using the
5347     * given reducer to combine values, and the given basis as an
5348     * identity value.
5349     *
5350     * @param map the map
5351     * @param transformer a function returning the transformation
5352     * for an element
5353     * @param basis the identity (initial default value) for the reduction
5354     * @param reducer a commutative associative combining function
5355     * @return the task
5356     */
5357     public static <K,V> ForkJoinTask<Long> reduceValuesToLong
5358     (ConcurrentHashMap<K,V> map,
5359     ObjectToLong<? super V> transformer,
5360     long basis,
5361     LongByLongToLong reducer) {
5362     if (transformer == null || reducer == null)
5363     throw new NullPointerException();
5364     return new MapReduceValuesToLongTask<K,V>
5365     (map, null, -1, null, transformer, basis, reducer);
5366     }
5367    
5368     /**
5369     * Returns a task that when invoked, returns the result of
5370     * accumulating the given transformation of all values using the
5371     * given reducer to combine values, and the given basis as an
5372     * identity value.
5373     *
5374     * @param map the map
5375     * @param transformer a function returning the transformation
5376     * for an element
5377     * @param basis the identity (initial default value) for the reduction
5378     * @param reducer a commutative associative combining function
5379     * @return the task
5380     */
5381     public static <K,V> ForkJoinTask<Integer> reduceValuesToInt
5382     (ConcurrentHashMap<K,V> map,
5383     ObjectToInt<? super V> transformer,
5384     int basis,
5385     IntByIntToInt reducer) {
5386     if (transformer == null || reducer == null)
5387     throw new NullPointerException();
5388     return new MapReduceValuesToIntTask<K,V>
5389     (map, null, -1, null, transformer, basis, reducer);
5390     }
5391    
5392     /**
5393     * Returns a task that when invoked, perform the given action
5394     * for each entry.
5395     *
5396     * @param map the map
5397     * @param action the action
5398     */
5399     public static <K,V> ForkJoinTask<Void> forEachEntry
5400     (ConcurrentHashMap<K,V> map,
5401     Action<Map.Entry<K,V>> action) {
5402     if (action == null) throw new NullPointerException();
5403     return new ForEachEntryTask<K,V>(map, null, -1, action);
5404     }
5405    
5406     /**
5407     * Returns a task that when invoked, perform the given action
5408     * for each non-null transformation of each entry.
5409     *
5410     * @param map the map
5411     * @param transformer a function returning the transformation
5412     * for an element, or null if there is no transformation (in
5413     * which case the action is not applied)
5414     * @param action the action
5415     */
5416     public static <K,V,U> ForkJoinTask<Void> forEachEntry
5417     (ConcurrentHashMap<K,V> map,
5418     Fun<Map.Entry<K,V>, ? extends U> transformer,
5419     Action<U> action) {
5420     if (transformer == null || action == null)
5421     throw new NullPointerException();
5422     return new ForEachTransformedEntryTask<K,V,U>
5423     (map, null, -1, transformer, action);
5424     }
5425    
5426     /**
5427     * Returns a task that when invoked, returns a non-null result
5428     * from applying the given search function on each entry, or
5429     * null if none. Upon success, further element processing is
5430     * suppressed and the results of any other parallel
5431     * invocations of the search function are ignored.
5432     *
5433     * @param map the map
5434     * @param searchFunction a function returning a non-null
5435     * result on success, else null
5436     * @return the task
5437     */
5438     public static <K,V,U> ForkJoinTask<U> searchEntries
5439     (ConcurrentHashMap<K,V> map,
5440     Fun<Map.Entry<K,V>, ? extends U> searchFunction) {
5441     if (searchFunction == null) throw new NullPointerException();
5442     return new SearchEntriesTask<K,V,U>
5443     (map, null, -1, searchFunction,
5444     new AtomicReference<U>());
5445     }
5446    
5447     /**
5448     * Returns a task that when invoked, returns the result of
5449     * accumulating all entries using the given reducer to combine
5450     * values, or null if none.
5451     *
5452     * @param map the map
5453     * @param reducer a commutative associative combining function
5454     * @return the task
5455     */
5456     public static <K,V> ForkJoinTask<Map.Entry<K,V>> reduceEntries
5457     (ConcurrentHashMap<K,V> map,
5458     BiFun<Map.Entry<K,V>, Map.Entry<K,V>, ? extends Map.Entry<K,V>> reducer) {
5459     if (reducer == null) throw new NullPointerException();
5460     return new ReduceEntriesTask<K,V>
5461     (map, null, -1, null, reducer);
5462     }
5463    
5464     /**
5465     * Returns a task that when invoked, returns the result of
5466     * accumulating the given transformation of all entries using the
5467     * given reducer to combine values, or null if none.
5468     *
5469     * @param map the map
5470     * @param transformer a function returning the transformation
5471     * for an element, or null if there is no transformation (in
5472     * which case it is not combined).
5473     * @param reducer a commutative associative combining function
5474     * @return the task
5475     */
5476     public static <K,V,U> ForkJoinTask<U> reduceEntries
5477     (ConcurrentHashMap<K,V> map,
5478     Fun<Map.Entry<K,V>, ? extends U> transformer,
5479     BiFun<? super U, ? super U, ? extends U> reducer) {
5480     if (transformer == null || reducer == null)
5481     throw new NullPointerException();
5482     return new MapReduceEntriesTask<K,V,U>
5483     (map, null, -1, null, transformer, reducer);
5484     }
5485    
5486     /**
5487     * Returns a task that when invoked, returns the result of
5488     * accumulating the given transformation of all entries using the
5489     * given reducer to combine values, and the given basis as an
5490     * identity value.
5491     *
5492     * @param map the map
5493     * @param transformer a function returning the transformation
5494     * for an element
5495     * @param basis the identity (initial default value) for the reduction
5496     * @param reducer a commutative associative combining function
5497     * @return the task
5498     */
5499     public static <K,V> ForkJoinTask<Double> reduceEntriesToDouble
5500     (ConcurrentHashMap<K,V> map,
5501     ObjectToDouble<Map.Entry<K,V>> transformer,
5502     double basis,
5503     DoubleByDoubleToDouble reducer) {
5504     if (transformer == null || reducer == null)
5505     throw new NullPointerException();
5506     return new MapReduceEntriesToDoubleTask<K,V>
5507     (map, null, -1, null, transformer, basis, reducer);
5508     }
5509    
5510     /**
5511     * Returns a task that when invoked, returns the result of
5512     * accumulating the given transformation of all entries using the
5513     * given reducer to combine values, and the given basis as an
5514     * identity value.
5515     *
5516     * @param map the map
5517     * @param transformer a function returning the transformation
5518     * for an element
5519     * @param basis the identity (initial default value) for the reduction
5520     * @param reducer a commutative associative combining function
5521     * @return the task
5522     */
5523     public static <K,V> ForkJoinTask<Long> reduceEntriesToLong
5524     (ConcurrentHashMap<K,V> map,
5525     ObjectToLong<Map.Entry<K,V>> transformer,
5526     long basis,
5527     LongByLongToLong reducer) {
5528     if (transformer == null || reducer == null)
5529     throw new NullPointerException();
5530     return new MapReduceEntriesToLongTask<K,V>
5531     (map, null, -1, null, transformer, basis, reducer);
5532     }
5533    
5534     /**
5535     * Returns a task that when invoked, returns the result of
5536     * accumulating the given transformation of all entries using the
5537     * given reducer to combine values, and the given basis as an
5538     * identity value.
5539     *
5540     * @param map the map
5541     * @param transformer a function returning the transformation
5542     * for an element
5543     * @param basis the identity (initial default value) for the reduction
5544     * @param reducer a commutative associative combining function
5545     * @return the task
5546     */
5547     public static <K,V> ForkJoinTask<Integer> reduceEntriesToInt
5548     (ConcurrentHashMap<K,V> map,
5549     ObjectToInt<Map.Entry<K,V>> transformer,
5550     int basis,
5551     IntByIntToInt reducer) {
5552     if (transformer == null || reducer == null)
5553     throw new NullPointerException();
5554     return new MapReduceEntriesToIntTask<K,V>
5555     (map, null, -1, null, transformer, basis, reducer);
5556     }
5557     }
5558    
5559     // -------------------------------------------------------
5560    
5561     /*
5562     * Task classes. Coded in a regular but ugly format/style to
5563     * simplify checks that each variant differs in the right way from
5564     * others. The null screenings exist because compilers cannot tell
5565     * that we've already null-checked task arguments, so we force
5566     * simplest hoisted bypass to help avoid convoluted traps.
5567     */
5568    
5569     @SuppressWarnings("serial") static final class ForEachKeyTask<K,V>
5570     extends Traverser<K,V,Void> {
5571     final Action<K> action;
5572     ForEachKeyTask
5573     (ConcurrentHashMap<K,V> m, Traverser<K,V,?> p, int b,
5574     Action<K> action) {
5575     super(m, p, b);
5576     this.action = action;
5577     }
5578     @SuppressWarnings("unchecked") public final void compute() {
5579     final Action<K> action;
5580     if ((action = this.action) != null) {
5581     for (int b; (b = preSplit()) > 0;)
5582     new ForEachKeyTask<K,V>(map, this, b, action).fork();
5583     while (advance() != null)
5584     action.apply((K)nextKey);
5585     propagateCompletion();
5586     }
5587     }
5588     }
5589    
5590     @SuppressWarnings("serial") static final class ForEachValueTask<K,V>
5591     extends Traverser<K,V,Void> {
5592     final Action<V> action;
5593     ForEachValueTask
5594     (ConcurrentHashMap<K,V> m, Traverser<K,V,?> p, int b,
5595     Action<V> action) {
5596     super(m, p, b);
5597     this.action = action;
5598     }
5599     @SuppressWarnings("unchecked") public final void compute() {
5600     final Action<V> action;
5601     if ((action = this.action) != null) {
5602     for (int b; (b = preSplit()) > 0;)
5603     new ForEachValueTask<K,V>(map, this, b, action).fork();
5604     V v;
5605     while ((v = advance()) != null)
5606     action.apply(v);
5607     propagateCompletion();
5608     }
5609     }
5610     }
5611    
5612     @SuppressWarnings("serial") static final class ForEachEntryTask<K,V>
5613     extends Traverser<K,V,Void> {
5614     final Action<Entry<K,V>> action;
5615     ForEachEntryTask
5616     (ConcurrentHashMap<K,V> m, Traverser<K,V,?> p, int b,
5617     Action<Entry<K,V>> action) {
5618     super(m, p, b);
5619     this.action = action;
5620     }
5621     @SuppressWarnings("unchecked") public final void compute() {
5622     final Action<Entry<K,V>> action;
5623     if ((action = this.action) != null) {
5624     for (int b; (b = preSplit()) > 0;)
5625     new ForEachEntryTask<K,V>(map, this, b, action).fork();
5626     V v;
5627     while ((v = advance()) != null)
5628     action.apply(entryFor((K)nextKey, v));
5629     propagateCompletion();
5630     }
5631     }
5632     }
5633    
5634     @SuppressWarnings("serial") static final class ForEachMappingTask<K,V>
5635     extends Traverser<K,V,Void> {
5636     final BiAction<K,V> action;
5637     ForEachMappingTask
5638     (ConcurrentHashMap<K,V> m, Traverser<K,V,?> p, int b,
5639     BiAction<K,V> action) {
5640     super(m, p, b);
5641     this.action = action;
5642     }
5643     @SuppressWarnings("unchecked") public final void compute() {
5644     final BiAction<K,V> action;
5645     if ((action = this.action) != null) {
5646     for (int b; (b = preSplit()) > 0;)
5647     new ForEachMappingTask<K,V>(map, this, b, action).fork();
5648     V v;
5649     while ((v = advance()) != null)
5650     action.apply((K)nextKey, v);
5651     propagateCompletion();
5652     }
5653     }
5654     }
5655    
5656     @SuppressWarnings("serial") static final class ForEachTransformedKeyTask<K,V,U>
5657     extends Traverser<K,V,Void> {
5658     final Fun<? super K, ? extends U> transformer;
5659     final Action<U> action;
5660     ForEachTransformedKeyTask
5661     (ConcurrentHashMap<K,V> m, Traverser<K,V,?> p, int b,
5662     Fun<? super K, ? extends U> transformer, Action<U> action) {
5663     super(m, p, b);
5664     this.transformer = transformer; this.action = action;
5665     }
5666     @SuppressWarnings("unchecked") public final void compute() {
5667     final Fun<? super K, ? extends U> transformer;
5668     final Action<U> action;
5669     if ((transformer = this.transformer) != null &&
5670     (action = this.action) != null) {
5671     for (int b; (b = preSplit()) > 0;)
5672     new ForEachTransformedKeyTask<K,V,U>
5673     (map, this, b, transformer, action).fork();
5674     U u;
5675     while (advance() != null) {
5676     if ((u = transformer.apply((K)nextKey)) != null)
5677     action.apply(u);
5678     }
5679     propagateCompletion();
5680     }
5681     }
5682     }
5683    
5684     @SuppressWarnings("serial") static final class ForEachTransformedValueTask<K,V,U>
5685     extends Traverser<K,V,Void> {
5686     final Fun<? super V, ? extends U> transformer;
5687     final Action<U> action;
5688     ForEachTransformedValueTask
5689     (ConcurrentHashMap<K,V> m, Traverser<K,V,?> p, int b,
5690     Fun<? super V, ? extends U> transformer, Action<U> action) {
5691     super(m, p, b);
5692     this.transformer = transformer; this.action = action;
5693     }
5694     @SuppressWarnings("unchecked") public final void compute() {
5695     final Fun<? super V, ? extends U> transformer;
5696     final Action<U> action;
5697     if ((transformer = this.transformer) != null &&
5698     (action = this.action) != null) {
5699     for (int b; (b = preSplit()) > 0;)
5700     new ForEachTransformedValueTask<K,V,U>
5701     (map, this, b, transformer, action).fork();
5702     V v; U u;
5703     while ((v = advance()) != null) {
5704     if ((u = transformer.apply(v)) != null)
5705     action.apply(u);
5706     }
5707     propagateCompletion();
5708     }
5709     }
5710     }
5711    
5712     @SuppressWarnings("serial") static final class ForEachTransformedEntryTask<K,V,U>
5713     extends Traverser<K,V,Void> {
5714     final Fun<Map.Entry<K,V>, ? extends U> transformer;
5715     final Action<U> action;
5716     ForEachTransformedEntryTask
5717     (ConcurrentHashMap<K,V> m, Traverser<K,V,?> p, int b,
5718     Fun<Map.Entry<K,V>, ? extends U> transformer, Action<U> action) {
5719     super(m, p, b);
5720     this.transformer = transformer; this.action = action;
5721     }
5722     @SuppressWarnings("unchecked") public final void compute() {
5723     final Fun<Map.Entry<K,V>, ? extends U> transformer;
5724     final Action<U> action;
5725     if ((transformer = this.transformer) != null &&
5726     (action = this.action) != null) {
5727     for (int b; (b = preSplit()) > 0;)
5728     new ForEachTransformedEntryTask<K,V,U>
5729     (map, this, b, transformer, action).fork();
5730     V v; U u;
5731     while ((v = advance()) != null) {
5732     if ((u = transformer.apply(entryFor((K)nextKey,
5733     v))) != null)
5734     action.apply(u);
5735     }
5736     propagateCompletion();
5737     }
5738     }
5739     }
5740    
5741     @SuppressWarnings("serial") static final class ForEachTransformedMappingTask<K,V,U>
5742     extends Traverser<K,V,Void> {
5743     final BiFun<? super K, ? super V, ? extends U> transformer;
5744     final Action<U> action;
5745     ForEachTransformedMappingTask
5746     (ConcurrentHashMap<K,V> m, Traverser<K,V,?> p, int b,
5747     BiFun<? super K, ? super V, ? extends U> transformer,
5748     Action<U> action) {
5749     super(m, p, b);
5750     this.transformer = transformer; this.action = action;
5751     }
5752     @SuppressWarnings("unchecked") public final void compute() {
5753     final BiFun<? super K, ? super V, ? extends U> transformer;
5754     final Action<U> action;
5755     if ((transformer = this.transformer) != null &&
5756     (action = this.action) != null) {
5757     for (int b; (b = preSplit()) > 0;)
5758     new ForEachTransformedMappingTask<K,V,U>
5759     (map, this, b, transformer, action).fork();
5760     V v; U u;
5761     while ((v = advance()) != null) {
5762     if ((u = transformer.apply((K)nextKey, v)) != null)
5763     action.apply(u);
5764     }
5765     propagateCompletion();
5766     }
5767     }
5768     }
5769    
5770     @SuppressWarnings("serial") static final class SearchKeysTask<K,V,U>
5771     extends Traverser<K,V,U> {
5772     final Fun<? super K, ? extends U> searchFunction;
5773     final AtomicReference<U> result;
5774     SearchKeysTask
5775     (ConcurrentHashMap<K,V> m, Traverser<K,V,?> p, int b,
5776     Fun<? super K, ? extends U> searchFunction,
5777     AtomicReference<U> result) {
5778     super(m, p, b);
5779     this.searchFunction = searchFunction; this.result = result;
5780     }
5781     public final U getRawResult() { return result.get(); }
5782     @SuppressWarnings("unchecked") public final void compute() {
5783     final Fun<? super K, ? extends U> searchFunction;
5784     final AtomicReference<U> result;
5785     if ((searchFunction = this.searchFunction) != null &&
5786     (result = this.result) != null) {
5787     for (int b;;) {
5788     if (result.get() != null)
5789     return;
5790     if ((b = preSplit()) <= 0)
5791     break;
5792     new SearchKeysTask<K,V,U>
5793     (map, this, b, searchFunction, result).fork();
5794     }
5795     while (result.get() == null) {
5796     U u;
5797     if (advance() == null) {
5798     propagateCompletion();
5799     break;
5800     }
5801     if ((u = searchFunction.apply((K)nextKey)) != null) {
5802     if (result.compareAndSet(null, u))
5803     quietlyCompleteRoot();
5804     break;
5805     }
5806     }
5807     }
5808     }
5809     }
5810    
5811     @SuppressWarnings("serial") static final class SearchValuesTask<K,V,U>
5812     extends Traverser<K,V,U> {
5813     final Fun<? super V, ? extends U> searchFunction;
5814     final AtomicReference<U> result;
5815     SearchValuesTask
5816     (ConcurrentHashMap<K,V> m, Traverser<K,V,?> p, int b,
5817     Fun<? super V, ? extends U> searchFunction,
5818     AtomicReference<U> result) {
5819     super(m, p, b);
5820     this.searchFunction = searchFunction; this.result = result;
5821     }
5822     public final U getRawResult() { return result.get(); }
5823     @SuppressWarnings("unchecked") public final void compute() {
5824     final Fun<? super V, ? extends U> searchFunction;
5825     final AtomicReference<U> result;
5826     if ((searchFunction = this.searchFunction) != null &&
5827     (result = this.result) != null) {
5828     for (int b;;) {
5829     if (result.get() != null)
5830     return;
5831     if ((b = preSplit()) <= 0)
5832     break;
5833     new SearchValuesTask<K,V,U>
5834     (map, this, b, searchFunction, result).fork();
5835     }
5836     while (result.get() == null) {
5837     V v; U u;
5838     if ((v = advance()) == null) {
5839     propagateCompletion();
5840     break;
5841     }
5842     if ((u = searchFunction.apply(v)) != null) {
5843     if (result.compareAndSet(null, u))
5844     quietlyCompleteRoot();
5845     break;
5846     }
5847     }
5848     }
5849     }
5850     }
5851    
5852     @SuppressWarnings("serial") static final class SearchEntriesTask<K,V,U>
5853     extends Traverser<K,V,U> {
5854     final Fun<Entry<K,V>, ? extends U> searchFunction;
5855     final AtomicReference<U> result;
5856     SearchEntriesTask
5857     (ConcurrentHashMap<K,V> m, Traverser<K,V,?> p, int b,
5858     Fun<Entry<K,V>, ? extends U> searchFunction,
5859     AtomicReference<U> result) {
5860     super(m, p, b);
5861     this.searchFunction = searchFunction; this.result = result;
5862     }
5863     public final U getRawResult() { return result.get(); }
5864     @SuppressWarnings("unchecked") public final void compute() {
5865     final Fun<Entry<K,V>, ? extends U> searchFunction;
5866     final AtomicReference<U> result;
5867     if ((searchFunction = this.searchFunction) != null &&
5868     (result = this.result) != null) {
5869     for (int b;;) {
5870     if (result.get() != null)
5871     return;
5872     if ((b = preSplit()) <= 0)
5873     break;
5874     new SearchEntriesTask<K,V,U>
5875     (map, this, b, searchFunction, result).fork();
5876     }
5877     while (result.get() == null) {
5878     V v; U u;
5879     if ((v = advance()) == null) {
5880     propagateCompletion();
5881     break;
5882     }
5883     if ((u = searchFunction.apply(entryFor((K)nextKey,
5884     v))) != null) {
5885     if (result.compareAndSet(null, u))
5886     quietlyCompleteRoot();
5887     return;
5888     }
5889     }
5890     }
5891     }
5892     }
5893    
5894     @SuppressWarnings("serial") static final class SearchMappingsTask<K,V,U>
5895     extends Traverser<K,V,U> {
5896     final BiFun<? super K, ? super V, ? extends U> searchFunction;
5897     final AtomicReference<U> result;
5898     SearchMappingsTask
5899     (ConcurrentHashMap<K,V> m, Traverser<K,V,?> p, int b,
5900     BiFun<? super K, ? super V, ? extends U> searchFunction,
5901     AtomicReference<U> result) {
5902     super(m, p, b);
5903     this.searchFunction = searchFunction; this.result = result;
5904     }
5905     public final U getRawResult() { return result.get(); }
5906     @SuppressWarnings("unchecked") public final void compute() {
5907     final BiFun<? super K, ? super V, ? extends U> searchFunction;
5908     final AtomicReference<U> result;
5909     if ((searchFunction = this.searchFunction) != null &&
5910     (result = this.result) != null) {
5911     for (int b;;) {
5912     if (result.get() != null)
5913     return;
5914     if ((b = preSplit()) <= 0)
5915     break;
5916     new SearchMappingsTask<K,V,U>
5917     (map, this, b, searchFunction, result).fork();
5918     }
5919     while (result.get() == null) {
5920     V v; U u;
5921     if ((v = advance()) == null) {
5922     propagateCompletion();
5923     break;
5924     }
5925     if ((u = searchFunction.apply((K)nextKey, v)) != null) {
5926     if (result.compareAndSet(null, u))
5927     quietlyCompleteRoot();
5928     break;
5929     }
5930     }
5931     }
5932     }
5933     }
5934    
5935     @SuppressWarnings("serial") static final class ReduceKeysTask<K,V>
5936     extends Traverser<K,V,K> {
5937     final BiFun<? super K, ? super K, ? extends K> reducer;
5938     K result;
5939     ReduceKeysTask<K,V> rights, nextRight;
5940     ReduceKeysTask
5941     (ConcurrentHashMap<K,V> m, Traverser<K,V,?> p, int b,
5942     ReduceKeysTask<K,V> nextRight,
5943     BiFun<? super K, ? super K, ? extends K> reducer) {
5944     super(m, p, b); this.nextRight = nextRight;
5945     this.reducer = reducer;
5946     }
5947     public final K getRawResult() { return result; }
5948     @SuppressWarnings("unchecked") public final void compute() {
5949     final BiFun<? super K, ? super K, ? extends K> reducer;
5950     if ((reducer = this.reducer) != null) {
5951     for (int b; (b = preSplit()) > 0;)
5952     (rights = new ReduceKeysTask<K,V>
5953     (map, this, b, rights, reducer)).fork();
5954     K r = null;
5955     while (advance() != null) {
5956     K u = (K)nextKey;
5957     r = (r == null) ? u : reducer.apply(r, u);
5958     }
5959     result = r;
5960     CountedCompleter<?> c;
5961     for (c = firstComplete(); c != null; c = c.nextComplete()) {
5962     ReduceKeysTask<K,V>
5963     t = (ReduceKeysTask<K,V>)c,
5964     s = t.rights;
5965     while (s != null) {
5966     K tr, sr;
5967     if ((sr = s.result) != null)
5968     t.result = (((tr = t.result) == null) ? sr :
5969     reducer.apply(tr, sr));
5970     s = t.rights = s.nextRight;
5971     }
5972     }
5973     }
5974     }
5975     }
5976    
5977     @SuppressWarnings("serial") static final class ReduceValuesTask<K,V>
5978     extends Traverser<K,V,V> {
5979     final BiFun<? super V, ? super V, ? extends V> reducer;
5980     V result;
5981     ReduceValuesTask<K,V> rights, nextRight;
5982     ReduceValuesTask
5983     (ConcurrentHashMap<K,V> m, Traverser<K,V,?> p, int b,
5984     ReduceValuesTask<K,V> nextRight,
5985     BiFun<? super V, ? super V, ? extends V> reducer) {
5986     super(m, p, b); this.nextRight = nextRight;
5987     this.reducer = reducer;
5988     }
5989     public final V getRawResult() { return result; }
5990     @SuppressWarnings("unchecked") public final void compute() {
5991     final BiFun<? super V, ? super V, ? extends V> reducer;
5992     if ((reducer = this.reducer) != null) {
5993     for (int b; (b = preSplit()) > 0;)
5994     (rights = new ReduceValuesTask<K,V>
5995     (map, this, b, rights, reducer)).fork();
5996     V r = null;
5997     V v;
5998     while ((v = advance()) != null) {
5999     V u = v;
6000     r = (r == null) ? u : reducer.apply(r, u);
6001     }
6002     result = r;
6003     CountedCompleter<?> c;
6004     for (c = firstComplete(); c != null; c = c.nextComplete()) {
6005     ReduceValuesTask<K,V>
6006     t = (ReduceValuesTask<K,V>)c,
6007     s = t.rights;
6008     while (s != null) {
6009     V tr, sr;
6010     if ((sr = s.result) != null)
6011     t.result = (((tr = t.result) == null) ? sr :
6012     reducer.apply(tr, sr));
6013     s = t.rights = s.nextRight;
6014     }
6015     }
6016     }
6017     }
6018     }
6019    
6020     @SuppressWarnings("serial") static final class ReduceEntriesTask<K,V>
6021     extends Traverser<K,V,Map.Entry<K,V>> {
6022     final BiFun<Map.Entry<K,V>, Map.Entry<K,V>, ? extends Map.Entry<K,V>> reducer;
6023     Map.Entry<K,V> result;
6024     ReduceEntriesTask<K,V> rights, nextRight;
6025     ReduceEntriesTask
6026     (ConcurrentHashMap<K,V> m, Traverser<K,V,?> p, int b,
6027     ReduceEntriesTask<K,V> nextRight,
6028     BiFun<Entry<K,V>, Map.Entry<K,V>, ? extends Map.Entry<K,V>> reducer) {
6029     super(m, p, b); this.nextRight = nextRight;
6030     this.reducer = reducer;
6031     }
6032     public final Map.Entry<K,V> getRawResult() { return result; }
6033     @SuppressWarnings("unchecked") public final void compute() {
6034     final BiFun<Map.Entry<K,V>, Map.Entry<K,V>, ? extends Map.Entry<K,V>> reducer;
6035     if ((reducer = this.reducer) != null) {
6036     for (int b; (b = preSplit()) > 0;)
6037     (rights = new ReduceEntriesTask<K,V>
6038     (map, this, b, rights, reducer)).fork();
6039     Map.Entry<K,V> r = null;
6040     V v;
6041     while ((v = advance()) != null) {
6042     Map.Entry<K,V> u = entryFor((K)nextKey, v);
6043     r = (r == null) ? u : reducer.apply(r, u);
6044     }
6045     result = r;
6046     CountedCompleter<?> c;
6047     for (c = firstComplete(); c != null; c = c.nextComplete()) {
6048     ReduceEntriesTask<K,V>
6049     t = (ReduceEntriesTask<K,V>)c,
6050     s = t.rights;
6051     while (s != null) {
6052     Map.Entry<K,V> tr, sr;
6053     if ((sr = s.result) != null)
6054     t.result = (((tr = t.result) == null) ? sr :
6055     reducer.apply(tr, sr));
6056     s = t.rights = s.nextRight;
6057     }
6058     }
6059     }
6060     }
6061     }
6062    
6063     @SuppressWarnings("serial") static final class MapReduceKeysTask<K,V,U>
6064     extends Traverser<K,V,U> {
6065     final Fun<? super K, ? extends U> transformer;
6066     final BiFun<? super U, ? super U, ? extends U> reducer;
6067     U result;
6068     MapReduceKeysTask<K,V,U> rights, nextRight;
6069     MapReduceKeysTask
6070     (ConcurrentHashMap<K,V> m, Traverser<K,V,?> p, int b,
6071     MapReduceKeysTask<K,V,U> nextRight,
6072     Fun<? super K, ? extends U> transformer,
6073     BiFun<? super U, ? super U, ? extends U> reducer) {
6074     super(m, p, b); this.nextRight = nextRight;
6075     this.transformer = transformer;
6076     this.reducer = reducer;
6077     }
6078     public final U getRawResult() { return result; }
6079     @SuppressWarnings("unchecked") public final void compute() {
6080     final Fun<? super K, ? extends U> transformer;
6081     final BiFun<? super U, ? super U, ? extends U> reducer;
6082     if ((transformer = this.transformer) != null &&
6083     (reducer = this.reducer) != null) {
6084     for (int b; (b = preSplit()) > 0;)
6085     (rights = new MapReduceKeysTask<K,V,U>
6086     (map, this, b, rights, transformer, reducer)).fork();
6087     U r = null, u;
6088     while (advance() != null) {
6089     if ((u = transformer.apply((K)nextKey)) != null)
6090     r = (r == null) ? u : reducer.apply(r, u);
6091     }
6092     result = r;
6093     CountedCompleter<?> c;
6094     for (c = firstComplete(); c != null; c = c.nextComplete()) {
6095     MapReduceKeysTask<K,V,U>
6096     t = (MapReduceKeysTask<K,V,U>)c,
6097     s = t.rights;
6098     while (s != null) {
6099     U tr, sr;
6100     if ((sr = s.result) != null)
6101     t.result = (((tr = t.result) == null) ? sr :
6102     reducer.apply(tr, sr));
6103     s = t.rights = s.nextRight;
6104     }
6105     }
6106     }
6107     }
6108     }
6109    
6110     @SuppressWarnings("serial") static final class MapReduceValuesTask<K,V,U>
6111     extends Traverser<K,V,U> {
6112     final Fun<? super V, ? extends U> transformer;
6113     final BiFun<? super U, ? super U, ? extends U> reducer;
6114     U result;
6115     MapReduceValuesTask<K,V,U> rights, nextRight;
6116     MapReduceValuesTask
6117     (ConcurrentHashMap<K,V> m, Traverser<K,V,?> p, int b,
6118     MapReduceValuesTask<K,V,U> nextRight,
6119     Fun<? super V, ? extends U> transformer,
6120     BiFun<? super U, ? super U, ? extends U> reducer) {
6121     super(m, p, b); this.nextRight = nextRight;
6122     this.transformer = transformer;
6123     this.reducer = reducer;
6124     }
6125     public final U getRawResult() { return result; }
6126     @SuppressWarnings("unchecked") public final void compute() {
6127     final Fun<? super V, ? extends U> transformer;
6128     final BiFun<? super U, ? super U, ? extends U> reducer;
6129     if ((transformer = this.transformer) != null &&
6130     (reducer = this.reducer) != null) {
6131     for (int b; (b = preSplit()) > 0;)
6132     (rights = new MapReduceValuesTask<K,V,U>
6133     (map, this, b, rights, transformer, reducer)).fork();
6134     U r = null, u;
6135     V v;
6136     while ((v = advance()) != null) {
6137     if ((u = transformer.apply(v)) != null)
6138     r = (r == null) ? u : reducer.apply(r, u);
6139     }
6140     result = r;
6141     CountedCompleter<?> c;
6142     for (c = firstComplete(); c != null; c = c.nextComplete()) {
6143     MapReduceValuesTask<K,V,U>
6144     t = (MapReduceValuesTask<K,V,U>)c,
6145     s = t.rights;
6146     while (s != null) {
6147     U tr, sr;
6148     if ((sr = s.result) != null)
6149     t.result = (((tr = t.result) == null) ? sr :
6150     reducer.apply(tr, sr));
6151     s = t.rights = s.nextRight;
6152     }
6153     }
6154     }
6155     }
6156     }
6157    
6158     @SuppressWarnings("serial") static final class MapReduceEntriesTask<K,V,U>
6159     extends Traverser<K,V,U> {
6160     final Fun<Map.Entry<K,V>, ? extends U> transformer;
6161     final BiFun<? super U, ? super U, ? extends U> reducer;
6162     U result;
6163     MapReduceEntriesTask<K,V,U> rights, nextRight;
6164     MapReduceEntriesTask
6165     (ConcurrentHashMap<K,V> m, Traverser<K,V,?> p, int b,
6166     MapReduceEntriesTask<K,V,U> nextRight,
6167     Fun<Map.Entry<K,V>, ? extends U> transformer,
6168     BiFun<? super U, ? super U, ? extends U> reducer) {
6169     super(m, p, b); this.nextRight = nextRight;
6170     this.transformer = transformer;
6171     this.reducer = reducer;
6172     }
6173     public final U getRawResult() { return result; }
6174     @SuppressWarnings("unchecked") public final void compute() {
6175     final Fun<Map.Entry<K,V>, ? extends U> transformer;
6176     final BiFun<? super U, ? super U, ? extends U> reducer;
6177     if ((transformer = this.transformer) != null &&
6178     (reducer = this.reducer) != null) {
6179     for (int b; (b = preSplit()) > 0;)
6180     (rights = new MapReduceEntriesTask<K,V,U>
6181     (map, this, b, rights, transformer, reducer)).fork();
6182     U r = null, u;
6183     V v;
6184     while ((v = advance()) != null) {
6185     if ((u = transformer.apply(entryFor((K)nextKey,
6186     v))) != null)
6187     r = (r == null) ? u : reducer.apply(r, u);
6188     }
6189     result = r;
6190     CountedCompleter<?> c;
6191     for (c = firstComplete(); c != null; c = c.nextComplete()) {
6192     MapReduceEntriesTask<K,V,U>
6193     t = (MapReduceEntriesTask<K,V,U>)c,
6194     s = t.rights;
6195     while (s != null) {
6196     U tr, sr;
6197     if ((sr = s.result) != null)
6198     t.result = (((tr = t.result) == null) ? sr :
6199     reducer.apply(tr, sr));
6200     s = t.rights = s.nextRight;
6201     }
6202     }
6203     }
6204     }
6205     }
6206    
6207     @SuppressWarnings("serial") static final class MapReduceMappingsTask<K,V,U>
6208     extends Traverser<K,V,U> {
6209     final BiFun<? super K, ? super V, ? extends U> transformer;
6210     final BiFun<? super U, ? super U, ? extends U> reducer;
6211     U result;
6212     MapReduceMappingsTask<K,V,U> rights, nextRight;
6213     MapReduceMappingsTask
6214     (ConcurrentHashMap<K,V> m, Traverser<K,V,?> p, int b,
6215     MapReduceMappingsTask<K,V,U> nextRight,
6216     BiFun<? super K, ? super V, ? extends U> transformer,
6217     BiFun<? super U, ? super U, ? extends U> reducer) {
6218     super(m, p, b); this.nextRight = nextRight;
6219     this.transformer = transformer;
6220     this.reducer = reducer;
6221     }
6222     public final U getRawResult() { return result; }
6223     @SuppressWarnings("unchecked") public final void compute() {
6224     final BiFun<? super K, ? super V, ? extends U> transformer;
6225     final BiFun<? super U, ? super U, ? extends U> reducer;
6226     if ((transformer = this.transformer) != null &&
6227     (reducer = this.reducer) != null) {
6228     for (int b; (b = preSplit()) > 0;)
6229     (rights = new MapReduceMappingsTask<K,V,U>
6230     (map, this, b, rights, transformer, reducer)).fork();
6231     U r = null, u;
6232     V v;
6233     while ((v = advance()) != null) {
6234     if ((u = transformer.apply((K)nextKey, v)) != null)
6235     r = (r == null) ? u : reducer.apply(r, u);
6236     }
6237     result = r;
6238     CountedCompleter<?> c;
6239     for (c = firstComplete(); c != null; c = c.nextComplete()) {
6240     MapReduceMappingsTask<K,V,U>
6241     t = (MapReduceMappingsTask<K,V,U>)c,
6242     s = t.rights;
6243     while (s != null) {
6244     U tr, sr;
6245     if ((sr = s.result) != null)
6246     t.result = (((tr = t.result) == null) ? sr :
6247     reducer.apply(tr, sr));
6248     s = t.rights = s.nextRight;
6249     }
6250     }
6251     }
6252     }
6253     }
6254    
6255     @SuppressWarnings("serial") static final class MapReduceKeysToDoubleTask<K,V>
6256     extends Traverser<K,V,Double> {
6257     final ObjectToDouble<? super K> transformer;
6258     final DoubleByDoubleToDouble reducer;
6259     final double basis;
6260     double result;
6261     MapReduceKeysToDoubleTask<K,V> rights, nextRight;
6262     MapReduceKeysToDoubleTask
6263     (ConcurrentHashMap<K,V> m, Traverser<K,V,?> p, int b,
6264     MapReduceKeysToDoubleTask<K,V> nextRight,
6265     ObjectToDouble<? super K> transformer,
6266     double basis,
6267     DoubleByDoubleToDouble reducer) {
6268     super(m, p, b); this.nextRight = nextRight;
6269     this.transformer = transformer;
6270     this.basis = basis; this.reducer = reducer;
6271     }
6272     public final Double getRawResult() { return result; }
6273     @SuppressWarnings("unchecked") public final void compute() {
6274     final ObjectToDouble<? super K> transformer;
6275     final DoubleByDoubleToDouble reducer;
6276     if ((transformer = this.transformer) != null &&
6277     (reducer = this.reducer) != null) {
6278     double r = this.basis;
6279     for (int b; (b = preSplit()) > 0;)
6280     (rights = new MapReduceKeysToDoubleTask<K,V>
6281     (map, this, b, rights, transformer, r, reducer)).fork();
6282     while (advance() != null)
6283     r = reducer.apply(r, transformer.apply((K)nextKey));
6284     result = r;
6285     CountedCompleter<?> c;
6286     for (c = firstComplete(); c != null; c = c.nextComplete()) {
6287     MapReduceKeysToDoubleTask<K,V>
6288     t = (MapReduceKeysToDoubleTask<K,V>)c,
6289     s = t.rights;
6290     while (s != null) {
6291     t.result = reducer.apply(t.result, s.result);
6292     s = t.rights = s.nextRight;
6293     }
6294     }
6295     }
6296     }
6297     }
6298    
6299     @SuppressWarnings("serial") static final class MapReduceValuesToDoubleTask<K,V>
6300     extends Traverser<K,V,Double> {
6301     final ObjectToDouble<? super V> transformer;
6302     final DoubleByDoubleToDouble reducer;
6303     final double basis;
6304     double result;
6305     MapReduceValuesToDoubleTask<K,V> rights, nextRight;
6306     MapReduceValuesToDoubleTask
6307     (ConcurrentHashMap<K,V> m, Traverser<K,V,?> p, int b,
6308     MapReduceValuesToDoubleTask<K,V> nextRight,
6309     ObjectToDouble<? super V> transformer,
6310     double basis,
6311     DoubleByDoubleToDouble reducer) {
6312     super(m, p, b); this.nextRight = nextRight;
6313     this.transformer = transformer;
6314     this.basis = basis; this.reducer = reducer;
6315     }
6316     public final Double getRawResult() { return result; }
6317     @SuppressWarnings("unchecked") public final void compute() {
6318     final ObjectToDouble<? super V> transformer;
6319     final DoubleByDoubleToDouble reducer;
6320     if ((transformer = this.transformer) != null &&
6321     (reducer = this.reducer) != null) {
6322     double r = this.basis;
6323     for (int b; (b = preSplit()) > 0;)
6324     (rights = new MapReduceValuesToDoubleTask<K,V>
6325     (map, this, b, rights, transformer, r, reducer)).fork();
6326     V v;
6327     while ((v = advance()) != null)
6328     r = reducer.apply(r, transformer.apply(v));
6329     result = r;
6330     CountedCompleter<?> c;
6331     for (c = firstComplete(); c != null; c = c.nextComplete()) {
6332     MapReduceValuesToDoubleTask<K,V>
6333     t = (MapReduceValuesToDoubleTask<K,V>)c,
6334     s = t.rights;
6335     while (s != null) {
6336     t.result = reducer.apply(t.result, s.result);
6337     s = t.rights = s.nextRight;
6338     }
6339     }
6340     }
6341     }
6342     }
6343    
6344     @SuppressWarnings("serial") static final class MapReduceEntriesToDoubleTask<K,V>
6345     extends Traverser<K,V,Double> {
6346     final ObjectToDouble<Map.Entry<K,V>> transformer;
6347     final DoubleByDoubleToDouble reducer;
6348     final double basis;
6349     double result;
6350     MapReduceEntriesToDoubleTask<K,V> rights, nextRight;
6351     MapReduceEntriesToDoubleTask
6352     (ConcurrentHashMap<K,V> m, Traverser<K,V,?> p, int b,
6353     MapReduceEntriesToDoubleTask<K,V> nextRight,
6354     ObjectToDouble<Map.Entry<K,V>> transformer,
6355     double basis,
6356     DoubleByDoubleToDouble reducer) {
6357     super(m, p, b); this.nextRight = nextRight;
6358     this.transformer = transformer;
6359     this.basis = basis; this.reducer = reducer;
6360     }
6361     public final Double getRawResult() { return result; }
6362     @SuppressWarnings("unchecked") public final void compute() {
6363     final ObjectToDouble<Map.Entry<K,V>> transformer;
6364     final DoubleByDoubleToDouble reducer;
6365     if ((transformer = this.transformer) != null &&
6366     (reducer = this.reducer) != null) {
6367     double r = this.basis;
6368     for (int b; (b = preSplit()) > 0;)
6369     (rights = new MapReduceEntriesToDoubleTask<K,V>
6370     (map, this, b, rights, transformer, r, reducer)).fork();
6371     V v;
6372     while ((v = advance()) != null)
6373     r = reducer.apply(r, transformer.apply(entryFor((K)nextKey,
6374     v)));
6375     result = r;
6376     CountedCompleter<?> c;
6377     for (c = firstComplete(); c != null; c = c.nextComplete()) {
6378     MapReduceEntriesToDoubleTask<K,V>
6379     t = (MapReduceEntriesToDoubleTask<K,V>)c,
6380     s = t.rights;
6381     while (s != null) {
6382     t.result = reducer.apply(t.result, s.result);
6383     s = t.rights = s.nextRight;
6384     }
6385     }
6386     }
6387     }
6388     }
6389    
6390     @SuppressWarnings("serial") static final class MapReduceMappingsToDoubleTask<K,V>
6391     extends Traverser<K,V,Double> {
6392     final ObjectByObjectToDouble<? super K, ? super V> transformer;
6393     final DoubleByDoubleToDouble reducer;
6394     final double basis;
6395     double result;
6396     MapReduceMappingsToDoubleTask<K,V> rights, nextRight;
6397     MapReduceMappingsToDoubleTask
6398     (ConcurrentHashMap<K,V> m, Traverser<K,V,?> p, int b,
6399     MapReduceMappingsToDoubleTask<K,V> nextRight,
6400     ObjectByObjectToDouble<? super K, ? super V> transformer,
6401     double basis,
6402     DoubleByDoubleToDouble reducer) {
6403     super(m, p, b); this.nextRight = nextRight;
6404     this.transformer = transformer;
6405     this.basis = basis; this.reducer = reducer;
6406     }
6407     public final Double getRawResult() { return result; }
6408     @SuppressWarnings("unchecked") public final void compute() {
6409     final ObjectByObjectToDouble<? super K, ? super V> transformer;
6410     final DoubleByDoubleToDouble reducer;
6411     if ((transformer = this.transformer) != null &&
6412     (reducer = this.reducer) != null) {
6413     double r = this.basis;
6414     for (int b; (b = preSplit()) > 0;)
6415     (rights = new MapReduceMappingsToDoubleTask<K,V>
6416     (map, this, b, rights, transformer, r, reducer)).fork();
6417     V v;
6418     while ((v = advance()) != null)
6419     r = reducer.apply(r, transformer.apply((K)nextKey, v));
6420     result = r;
6421     CountedCompleter<?> c;
6422     for (c = firstComplete(); c != null; c = c.nextComplete()) {
6423     MapReduceMappingsToDoubleTask<K,V>
6424     t = (MapReduceMappingsToDoubleTask<K,V>)c,
6425     s = t.rights;
6426     while (s != null) {
6427     t.result = reducer.apply(t.result, s.result);
6428     s = t.rights = s.nextRight;
6429     }
6430     }
6431     }
6432     }
6433     }
6434    
6435     @SuppressWarnings("serial") static final class MapReduceKeysToLongTask<K,V>
6436     extends Traverser<K,V,Long> {
6437     final ObjectToLong<? super K> transformer;
6438     final LongByLongToLong reducer;
6439     final long basis;
6440     long result;
6441     MapReduceKeysToLongTask<K,V> rights, nextRight;
6442     MapReduceKeysToLongTask
6443     (ConcurrentHashMap<K,V> m, Traverser<K,V,?> p, int b,
6444     MapReduceKeysToLongTask<K,V> nextRight,
6445     ObjectToLong<? super K> transformer,
6446     long basis,
6447     LongByLongToLong reducer) {
6448     super(m, p, b); this.nextRight = nextRight;
6449     this.transformer = transformer;
6450     this.basis = basis; this.reducer = reducer;
6451     }
6452     public final Long getRawResult() { return result; }
6453     @SuppressWarnings("unchecked") public final void compute() {
6454     final ObjectToLong<? super K> transformer;
6455     final LongByLongToLong reducer;
6456     if ((transformer = this.transformer) != null &&
6457     (reducer = this.reducer) != null) {
6458     long r = this.basis;
6459     for (int b; (b = preSplit()) > 0;)
6460     (rights = new MapReduceKeysToLongTask<K,V>
6461     (map, this, b, rights, transformer, r, reducer)).fork();
6462     while (advance() != null)
6463     r = reducer.apply(r, transformer.apply((K)nextKey));
6464     result = r;
6465     CountedCompleter<?> c;
6466     for (c = firstComplete(); c != null; c = c.nextComplete()) {
6467     MapReduceKeysToLongTask<K,V>
6468     t = (MapReduceKeysToLongTask<K,V>)c,
6469     s = t.rights;
6470     while (s != null) {
6471     t.result = reducer.apply(t.result, s.result);
6472     s = t.rights = s.nextRight;
6473     }
6474     }
6475     }
6476     }
6477     }
6478    
6479     @SuppressWarnings("serial") static final class MapReduceValuesToLongTask<K,V>
6480     extends Traverser<K,V,Long> {
6481     final ObjectToLong<? super V> transformer;
6482     final LongByLongToLong reducer;
6483     final long basis;
6484     long result;
6485     MapReduceValuesToLongTask<K,V> rights, nextRight;
6486     MapReduceValuesToLongTask
6487     (ConcurrentHashMap<K,V> m, Traverser<K,V,?> p, int b,
6488     MapReduceValuesToLongTask<K,V> nextRight,
6489     ObjectToLong<? super V> transformer,
6490     long basis,
6491     LongByLongToLong reducer) {
6492     super(m, p, b); this.nextRight = nextRight;
6493     this.transformer = transformer;
6494     this.basis = basis; this.reducer = reducer;
6495     }
6496     public final Long getRawResult() { return result; }
6497     @SuppressWarnings("unchecked") public final void compute() {
6498     final ObjectToLong<? super V> transformer;
6499     final LongByLongToLong reducer;
6500     if ((transformer = this.transformer) != null &&
6501     (reducer = this.reducer) != null) {
6502     long r = this.basis;
6503     for (int b; (b = preSplit()) > 0;)
6504     (rights = new MapReduceValuesToLongTask<K,V>
6505     (map, this, b, rights, transformer, r, reducer)).fork();
6506     V v;
6507     while ((v = advance()) != null)
6508     r = reducer.apply(r, transformer.apply(v));
6509     result = r;
6510     CountedCompleter<?> c;
6511     for (c = firstComplete(); c != null; c = c.nextComplete()) {
6512     MapReduceValuesToLongTask<K,V>
6513     t = (MapReduceValuesToLongTask<K,V>)c,
6514     s = t.rights;
6515     while (s != null) {
6516     t.result = reducer.apply(t.result, s.result);
6517     s = t.rights = s.nextRight;
6518     }
6519     }
6520     }
6521     }
6522     }
6523    
6524     @SuppressWarnings("serial") static final class MapReduceEntriesToLongTask<K,V>
6525     extends Traverser<K,V,Long> {
6526     final ObjectToLong<Map.Entry<K,V>> transformer;
6527     final LongByLongToLong reducer;
6528     final long basis;
6529     long result;
6530     MapReduceEntriesToLongTask<K,V> rights, nextRight;
6531     MapReduceEntriesToLongTask
6532     (ConcurrentHashMap<K,V> m, Traverser<K,V,?> p, int b,
6533     MapReduceEntriesToLongTask<K,V> nextRight,
6534     ObjectToLong<Map.Entry<K,V>> transformer,
6535     long basis,
6536     LongByLongToLong reducer) {
6537     super(m, p, b); this.nextRight = nextRight;
6538     this.transformer = transformer;
6539     this.basis = basis; this.reducer = reducer;
6540     }
6541     public final Long getRawResult() { return result; }
6542     @SuppressWarnings("unchecked") public final void compute() {
6543     final ObjectToLong<Map.Entry<K,V>> transformer;
6544     final LongByLongToLong reducer;
6545     if ((transformer = this.transformer) != null &&
6546     (reducer = this.reducer) != null) {
6547     long r = this.basis;
6548     for (int b; (b = preSplit()) > 0;)
6549     (rights = new MapReduceEntriesToLongTask<K,V>
6550     (map, this, b, rights, transformer, r, reducer)).fork();
6551     V v;
6552     while ((v = advance()) != null)
6553     r = reducer.apply(r, transformer.apply(entryFor((K)nextKey,
6554     v)));
6555     result = r;
6556     CountedCompleter<?> c;
6557     for (c = firstComplete(); c != null; c = c.nextComplete()) {
6558     MapReduceEntriesToLongTask<K,V>
6559     t = (MapReduceEntriesToLongTask<K,V>)c,
6560     s = t.rights;
6561     while (s != null) {
6562     t.result = reducer.apply(t.result, s.result);
6563     s = t.rights = s.nextRight;
6564     }
6565     }
6566     }
6567     }
6568     }
6569    
6570     @SuppressWarnings("serial") static final class MapReduceMappingsToLongTask<K,V>
6571     extends Traverser<K,V,Long> {
6572     final ObjectByObjectToLong<? super K, ? super V> transformer;
6573     final LongByLongToLong reducer;
6574     final long basis;
6575     long result;
6576     MapReduceMappingsToLongTask<K,V> rights, nextRight;
6577     MapReduceMappingsToLongTask
6578     (ConcurrentHashMap<K,V> m, Traverser<K,V,?> p, int b,
6579     MapReduceMappingsToLongTask<K,V> nextRight,
6580     ObjectByObjectToLong<? super K, ? super V> transformer,
6581     long basis,
6582     LongByLongToLong reducer) {
6583     super(m, p, b); this.nextRight = nextRight;
6584     this.transformer = transformer;
6585     this.basis = basis; this.reducer = reducer;
6586     }
6587     public final Long getRawResult() { return result; }
6588     @SuppressWarnings("unchecked") public final void compute() {
6589     final ObjectByObjectToLong<? super K, ? super V> transformer;
6590     final LongByLongToLong reducer;
6591     if ((transformer = this.transformer) != null &&
6592     (reducer = this.reducer) != null) {
6593     long r = this.basis;
6594     for (int b; (b = preSplit()) > 0;)
6595     (rights = new MapReduceMappingsToLongTask<K,V>
6596     (map, this, b, rights, transformer, r, reducer)).fork();
6597     V v;
6598     while ((v = advance()) != null)
6599     r = reducer.apply(r, transformer.apply((K)nextKey, v));
6600     result = r;
6601     CountedCompleter<?> c;
6602     for (c = firstComplete(); c != null; c = c.nextComplete()) {
6603     MapReduceMappingsToLongTask<K,V>
6604     t = (MapReduceMappingsToLongTask<K,V>)c,
6605     s = t.rights;
6606     while (s != null) {
6607     t.result = reducer.apply(t.result, s.result);
6608     s = t.rights = s.nextRight;
6609     }
6610     }
6611     }
6612     }
6613     }
6614    
6615     @SuppressWarnings("serial") static final class MapReduceKeysToIntTask<K,V>
6616     extends Traverser<K,V,Integer> {
6617     final ObjectToInt<? super K> transformer;
6618     final IntByIntToInt reducer;
6619     final int basis;
6620     int result;
6621     MapReduceKeysToIntTask<K,V> rights, nextRight;
6622     MapReduceKeysToIntTask
6623     (ConcurrentHashMap<K,V> m, Traverser<K,V,?> p, int b,
6624     MapReduceKeysToIntTask<K,V> nextRight,
6625     ObjectToInt<? super K> transformer,
6626     int basis,
6627     IntByIntToInt reducer) {
6628     super(m, p, b); this.nextRight = nextRight;
6629     this.transformer = transformer;
6630     this.basis = basis; this.reducer = reducer;
6631     }
6632     public final Integer getRawResult() { return result; }
6633     @SuppressWarnings("unchecked") public final void compute() {
6634     final ObjectToInt<? super K> transformer;
6635     final IntByIntToInt reducer;
6636     if ((transformer = this.transformer) != null &&
6637     (reducer = this.reducer) != null) {
6638     int r = this.basis;
6639     for (int b; (b = preSplit()) > 0;)
6640     (rights = new MapReduceKeysToIntTask<K,V>
6641     (map, this, b, rights, transformer, r, reducer)).fork();
6642     while (advance() != null)
6643     r = reducer.apply(r, transformer.apply((K)nextKey));
6644     result = r;
6645     CountedCompleter<?> c;
6646     for (c = firstComplete(); c != null; c = c.nextComplete()) {
6647     MapReduceKeysToIntTask<K,V>
6648     t = (MapReduceKeysToIntTask<K,V>)c,
6649     s = t.rights;
6650     while (s != null) {
6651     t.result = reducer.apply(t.result, s.result);
6652     s = t.rights = s.nextRight;
6653     }
6654     }
6655     }
6656     }
6657     }
6658    
6659     @SuppressWarnings("serial") static final class MapReduceValuesToIntTask<K,V>
6660     extends Traverser<K,V,Integer> {
6661     final ObjectToInt<? super V> transformer;
6662     final IntByIntToInt reducer;
6663     final int basis;
6664     int result;
6665     MapReduceValuesToIntTask<K,V> rights, nextRight;
6666     MapReduceValuesToIntTask
6667     (ConcurrentHashMap<K,V> m, Traverser<K,V,?> p, int b,
6668     MapReduceValuesToIntTask<K,V> nextRight,
6669     ObjectToInt<? super V> transformer,
6670     int basis,
6671     IntByIntToInt reducer) {
6672     super(m, p, b); this.nextRight = nextRight;
6673     this.transformer = transformer;
6674     this.basis = basis; this.reducer = reducer;
6675     }
6676     public final Integer getRawResult() { return result; }
6677     @SuppressWarnings("unchecked") public final void compute() {
6678     final ObjectToInt<? super V> transformer;
6679     final IntByIntToInt reducer;
6680     if ((transformer = this.transformer) != null &&
6681     (reducer = this.reducer) != null) {
6682     int r = this.basis;
6683     for (int b; (b = preSplit()) > 0;)
6684     (rights = new MapReduceValuesToIntTask<K,V>
6685     (map, this, b, rights, transformer, r, reducer)).fork();
6686     V v;
6687     while ((v = advance()) != null)
6688     r = reducer.apply(r, transformer.apply(v));
6689     result = r;
6690     CountedCompleter<?> c;
6691     for (c = firstComplete(); c != null; c = c.nextComplete()) {
6692     MapReduceValuesToIntTask<K,V>
6693     t = (MapReduceValuesToIntTask<K,V>)c,
6694     s = t.rights;
6695     while (s != null) {
6696     t.result = reducer.apply(t.result, s.result);
6697     s = t.rights = s.nextRight;
6698     }
6699     }
6700     }
6701     }
6702     }
6703    
6704     @SuppressWarnings("serial") static final class MapReduceEntriesToIntTask<K,V>
6705     extends Traverser<K,V,Integer> {
6706     final ObjectToInt<Map.Entry<K,V>> transformer;
6707     final IntByIntToInt reducer;
6708     final int basis;
6709     int result;
6710     MapReduceEntriesToIntTask<K,V> rights, nextRight;
6711     MapReduceEntriesToIntTask
6712     (ConcurrentHashMap<K,V> m, Traverser<K,V,?> p, int b,
6713     MapReduceEntriesToIntTask<K,V> nextRight,
6714     ObjectToInt<Map.Entry<K,V>> transformer,
6715     int basis,
6716     IntByIntToInt reducer) {
6717     super(m, p, b); this.nextRight = nextRight;
6718     this.transformer = transformer;
6719     this.basis = basis; this.reducer = reducer;
6720     }
6721     public final Integer getRawResult() { return result; }
6722     @SuppressWarnings("unchecked") public final void compute() {
6723     final ObjectToInt<Map.Entry<K,V>> transformer;
6724     final IntByIntToInt reducer;
6725     if ((transformer = this.transformer) != null &&
6726     (reducer = this.reducer) != null) {
6727     int r = this.basis;
6728     for (int b; (b = preSplit()) > 0;)
6729     (rights = new MapReduceEntriesToIntTask<K,V>
6730     (map, this, b, rights, transformer, r, reducer)).fork();
6731     V v;
6732     while ((v = advance()) != null)
6733     r = reducer.apply(r, transformer.apply(entryFor((K)nextKey,
6734     v)));
6735     result = r;
6736     CountedCompleter<?> c;
6737     for (c = firstComplete(); c != null; c = c.nextComplete()) {
6738     MapReduceEntriesToIntTask<K,V>
6739     t = (MapReduceEntriesToIntTask<K,V>)c,
6740     s = t.rights;
6741     while (s != null) {
6742     t.result = reducer.apply(t.result, s.result);
6743     s = t.rights = s.nextRight;
6744     }
6745     }
6746     }
6747     }
6748     }
6749    
6750     @SuppressWarnings("serial") static final class MapReduceMappingsToIntTask<K,V>
6751     extends Traverser<K,V,Integer> {
6752     final ObjectByObjectToInt<? super K, ? super V> transformer;
6753     final IntByIntToInt reducer;
6754     final int basis;
6755     int result;
6756     MapReduceMappingsToIntTask<K,V> rights, nextRight;
6757     MapReduceMappingsToIntTask
6758     (ConcurrentHashMap<K,V> m, Traverser<K,V,?> p, int b,
6759     MapReduceMappingsToIntTask<K,V> nextRight,
6760     ObjectByObjectToInt<? super K, ? super V> transformer,
6761     int basis,
6762     IntByIntToInt reducer) {
6763     super(m, p, b); this.nextRight = nextRight;
6764     this.transformer = transformer;
6765     this.basis = basis; this.reducer = reducer;
6766     }
6767     public final Integer getRawResult() { return result; }
6768     @SuppressWarnings("unchecked") public final void compute() {
6769     final ObjectByObjectToInt<? super K, ? super V> transformer;
6770     final IntByIntToInt reducer;
6771     if ((transformer = this.transformer) != null &&
6772     (reducer = this.reducer) != null) {
6773     int r = this.basis;
6774     for (int b; (b = preSplit()) > 0;)
6775     (rights = new MapReduceMappingsToIntTask<K,V>
6776     (map, this, b, rights, transformer, r, reducer)).fork();
6777     V v;
6778     while ((v = advance()) != null)
6779     r = reducer.apply(r, transformer.apply((K)nextKey, v));
6780     result = r;
6781     CountedCompleter<?> c;
6782     for (c = firstComplete(); c != null; c = c.nextComplete()) {
6783     MapReduceMappingsToIntTask<K,V>
6784     t = (MapReduceMappingsToIntTask<K,V>)c,
6785     s = t.rights;
6786     while (s != null) {
6787     t.result = reducer.apply(t.result, s.result);
6788     s = t.rights = s.nextRight;
6789     }
6790     }
6791     }
6792     }
6793     }
6794    
6795     // Unsafe mechanics
6796     private static final sun.misc.Unsafe U;
6797     private static final long SIZECTL;
6798     private static final long TRANSFERINDEX;
6799     private static final long TRANSFERORIGIN;
6800     private static final long BASECOUNT;
6801     private static final long COUNTERBUSY;
6802     private static final long CELLVALUE;
6803     private static final long ABASE;
6804     private static final int ASHIFT;
6805    
6806     static {
6807     int ss;
6808     try {
6809     U = sun.misc.Unsafe.getUnsafe();
6810     Class<?> k = ConcurrentHashMap.class;
6811     SIZECTL = U.objectFieldOffset
6812     (k.getDeclaredField("sizeCtl"));
6813     TRANSFERINDEX = U.objectFieldOffset
6814     (k.getDeclaredField("transferIndex"));
6815     TRANSFERORIGIN = U.objectFieldOffset
6816     (k.getDeclaredField("transferOrigin"));
6817     BASECOUNT = U.objectFieldOffset
6818     (k.getDeclaredField("baseCount"));
6819     COUNTERBUSY = U.objectFieldOffset
6820     (k.getDeclaredField("counterBusy"));
6821     Class<?> ck = CounterCell.class;
6822     CELLVALUE = U.objectFieldOffset
6823     (ck.getDeclaredField("value"));
6824     Class<?> sc = Node[].class;
6825     ABASE = U.arrayBaseOffset(sc);
6826     ss = U.arrayIndexScale(sc);
6827     ASHIFT = 31 - Integer.numberOfLeadingZeros(ss);
6828     } catch (Exception e) {
6829     throw new Error(e);
6830     }
6831     if ((ss & (ss-1)) != 0)
6832     throw new Error("data type scale not a power of two");
6833     }
6834    
6835     }