ViewVC Help
View File | Revision Log | Show Annotations | Download File | Root Listing
root/jsr166/jsr166/src/jsr166e/ConcurrentHashMapV8.java
Revision: 1.49
Committed: Fri Jul 6 23:45:09 2012 UTC (11 years, 10 months ago) by jsr166
Branch: MAIN
Changes since 1.48: +2 -2 lines
Log Message:
whitespace

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 jsr166e;
8     import jsr166e.LongAdder;
9 dl 1.24 import java.util.Arrays;
10 dl 1.1 import java.util.Map;
11     import java.util.Set;
12     import java.util.Collection;
13     import java.util.AbstractMap;
14     import java.util.AbstractSet;
15     import java.util.AbstractCollection;
16     import java.util.Hashtable;
17     import java.util.HashMap;
18     import java.util.Iterator;
19     import java.util.Enumeration;
20     import java.util.ConcurrentModificationException;
21     import java.util.NoSuchElementException;
22     import java.util.concurrent.ConcurrentMap;
23 dl 1.37 import java.util.concurrent.ThreadLocalRandom;
24 dl 1.24 import java.util.concurrent.locks.LockSupport;
25 dl 1.38 import java.util.concurrent.locks.AbstractQueuedSynchronizer;
26 dl 1.1 import java.io.Serializable;
27    
28     /**
29     * A hash table supporting full concurrency of retrievals and
30     * high expected concurrency for updates. This class obeys the
31     * same functional specification as {@link java.util.Hashtable}, and
32     * includes versions of methods corresponding to each method of
33     * {@code Hashtable}. However, even though all operations are
34     * thread-safe, retrieval operations do <em>not</em> entail locking,
35     * and there is <em>not</em> any support for locking the entire table
36     * in a way that prevents all access. This class is fully
37     * interoperable with {@code Hashtable} in programs that rely on its
38     * thread safety but not on its synchronization details.
39     *
40     * <p> Retrieval operations (including {@code get}) generally do not
41     * block, so may overlap with update operations (including {@code put}
42     * and {@code remove}). Retrievals reflect the results of the most
43     * recently <em>completed</em> update operations holding upon their
44     * onset. For aggregate operations such as {@code putAll} and {@code
45     * clear}, concurrent retrievals may reflect insertion or removal of
46     * only some entries. Similarly, Iterators and Enumerations return
47     * elements reflecting the state of the hash table at some point at or
48     * since the creation of the iterator/enumeration. They do
49     * <em>not</em> throw {@link ConcurrentModificationException}.
50     * However, iterators are designed to be used by only one thread at a
51     * time. Bear in mind that the results of aggregate status methods
52     * including {@code size}, {@code isEmpty}, and {@code containsValue}
53     * are typically useful only when a map is not undergoing concurrent
54     * updates in other threads. Otherwise the results of these methods
55     * reflect transient states that may be adequate for monitoring
56 dl 1.16 * or estimation purposes, but not for program control.
57 dl 1.1 *
58 dl 1.16 * <p> The table is dynamically expanded when there are too many
59     * collisions (i.e., keys that have distinct hash codes but fall into
60     * the same slot modulo the table size), with the expected average
61 dl 1.24 * effect of maintaining roughly two bins per mapping (corresponding
62     * to a 0.75 load factor threshold for resizing). There may be much
63     * variance around this average as mappings are added and removed, but
64     * overall, this maintains a commonly accepted time/space tradeoff for
65     * hash tables. However, resizing this or any other kind of hash
66     * table may be a relatively slow operation. When possible, it is a
67     * good idea to provide a size estimate as an optional {@code
68 dl 1.16 * initialCapacity} constructor argument. An additional optional
69     * {@code loadFactor} constructor argument provides a further means of
70     * customizing initial table capacity by specifying the table density
71     * to be used in calculating the amount of space to allocate for the
72     * given number of elements. Also, for compatibility with previous
73     * versions of this class, constructors may optionally specify an
74     * expected {@code concurrencyLevel} as an additional hint for
75     * internal sizing. Note that using many keys with exactly the same
76 jsr166 1.31 * {@code hashCode()} is a sure way to slow down performance of any
77 dl 1.16 * hash table.
78 dl 1.1 *
79     * <p>This class and its views and iterators implement all of the
80     * <em>optional</em> methods of the {@link Map} and {@link Iterator}
81     * interfaces.
82     *
83     * <p> Like {@link Hashtable} but unlike {@link HashMap}, this class
84     * does <em>not</em> allow {@code null} to be used as a key or value.
85     *
86     * <p>This class is a member of the
87     * <a href="{@docRoot}/../technotes/guides/collections/index.html">
88     * Java Collections Framework</a>.
89     *
90     * <p><em>jsr166e note: This class is a candidate replacement for
91     * java.util.concurrent.ConcurrentHashMap.<em>
92     *
93 jsr166 1.22 * @since 1.5
94 dl 1.1 * @author Doug Lea
95     * @param <K> the type of keys maintained by this map
96     * @param <V> the type of mapped values
97     */
98     public class ConcurrentHashMapV8<K, V>
99     implements ConcurrentMap<K, V>, Serializable {
100     private static final long serialVersionUID = 7249069246763182397L;
101    
102     /**
103 dl 1.27 * A function computing a mapping from the given key to a value.
104     * This is a place-holder for an upcoming JDK8 interface.
105 dl 1.1 */
106     public static interface MappingFunction<K, V> {
107     /**
108 jsr166 1.43 * Returns a value for the given key, or null if there is no mapping.
109 dl 1.1 *
110     * @param key the (non-null) key
111 dl 1.41 * @return a value for the key, or null if none
112 dl 1.1 */
113     V map(K key);
114     }
115    
116 dl 1.27 /**
117     * A function computing a new mapping given a key and its current
118     * mapped value (or {@code null} if there is no current
119     * mapping). This is a place-holder for an upcoming JDK8
120     * interface.
121     */
122     public static interface RemappingFunction<K, V> {
123     /**
124     * Returns a new value given a key and its current value.
125     *
126     * @param key the (non-null) key
127     * @param value the current value, or null if there is no mapping
128 dl 1.41 * @return a value for the key, or null if none
129 dl 1.27 */
130     V remap(K key, V value);
131     }
132    
133 dl 1.41 /**
134     * A partitionable iterator. A Spliterator can be traversed
135     * directly, but can also be partitioned (before traversal) by
136     * creating another Spliterator that covers a non-overlapping
137     * portion of the elements, and so may be amenable to parallel
138     * execution.
139     *
140     * <p> This interface exports a subset of expected JDK8
141     * functionality.
142     *
143     * <p>Sample usage: Here is one (of the several) ways to compute
144     * the sum of the values held in a map using the ForkJoin
145     * framework. As illustrated here, Spliterators are well suited to
146     * designs in which a task repeatedly splits off half its work
147     * into forked subtasks until small enough to process directly,
148 jsr166 1.44 * and then joins these subtasks. Variants of this style can also
149     * be used in completion-based designs.
150 dl 1.41 *
151     * <pre>
152     * {@code ConcurrentHashMapV8<String, Long> m = ...
153     * // Uses parallel depth of log2 of size / (parallelism * slack of 8).
154     * int depth = 32 - Integer.numberOfLeadingZeros(m.size() / (aForkJoinPool.getParallelism() * 8));
155     * long sum = aForkJoinPool.invoke(new SumValues(m.valueSpliterator(), depth, null));
156     * // ...
157     * static class SumValues extends RecursiveTask<Long> {
158     * final Spliterator<Long> s;
159     * final int depth; // number of splits before processing
160     * final SumValues nextJoin; // records forked subtasks to join
161     * SumValues(Spliterator<Long> s, int depth, SumValues nextJoin) {
162     * this.s = s; this.depth = depth; this.nextJoin = nextJoin;
163     * }
164     * public Long compute() {
165     * long sum = 0;
166     * SumValues subtasks = null; // fork subtasks
167     * for (int d = depth - 1; d >= 0; --d)
168     * (subtasks = new SumValues(s.split(), d, subtasks)).fork();
169     * while (s.hasNext()) // directly process remaining elements
170     * sum += s.next();
171     * for (SumValues t = subtasks; t != null; t = t.nextJoin)
172     * sum += t.join(); // collect subtask results
173     * return sum;
174     * }
175     * }
176     * }</pre>
177     */
178     public static interface Spliterator<T> extends Iterator<T> {
179     /**
180     * Returns a Spliterator covering approximately half of the
181     * elements, guaranteed not to overlap with those subsequently
182     * returned by this Spliterator. After invoking this method,
183     * the current Spliterator will <em>not</em> produce any of
184     * the elements of the returned Spliterator, but the two
185     * Spliterators together will produce all of the elements that
186     * would have been produced by this Spliterator had this
187     * method not been called. The exact number of elements
188     * produced by the returned Spliterator is not guaranteed, and
189     * may be zero (i.e., with {@code hasNext()} reporting {@code
190     * false}) if this Spliterator cannot be further split.
191     *
192     * @return a Spliterator covering approximately half of the
193     * elements
194     * @throws IllegalStateException if this Spliterator has
195 jsr166 1.45 * already commenced traversing elements
196 dl 1.41 */
197     Spliterator<T> split();
198     }
199    
200 dl 1.1 /*
201     * Overview:
202     *
203     * The primary design goal of this hash table is to maintain
204     * concurrent readability (typically method get(), but also
205     * iterators and related methods) while minimizing update
206 dl 1.24 * contention. Secondary goals are to keep space consumption about
207     * the same or better than java.util.HashMap, and to support high
208     * initial insertion rates on an empty table by many threads.
209 dl 1.1 *
210     * Each key-value mapping is held in a Node. Because Node fields
211     * can contain special values, they are defined using plain Object
212     * types. Similarly in turn, all internal methods that use them
213 dl 1.14 * work off Object types. And similarly, so do the internal
214     * methods of auxiliary iterator and view classes. All public
215     * generic typed methods relay in/out of these internal methods,
216 dl 1.27 * supplying null-checks and casts as needed. This also allows
217     * many of the public methods to be factored into a smaller number
218     * of internal methods (although sadly not so for the five
219 dl 1.38 * variants of put-related operations). The validation-based
220     * approach explained below leads to a lot of code sprawl because
221     * retry-control precludes factoring into smaller methods.
222 dl 1.1 *
223     * The table is lazily initialized to a power-of-two size upon the
224 dl 1.38 * first insertion. Each bin in the table normally contains a
225     * list of Nodes (most often, the list has only zero or one Node).
226     * Table accesses require volatile/atomic reads, writes, and
227     * CASes. Because there is no other way to arrange this without
228     * adding further indirections, we use intrinsics
229     * (sun.misc.Unsafe) operations. The lists of nodes within bins
230     * are always accurately traversable under volatile reads, so long
231     * as lookups check hash code and non-nullness of value before
232     * checking key equality.
233 dl 1.24 *
234     * We use the top two bits of Node hash fields for control
235     * purposes -- they are available anyway because of addressing
236     * constraints. As explained further below, these top bits are
237 dl 1.27 * used as follows:
238 dl 1.24 * 00 - Normal
239     * 01 - Locked
240     * 11 - Locked and may have a thread waiting for lock
241     * 10 - Node is a forwarding node
242     *
243     * The lower 30 bits of each Node's hash field contain a
244 dl 1.38 * transformation of the key's hash code, except for forwarding
245     * nodes, for which the lower bits are zero (and so always have
246     * hash field == MOVED).
247 dl 1.14 *
248 dl 1.27 * Insertion (via put or its variants) of the first node in an
249 dl 1.14 * empty bin is performed by just CASing it to the bin. This is
250 dl 1.38 * by far the most common case for put operations under most
251     * key/hash distributions. Other update operations (insert,
252     * delete, and replace) require locks. We do not want to waste
253     * the space required to associate a distinct lock object with
254     * each bin, so instead use the first node of a bin list itself as
255     * a lock. Blocking support for these locks relies on the builtin
256     * "synchronized" monitors. However, we also need a tryLock
257     * construction, so we overlay these by using bits of the Node
258     * hash field for lock control (see above), and so normally use
259     * builtin monitors only for blocking and signalling using
260     * wait/notifyAll constructions. See Node.tryAwaitLock.
261 dl 1.24 *
262     * Using the first node of a list as a lock does not by itself
263     * suffice though: When a node is locked, any update must first
264     * validate that it is still the first node after locking it, and
265     * retry if not. Because new nodes are always appended to lists,
266     * once a node is first in a bin, it remains first until deleted
267 dl 1.27 * or the bin becomes invalidated (upon resizing). However,
268     * operations that only conditionally update may inspect nodes
269     * until the point of update. This is a converse of sorts to the
270     * lazy locking technique described by Herlihy & Shavit.
271 dl 1.14 *
272 dl 1.24 * The main disadvantage of per-bin locks is that other update
273 dl 1.14 * operations on other nodes in a bin list protected by the same
274     * lock can stall, for example when user equals() or mapping
275 dl 1.38 * functions take a long time. However, statistically, under
276     * random hash codes, this is not a common problem. Ideally, the
277     * frequency of nodes in bins follows a Poisson distribution
278 dl 1.14 * (http://en.wikipedia.org/wiki/Poisson_distribution) with a
279 dl 1.16 * parameter of about 0.5 on average, given the resizing threshold
280     * of 0.75, although with a large variance because of resizing
281     * granularity. Ignoring variance, the expected occurrences of
282     * list size k are (exp(-0.5) * pow(0.5, k) / factorial(k)). The
283 dl 1.38 * first values are:
284 dl 1.16 *
285 dl 1.38 * 0: 0.60653066
286     * 1: 0.30326533
287     * 2: 0.07581633
288     * 3: 0.01263606
289     * 4: 0.00157952
290     * 5: 0.00015795
291     * 6: 0.00001316
292     * 7: 0.00000094
293     * 8: 0.00000006
294     * more: less than 1 in ten million
295 dl 1.16 *
296     * Lock contention probability for two threads accessing distinct
297 dl 1.38 * elements is roughly 1 / (8 * #elements) under random hashes.
298     *
299     * Actual hash code distributions encountered in practice
300     * sometimes deviate significantly from uniform randomness. This
301     * includes the case when N > (1<<30), so some keys MUST collide.
302     * Similarly for dumb or hostile usages in which multiple keys are
303     * designed to have identical hash codes. Also, although we guard
304     * against the worst effects of this (see method spread), sets of
305     * hashes may differ only in bits that do not impact their bin
306     * index for a given power-of-two mask. So we use a secondary
307     * strategy that applies when the number of nodes in a bin exceeds
308     * a threshold, and at least one of the keys implements
309     * Comparable. These TreeBins use a balanced tree to hold nodes
310     * (a specialized form of red-black trees), bounding search time
311     * to O(log N). Each search step in a TreeBin is around twice as
312     * slow as in a regular list, but given that N cannot exceed
313     * (1<<64) (before running out of addresses) this bounds search
314     * steps, lock hold times, etc, to reasonable constants (roughly
315     * 100 nodes inspected per operation worst case) so long as keys
316     * are Comparable (which is very common -- String, Long, etc).
317     * TreeBin nodes (TreeNodes) also maintain the same "next"
318     * traversal pointers as regular nodes, so can be traversed in
319     * iterators in the same way.
320 dl 1.1 *
321 dl 1.38 * The table is resized when occupancy exceeds a percentage
322 dl 1.24 * threshold (nominally, 0.75, but see below). Only a single
323     * thread performs the resize (using field "sizeCtl", to arrange
324     * exclusion), but the table otherwise remains usable for reads
325     * and updates. Resizing proceeds by transferring bins, one by
326     * one, from the table to the next table. Because we are using
327     * power-of-two expansion, the elements from each bin must either
328     * stay at same index, or move with a power of two offset. We
329     * eliminate unnecessary node creation by catching cases where old
330     * nodes can be reused because their next fields won't change. On
331     * average, only about one-sixth of them need cloning when a table
332     * doubles. The nodes they replace will be garbage collectable as
333     * soon as they are no longer referenced by any reader thread that
334     * may be in the midst of concurrently traversing table. Upon
335     * transfer, the old table bin contains only a special forwarding
336     * node (with hash field "MOVED") that contains the next table as
337     * its key. On encountering a forwarding node, access and update
338     * operations restart, using the new table.
339     *
340     * Each bin transfer requires its bin lock. However, unlike other
341     * cases, a transfer can skip a bin if it fails to acquire its
342 dl 1.38 * lock, and revisit it later (unless it is a TreeBin). Method
343     * rebuild maintains a buffer of TRANSFER_BUFFER_SIZE bins that
344     * have been skipped because of failure to acquire a lock, and
345     * blocks only if none are available (i.e., only very rarely).
346     * The transfer operation must also ensure that all accessible
347     * bins in both the old and new table are usable by any traversal.
348     * When there are no lock acquisition failures, this is arranged
349     * simply by proceeding from the last bin (table.length - 1) up
350     * towards the first. Upon seeing a forwarding node, traversals
351     * (see class InternalIterator) arrange to move to the new table
352     * without revisiting nodes. However, when any node is skipped
353     * during a transfer, all earlier table bins may have become
354     * visible, so are initialized with a reverse-forwarding node back
355     * to the old table until the new ones are established. (This
356     * sometimes requires transiently locking a forwarding node, which
357     * is possible under the above encoding.) These more expensive
358 dl 1.24 * mechanics trigger only when necessary.
359 dl 1.14 *
360 dl 1.24 * The traversal scheme also applies to partial traversals of
361 dl 1.14 * ranges of bins (via an alternate InternalIterator constructor)
362 dl 1.41 * to support partitioned aggregate operations. Also, read-only
363     * operations give up if ever forwarded to a null table, which
364     * provides support for shutdown-style clearing, which is also not
365     * currently implemented.
366 dl 1.14 *
367     * Lazy table initialization minimizes footprint until first use,
368     * and also avoids resizings when the first operation is from a
369     * putAll, constructor with map argument, or deserialization.
370 dl 1.24 * These cases attempt to override the initial capacity settings,
371     * but harmlessly fail to take effect in cases of races.
372 dl 1.1 *
373     * The element count is maintained using a LongAdder, which avoids
374     * contention on updates but can encounter cache thrashing if read
375 dl 1.14 * too frequently during concurrent access. To avoid reading so
376 dl 1.27 * often, resizing is attempted either when a bin lock is
377     * contended, or upon adding to a bin already holding two or more
378     * nodes (checked before adding in the xIfAbsent methods, after
379     * adding in others). Under uniform hash distributions, the
380     * probability of this occurring at threshold is around 13%,
381     * meaning that only about 1 in 8 puts check threshold (and after
382     * resizing, many fewer do so). But this approximation has high
383     * variance for small table sizes, so we check on any collision
384     * for sizes <= 64. The bulk putAll operation further reduces
385     * contention by only committing count updates upon these size
386     * checks.
387 dl 1.14 *
388     * Maintaining API and serialization compatibility with previous
389     * versions of this class introduces several oddities. Mainly: We
390     * leave untouched but unused constructor arguments refering to
391 dl 1.24 * concurrencyLevel. We accept a loadFactor constructor argument,
392     * but apply it only to initial table capacity (which is the only
393     * time that we can guarantee to honor it.) We also declare an
394     * unused "Segment" class that is instantiated in minimal form
395     * only when serializing.
396 dl 1.1 */
397    
398     /* ---------------- Constants -------------- */
399    
400     /**
401 dl 1.16 * The largest possible table capacity. This value must be
402     * exactly 1<<30 to stay within Java array allocation and indexing
403 dl 1.24 * bounds for power of two table sizes, and is further required
404     * because the top two bits of 32bit hash fields are used for
405     * control purposes.
406 dl 1.1 */
407 dl 1.14 private static final int MAXIMUM_CAPACITY = 1 << 30;
408 dl 1.1
409     /**
410 dl 1.14 * The default initial table capacity. Must be a power of 2
411     * (i.e., at least 1) and at most MAXIMUM_CAPACITY.
412 dl 1.1 */
413 dl 1.14 private static final int DEFAULT_CAPACITY = 16;
414 dl 1.1
415     /**
416 dl 1.24 * The largest possible (non-power of two) array size.
417     * Needed by toArray and related methods.
418     */
419     static final int MAX_ARRAY_SIZE = Integer.MAX_VALUE - 8;
420    
421     /**
422     * The default concurrency level for this table. Unused but
423     * defined for compatibility with previous versions of this class.
424     */
425     private static final int DEFAULT_CONCURRENCY_LEVEL = 16;
426    
427     /**
428 dl 1.16 * The load factor for this table. Overrides of this value in
429     * constructors affect only the initial table capacity. The
430 dl 1.24 * actual floating point value isn't normally used -- it is
431     * simpler to use expressions such as {@code n - (n >>> 2)} for
432     * the associated resizing threshold.
433 dl 1.1 */
434 dl 1.16 private static final float LOAD_FACTOR = 0.75f;
435 dl 1.1
436     /**
437 dl 1.24 * The buffer size for skipped bins during transfers. The
438     * value is arbitrary but should be large enough to avoid
439     * most locking stalls during resizes.
440     */
441     private static final int TRANSFER_BUFFER_SIZE = 32;
442    
443 dl 1.38 /**
444     * The bin count threshold for using a tree rather than list for a
445     * bin. The value reflects the approximate break-even point for
446     * using tree-based operations.
447     */
448     private static final int TREE_THRESHOLD = 8;
449    
450 dl 1.24 /*
451     * Encodings for special uses of Node hash fields. See above for
452     * explanation.
453 dl 1.1 */
454 jsr166 1.35 static final int MOVED = 0x80000000; // hash field for forwarding nodes
455 dl 1.24 static final int LOCKED = 0x40000000; // set/tested only as a bit
456     static final int WAITING = 0xc0000000; // both bits set/tested together
457     static final int HASH_BITS = 0x3fffffff; // usable bits of normal node hash
458    
459     /* ---------------- Fields -------------- */
460    
461     /**
462     * The array of bins. Lazily initialized upon first insertion.
463     * Size is always a power of two. Accessed directly by iterators.
464     */
465     transient volatile Node[] table;
466 dl 1.14
467 dl 1.16 /**
468 dl 1.24 * The counter maintaining number of elements.
469 dl 1.16 */
470 dl 1.24 private transient final LongAdder counter;
471    
472     /**
473     * Table initialization and resizing control. When negative, the
474     * table is being initialized or resized. Otherwise, when table is
475     * null, holds the initial table size to use upon creation, or 0
476     * for default. After initialization, holds the next element count
477     * value upon which to resize the table.
478     */
479     private transient volatile int sizeCtl;
480    
481     // views
482     private transient KeySet<K,V> keySet;
483     private transient Values<K,V> values;
484     private transient EntrySet<K,V> entrySet;
485    
486     /** For serialization compatibility. Null unless serialized; see below */
487     private Segment<K,V>[] segments;
488 dl 1.16
489 dl 1.38 /* ---------------- Table element access -------------- */
490    
491     /*
492     * Volatile access methods are used for table elements as well as
493     * elements of in-progress next table while resizing. Uses are
494     * null checked by callers, and implicitly bounds-checked, relying
495     * on the invariants that tab arrays have non-zero size, and all
496     * indices are masked with (tab.length - 1) which is never
497     * negative and always less than length. Note that, to be correct
498     * wrt arbitrary concurrency errors by users, bounds checks must
499     * operate on local variables, which accounts for some odd-looking
500     * inline assignments below.
501     */
502    
503     static final Node tabAt(Node[] tab, int i) { // used by InternalIterator
504     return (Node)UNSAFE.getObjectVolatile(tab, ((long)i<<ASHIFT)+ABASE);
505     }
506    
507     private static final boolean casTabAt(Node[] tab, int i, Node c, Node v) {
508     return UNSAFE.compareAndSwapObject(tab, ((long)i<<ASHIFT)+ABASE, c, v);
509     }
510    
511     private static final void setTabAt(Node[] tab, int i, Node v) {
512     UNSAFE.putObjectVolatile(tab, ((long)i<<ASHIFT)+ABASE, v);
513     }
514    
515 dl 1.14 /* ---------------- Nodes -------------- */
516 dl 1.1
517     /**
518 dl 1.14 * Key-value entry. Note that this is never exported out as a
519 dl 1.41 * user-visible Map.Entry (see MapEntry below). Nodes with a hash
520     * field of MOVED are special, and do not contain user keys or
521     * values. Otherwise, keys are never null, and null val fields
522     * indicate that a node is in the process of being deleted or
523     * created. For purposes of read-only access, a key may be read
524     * before a val, but can only be used after checking val to be
525     * non-null.
526 dl 1.1 */
527 dl 1.38 static class Node {
528 dl 1.24 volatile int hash;
529 dl 1.14 final Object key;
530     volatile Object val;
531     volatile Node next;
532    
533     Node(int hash, Object key, Object val, Node next) {
534     this.hash = hash;
535     this.key = key;
536     this.val = val;
537     this.next = next;
538     }
539    
540 dl 1.24 /** CompareAndSet the hash field */
541     final boolean casHash(int cmp, int val) {
542     return UNSAFE.compareAndSwapInt(this, hashOffset, cmp, val);
543     }
544 dl 1.1
545 dl 1.24 /** The number of spins before blocking for a lock */
546     static final int MAX_SPINS =
547     Runtime.getRuntime().availableProcessors() > 1 ? 64 : 1;
548 dl 1.1
549 dl 1.24 /**
550     * Spins a while if LOCKED bit set and this node is the first
551     * of its bin, and then sets WAITING bits on hash field and
552     * blocks (once) if they are still set. It is OK for this
553     * method to return even if lock is not available upon exit,
554     * which enables these simple single-wait mechanics.
555     *
556     * The corresponding signalling operation is performed within
557     * callers: Upon detecting that WAITING has been set when
558     * unlocking lock (via a failed CAS from non-waiting LOCKED
559     * state), unlockers acquire the sync lock and perform a
560     * notifyAll.
561     */
562     final void tryAwaitLock(Node[] tab, int i) {
563     if (tab != null && i >= 0 && i < tab.length) { // bounds check
564 dl 1.37 int r = ThreadLocalRandom.current().nextInt(); // randomize spins
565 dl 1.24 int spins = MAX_SPINS, h;
566     while (tabAt(tab, i) == this && ((h = hash) & LOCKED) != 0) {
567     if (spins >= 0) {
568 dl 1.37 r ^= r << 1; r ^= r >>> 3; r ^= r << 10; // xorshift
569     if (r >= 0 && --spins == 0)
570     Thread.yield(); // yield before block
571 dl 1.24 }
572     else if (casHash(h, h | WAITING)) {
573 jsr166 1.26 synchronized (this) {
574 dl 1.24 if (tabAt(tab, i) == this &&
575     (hash & WAITING) == WAITING) {
576     try {
577     wait();
578     } catch (InterruptedException ie) {
579     Thread.currentThread().interrupt();
580     }
581     }
582     else
583     notifyAll(); // possibly won race vs signaller
584     }
585     break;
586     }
587     }
588     }
589     }
590 dl 1.1
591 dl 1.24 // Unsafe mechanics for casHash
592     private static final sun.misc.Unsafe UNSAFE;
593     private static final long hashOffset;
594 dl 1.1
595 dl 1.24 static {
596     try {
597     UNSAFE = getUnsafe();
598     Class<?> k = Node.class;
599     hashOffset = UNSAFE.objectFieldOffset
600     (k.getDeclaredField("hash"));
601     } catch (Exception e) {
602     throw new Error(e);
603     }
604     }
605     }
606 dl 1.1
607 dl 1.38 /* ---------------- TreeBins -------------- */
608    
609     /**
610     * Nodes for use in TreeBins
611     */
612     static final class TreeNode extends Node {
613     TreeNode parent; // red-black tree links
614     TreeNode left;
615     TreeNode right;
616     TreeNode prev; // needed to unlink next upon deletion
617     boolean red;
618    
619     TreeNode(int hash, Object key, Object val, Node next, TreeNode parent) {
620     super(hash, key, val, next);
621     this.parent = parent;
622     }
623     }
624 dl 1.1
625 dl 1.38 /**
626     * A specialized form of red-black tree for use in bins
627     * whose size exceeds a threshold.
628     *
629     * TreeBins use a special form of comparison for search and
630     * related operations (which is the main reason we cannot use
631     * existing collections such as TreeMaps). TreeBins contain
632     * Comparable elements, but may contain others, as well as
633     * elements that are Comparable but not necessarily Comparable<T>
634     * for the same T, so we cannot invoke compareTo among them. To
635     * handle this, the tree is ordered primarily by hash value, then
636     * by getClass().getName() order, and then by Comparator order
637     * among elements of the same class. On lookup at a node, if
638 dl 1.41 * elements are not comparable or compare as 0, both left and
639     * right children may need to be searched in the case of tied hash
640     * values. (This corresponds to the full list search that would be
641     * necessary if all elements were non-Comparable and had tied
642     * hashes.) The red-black balancing code is updated from
643     * pre-jdk-collections
644     * (http://gee.cs.oswego.edu/dl/classes/collections/RBCell.java)
645     * based in turn on Cormen, Leiserson, and Rivest "Introduction to
646     * Algorithms" (CLR).
647 dl 1.38 *
648     * TreeBins also maintain a separate locking discipline than
649     * regular bins. Because they are forwarded via special MOVED
650     * nodes at bin heads (which can never change once established),
651     * we cannot use use those nodes as locks. Instead, TreeBin
652     * extends AbstractQueuedSynchronizer to support a simple form of
653     * read-write lock. For update operations and table validation,
654     * the exclusive form of lock behaves in the same way as bin-head
655     * locks. However, lookups use shared read-lock mechanics to allow
656     * multiple readers in the absence of writers. Additionally,
657     * these lookups do not ever block: While the lock is not
658     * available, they proceed along the slow traversal path (via
659     * next-pointers) until the lock becomes available or the list is
660     * exhausted, whichever comes first. (These cases are not fast,
661     * but maximize aggregate expected throughput.) The AQS mechanics
662     * for doing this are straightforward. The lock state is held as
663     * AQS getState(). Read counts are negative; the write count (1)
664     * is positive. There are no signalling preferences among readers
665     * and writers. Since we don't need to export full Lock API, we
666     * just override the minimal AQS methods and use them directly.
667 dl 1.1 */
668 dl 1.38 static final class TreeBin extends AbstractQueuedSynchronizer {
669     private static final long serialVersionUID = 2249069246763182397L;
670 dl 1.41 transient TreeNode root; // root of tree
671     transient TreeNode first; // head of next-pointer list
672 dl 1.1
673 dl 1.38 /* AQS overrides */
674     public final boolean isHeldExclusively() { return getState() > 0; }
675     public final boolean tryAcquire(int ignore) {
676     if (compareAndSetState(0, 1)) {
677     setExclusiveOwnerThread(Thread.currentThread());
678     return true;
679     }
680     return false;
681     }
682     public final boolean tryRelease(int ignore) {
683     setExclusiveOwnerThread(null);
684     setState(0);
685     return true;
686     }
687     public final int tryAcquireShared(int ignore) {
688     for (int c;;) {
689     if ((c = getState()) > 0)
690     return -1;
691     if (compareAndSetState(c, c -1))
692     return 1;
693     }
694     }
695     public final boolean tryReleaseShared(int ignore) {
696     int c;
697     do {} while (!compareAndSetState(c = getState(), c + 1));
698     return c == -1;
699     }
700    
701 dl 1.41 /** From CLR */
702     private void rotateLeft(TreeNode p) {
703     if (p != null) {
704     TreeNode r = p.right, pp, rl;
705     if ((rl = p.right = r.left) != null)
706     rl.parent = p;
707     if ((pp = r.parent = p.parent) == null)
708     root = r;
709     else if (pp.left == p)
710     pp.left = r;
711     else
712     pp.right = r;
713     r.left = p;
714     p.parent = r;
715     }
716     }
717    
718     /** From CLR */
719     private void rotateRight(TreeNode p) {
720     if (p != null) {
721     TreeNode l = p.left, pp, lr;
722     if ((lr = p.left = l.right) != null)
723     lr.parent = p;
724     if ((pp = l.parent = p.parent) == null)
725     root = l;
726     else if (pp.right == p)
727     pp.right = l;
728     else
729     pp.left = l;
730     l.right = p;
731     p.parent = l;
732     }
733     }
734    
735 dl 1.38 /**
736     * Return the TreeNode (or null if not found) for the given key
737     * starting at given root.
738     */
739     @SuppressWarnings("unchecked") // suppress Comparable cast warning
740     final TreeNode getTreeNode(int h, Object k, TreeNode p) {
741     Class<?> c = k.getClass();
742     while (p != null) {
743 dl 1.41 int dir, ph; Object pk; Class<?> pc;
744     if ((ph = p.hash) == h) {
745     if ((pk = p.key) == k || k.equals(pk))
746     return p;
747     if (c != (pc = pk.getClass()) ||
748     !(k instanceof Comparable) ||
749     (dir = ((Comparable)k).compareTo((Comparable)pk)) == 0) {
750 jsr166 1.42 dir = (c == pc) ? 0 : c.getName().compareTo(pc.getName());
751 dl 1.41 TreeNode r = null, s = null, pl, pr;
752     if (dir >= 0) {
753     if ((pl = p.left) != null && h <= pl.hash)
754     s = pl;
755     }
756     else if ((pr = p.right) != null && h >= pr.hash)
757     s = pr;
758     if (s != null && (r = getTreeNode(h, k, s)) != null)
759     return r;
760     }
761     }
762 dl 1.38 else
763 dl 1.41 dir = (h < ph) ? -1 : 1;
764     p = (dir > 0) ? p.right : p.left;
765 dl 1.38 }
766     return null;
767     }
768    
769     /**
770     * Wrapper for getTreeNode used by CHM.get. Tries to obtain
771     * read-lock to call getTreeNode, but during failure to get
772     * lock, searches along next links.
773     */
774     final Object getValue(int h, Object k) {
775     Node r = null;
776     int c = getState(); // Must read lock state first
777     for (Node e = first; e != null; e = e.next) {
778     if (c <= 0 && compareAndSetState(c, c - 1)) {
779     try {
780     r = getTreeNode(h, k, root);
781     } finally {
782     releaseShared(0);
783     }
784     break;
785     }
786     else if ((e.hash & HASH_BITS) == h && k.equals(e.key)) {
787     r = e;
788     break;
789     }
790     else
791     c = getState();
792     }
793     return r == null ? null : r.val;
794     }
795    
796     /**
797 jsr166 1.45 * Finds or adds a node.
798 dl 1.38 * @return null if added
799     */
800     @SuppressWarnings("unchecked") // suppress Comparable cast warning
801     final TreeNode putTreeNode(int h, Object k, Object v) {
802     Class<?> c = k.getClass();
803 dl 1.41 TreeNode pp = root, p = null;
804 dl 1.38 int dir = 0;
805 dl 1.41 while (pp != null) { // find existing node or leaf to insert at
806     int ph; Object pk; Class<?> pc;
807     p = pp;
808     if ((ph = p.hash) == h) {
809     if ((pk = p.key) == k || k.equals(pk))
810 dl 1.38 return p;
811 dl 1.41 if (c != (pc = pk.getClass()) ||
812     !(k instanceof Comparable) ||
813     (dir = ((Comparable)k).compareTo((Comparable)pk)) == 0) {
814 jsr166 1.42 dir = (c == pc) ? 0 : c.getName().compareTo(pc.getName());
815 dl 1.41 TreeNode r = null, s = null, pl, pr;
816     if (dir >= 0) {
817     if ((pl = p.left) != null && h <= pl.hash)
818     s = pl;
819     }
820     else if ((pr = p.right) != null && h >= pr.hash)
821     s = pr;
822     if (s != null && (r = getTreeNode(h, k, s)) != null)
823     return r;
824 dl 1.38 }
825     }
826 dl 1.41 else
827     dir = (h < ph) ? -1 : 1;
828     pp = (dir > 0) ? p.right : p.left;
829 dl 1.38 }
830 dl 1.41
831 dl 1.38 TreeNode f = first;
832 dl 1.41 TreeNode x = first = new TreeNode(h, k, v, f, p);
833 dl 1.38 if (p == null)
834 dl 1.41 root = x;
835     else { // attach and rebalance; adapted from CLR
836     TreeNode xp, xpp;
837     if (f != null)
838     f.prev = x;
839 dl 1.38 if (dir <= 0)
840 dl 1.41 p.left = x;
841 dl 1.38 else
842 dl 1.41 p.right = x;
843     x.red = true;
844     while (x != null && (xp = x.parent) != null && xp.red &&
845     (xpp = xp.parent) != null) {
846     TreeNode xppl = xpp.left;
847     if (xp == xppl) {
848     TreeNode y = xpp.right;
849     if (y != null && y.red) {
850     y.red = false;
851     xp.red = false;
852     xpp.red = true;
853     x = xpp;
854     }
855     else {
856     if (x == xp.right) {
857     rotateLeft(x = xp);
858     xpp = (xp = x.parent) == null ? null : xp.parent;
859     }
860     if (xp != null) {
861     xp.red = false;
862     if (xpp != null) {
863     xpp.red = true;
864     rotateRight(xpp);
865     }
866     }
867     }
868     }
869     else {
870     TreeNode y = xppl;
871     if (y != null && y.red) {
872     y.red = false;
873     xp.red = false;
874     xpp.red = true;
875     x = xpp;
876     }
877     else {
878     if (x == xp.left) {
879     rotateRight(x = xp);
880     xpp = (xp = x.parent) == null ? null : xp.parent;
881     }
882     if (xp != null) {
883     xp.red = false;
884     if (xpp != null) {
885     xpp.red = true;
886     rotateLeft(xpp);
887     }
888     }
889     }
890     }
891     }
892     TreeNode r = root;
893     if (r != null && r.red)
894     r.red = false;
895 dl 1.38 }
896     return null;
897     }
898 dl 1.1
899 dl 1.38 /**
900     * Removes the given node, that must be present before this
901     * call. This is messier than typical red-black deletion code
902     * because we cannot swap the contents of an interior node
903     * with a leaf successor that is pinned by "next" pointers
904     * that are accessible independently of lock. So instead we
905     * swap the tree linkages.
906     */
907     final void deleteTreeNode(TreeNode p) {
908     TreeNode next = (TreeNode)p.next; // unlink traversal pointers
909     TreeNode pred = p.prev;
910     if (pred == null)
911     first = next;
912     else
913     pred.next = next;
914     if (next != null)
915     next.prev = pred;
916     TreeNode replacement;
917     TreeNode pl = p.left;
918     TreeNode pr = p.right;
919     if (pl != null && pr != null) {
920 dl 1.41 TreeNode s = pr, sl;
921     while ((sl = s.left) != null) // find successor
922     s = sl;
923 dl 1.38 boolean c = s.red; s.red = p.red; p.red = c; // swap colors
924     TreeNode sr = s.right;
925     TreeNode pp = p.parent;
926     if (s == pr) { // p was s's direct parent
927     p.parent = s;
928     s.right = p;
929     }
930     else {
931     TreeNode sp = s.parent;
932     if ((p.parent = sp) != null) {
933     if (s == sp.left)
934     sp.left = p;
935     else
936     sp.right = p;
937     }
938     if ((s.right = pr) != null)
939     pr.parent = s;
940     }
941     p.left = null;
942     if ((p.right = sr) != null)
943     sr.parent = p;
944     if ((s.left = pl) != null)
945     pl.parent = s;
946     if ((s.parent = pp) == null)
947     root = s;
948     else if (p == pp.left)
949     pp.left = s;
950     else
951     pp.right = s;
952     replacement = sr;
953     }
954     else
955     replacement = (pl != null) ? pl : pr;
956     TreeNode pp = p.parent;
957     if (replacement == null) {
958     if (pp == null) {
959     root = null;
960     return;
961     }
962     replacement = p;
963     }
964     else {
965     replacement.parent = pp;
966     if (pp == null)
967     root = replacement;
968     else if (p == pp.left)
969     pp.left = replacement;
970     else
971     pp.right = replacement;
972     p.left = p.right = p.parent = null;
973     }
974 dl 1.41 if (!p.red) { // rebalance, from CLR
975     TreeNode x = replacement;
976     while (x != null) {
977     TreeNode xp, xpl;
978     if (x.red || (xp = x.parent) == null) {
979     x.red = false;
980     break;
981 dl 1.38 }
982 dl 1.41 if (x == (xpl = xp.left)) {
983     TreeNode sib = xp.right;
984     if (sib != null && sib.red) {
985     sib.red = false;
986     xp.red = true;
987     rotateLeft(xp);
988     sib = (xp = x.parent) == null ? null : xp.right;
989 dl 1.38 }
990 dl 1.41 if (sib == null)
991 dl 1.38 x = xp;
992     else {
993 dl 1.41 TreeNode sl = sib.left, sr = sib.right;
994     if ((sr == null || !sr.red) &&
995     (sl == null || !sl.red)) {
996 dl 1.38 sib.red = true;
997 dl 1.41 x = xp;
998 dl 1.38 }
999 dl 1.41 else {
1000     if (sr == null || !sr.red) {
1001     if (sl != null)
1002     sl.red = false;
1003     sib.red = true;
1004     rotateRight(sib);
1005     sib = (xp = x.parent) == null ? null : xp.right;
1006     }
1007     if (sib != null) {
1008 jsr166 1.42 sib.red = (xp == null) ? false : xp.red;
1009 dl 1.41 if ((sr = sib.right) != null)
1010     sr.red = false;
1011     }
1012     if (xp != null) {
1013     xp.red = false;
1014     rotateLeft(xp);
1015     }
1016     x = root;
1017 dl 1.38 }
1018     }
1019     }
1020 dl 1.41 else { // symmetric
1021     TreeNode sib = xpl;
1022     if (sib != null && sib.red) {
1023     sib.red = false;
1024     xp.red = true;
1025     rotateRight(xp);
1026     sib = (xp = x.parent) == null ? null : xp.left;
1027     }
1028     if (sib == null)
1029 dl 1.38 x = xp;
1030     else {
1031 dl 1.41 TreeNode sl = sib.left, sr = sib.right;
1032     if ((sl == null || !sl.red) &&
1033     (sr == null || !sr.red)) {
1034 dl 1.38 sib.red = true;
1035 dl 1.41 x = xp;
1036 dl 1.38 }
1037 dl 1.41 else {
1038     if (sl == null || !sl.red) {
1039     if (sr != null)
1040     sr.red = false;
1041     sib.red = true;
1042     rotateLeft(sib);
1043     sib = (xp = x.parent) == null ? null : xp.left;
1044     }
1045     if (sib != null) {
1046 jsr166 1.42 sib.red = (xp == null) ? false : xp.red;
1047 dl 1.41 if ((sl = sib.left) != null)
1048     sl.red = false;
1049     }
1050     if (xp != null) {
1051     xp.red = false;
1052     rotateRight(xp);
1053     }
1054     x = root;
1055 dl 1.38 }
1056     }
1057     }
1058     }
1059     }
1060 dl 1.41 if (p == replacement && (pp = p.parent) != null) {
1061     if (p == pp.left) // detach pointers
1062     pp.left = null;
1063     else if (p == pp.right)
1064     pp.right = null;
1065     p.parent = null;
1066     }
1067 dl 1.38 }
1068 dl 1.1 }
1069    
1070 dl 1.38 /* ---------------- Collision reduction methods -------------- */
1071 dl 1.14
1072     /**
1073 dl 1.38 * Spreads higher bits to lower, and also forces top 2 bits to 0.
1074     * Because the table uses power-of-two masking, sets of hashes
1075     * that vary only in bits above the current mask will always
1076     * collide. (Among known examples are sets of Float keys holding
1077     * consecutive whole numbers in small tables.) To counter this,
1078     * we apply a transform that spreads the impact of higher bits
1079     * downward. There is a tradeoff between speed, utility, and
1080     * quality of bit-spreading. Because many common sets of hashes
1081 jsr166 1.40 * are already reasonably distributed across bits (so don't benefit
1082 dl 1.38 * from spreading), and because we use trees to handle large sets
1083     * of collisions in bins, we don't need excessively high quality.
1084 dl 1.14 */
1085     private static final int spread(int h) {
1086 dl 1.38 h ^= (h >>> 18) ^ (h >>> 12);
1087     return (h ^ (h >>> 10)) & HASH_BITS;
1088     }
1089    
1090     /**
1091     * Replaces a list bin with a tree bin. Call only when locked.
1092     * Fails to replace if the given key is non-comparable or table
1093     * is, or needs, resizing.
1094     */
1095     private final void replaceWithTreeBin(Node[] tab, int index, Object key) {
1096     if ((key instanceof Comparable) &&
1097     (tab.length >= MAXIMUM_CAPACITY || counter.sum() < (long)sizeCtl)) {
1098     TreeBin t = new TreeBin();
1099     for (Node e = tabAt(tab, index); e != null; e = e.next)
1100     t.putTreeNode(e.hash & HASH_BITS, e.key, e.val);
1101     setTabAt(tab, index, new Node(MOVED, t, null, null));
1102     }
1103 dl 1.14 }
1104 dl 1.1
1105 dl 1.38 /* ---------------- Internal access and update methods -------------- */
1106    
1107 dl 1.14 /** Implementation for get and containsKey */
1108 jsr166 1.4 private final Object internalGet(Object k) {
1109 dl 1.1 int h = spread(k.hashCode());
1110 dl 1.14 retry: for (Node[] tab = table; tab != null;) {
1111 dl 1.38 Node e, p; Object ek, ev; int eh; // locals to read fields once
1112 dl 1.14 for (e = tabAt(tab, (tab.length - 1) & h); e != null; e = e.next) {
1113 dl 1.24 if ((eh = e.hash) == MOVED) {
1114 dl 1.38 if ((ek = e.key) instanceof TreeBin) // search TreeBin
1115     return ((TreeBin)ek).getValue(h, k);
1116     else { // restart with new table
1117     tab = (Node[])ek;
1118     continue retry;
1119     }
1120 dl 1.1 }
1121 dl 1.38 else if ((eh & HASH_BITS) == h && (ev = e.val) != null &&
1122     ((ek = e.key) == k || k.equals(ek)))
1123 dl 1.24 return ev;
1124 dl 1.1 }
1125     break;
1126     }
1127     return null;
1128     }
1129    
1130 dl 1.27 /**
1131     * Implementation for the four public remove/replace methods:
1132     * Replaces node value with v, conditional upon match of cv if
1133     * non-null. If resulting value is null, delete.
1134     */
1135     private final Object internalReplace(Object k, Object v, Object cv) {
1136     int h = spread(k.hashCode());
1137     Object oldVal = null;
1138     for (Node[] tab = table;;) {
1139 dl 1.38 Node f; int i, fh; Object fk;
1140 dl 1.27 if (tab == null ||
1141     (f = tabAt(tab, i = (tab.length - 1) & h)) == null)
1142     break;
1143 dl 1.38 else if ((fh = f.hash) == MOVED) {
1144     if ((fk = f.key) instanceof TreeBin) {
1145     TreeBin t = (TreeBin)fk;
1146     boolean validated = false;
1147     boolean deleted = false;
1148     t.acquire(0);
1149     try {
1150     if (tabAt(tab, i) == f) {
1151     validated = true;
1152     TreeNode p = t.getTreeNode(h, k, t.root);
1153     if (p != null) {
1154     Object pv = p.val;
1155     if (cv == null || cv == pv || cv.equals(pv)) {
1156     oldVal = pv;
1157     if ((p.val = v) == null) {
1158     deleted = true;
1159     t.deleteTreeNode(p);
1160     }
1161     }
1162     }
1163     }
1164     } finally {
1165     t.release(0);
1166     }
1167     if (validated) {
1168     if (deleted)
1169     counter.add(-1L);
1170     break;
1171     }
1172     }
1173     else
1174     tab = (Node[])fk;
1175     }
1176 dl 1.27 else if ((fh & HASH_BITS) != h && f.next == null) // precheck
1177     break; // rules out possible existence
1178     else if ((fh & LOCKED) != 0) {
1179     checkForResize(); // try resizing if can't get lock
1180     f.tryAwaitLock(tab, i);
1181     }
1182     else if (f.casHash(fh, fh | LOCKED)) {
1183     boolean validated = false;
1184     boolean deleted = false;
1185     try {
1186     if (tabAt(tab, i) == f) {
1187     validated = true;
1188     for (Node e = f, pred = null;;) {
1189     Object ek, ev;
1190     if ((e.hash & HASH_BITS) == h &&
1191     ((ev = e.val) != null) &&
1192     ((ek = e.key) == k || k.equals(ek))) {
1193     if (cv == null || cv == ev || cv.equals(ev)) {
1194     oldVal = ev;
1195     if ((e.val = v) == null) {
1196     deleted = true;
1197     Node en = e.next;
1198     if (pred != null)
1199     pred.next = en;
1200     else
1201     setTabAt(tab, i, en);
1202     }
1203     }
1204     break;
1205     }
1206     pred = e;
1207     if ((e = e.next) == null)
1208     break;
1209     }
1210     }
1211     } finally {
1212     if (!f.casHash(fh | LOCKED, fh)) {
1213     f.hash = fh;
1214 jsr166 1.30 synchronized (f) { f.notifyAll(); };
1215 dl 1.27 }
1216     }
1217     if (validated) {
1218     if (deleted)
1219     counter.add(-1L);
1220     break;
1221     }
1222     }
1223     }
1224     return oldVal;
1225     }
1226    
1227     /*
1228     * Internal versions of the five insertion methods, each a
1229     * little more complicated than the last. All have
1230     * the same basic structure as the first (internalPut):
1231     * 1. If table uninitialized, create
1232     * 2. If bin empty, try to CAS new node
1233     * 3. If bin stale, use new table
1234 dl 1.38 * 4. if bin converted to TreeBin, validate and relay to TreeBin methods
1235     * 5. Lock and validate; if valid, scan and add or update
1236 dl 1.27 *
1237     * The others interweave other checks and/or alternative actions:
1238     * * Plain put checks for and performs resize after insertion.
1239     * * putIfAbsent prescans for mapping without lock (and fails to add
1240     * if present), which also makes pre-emptive resize checks worthwhile.
1241     * * computeIfAbsent extends form used in putIfAbsent with additional
1242     * mechanics to deal with, calls, potential exceptions and null
1243     * returns from function call.
1244     * * compute uses the same function-call mechanics, but without
1245     * the prescans
1246     * * putAll attempts to pre-allocate enough table space
1247     * and more lazily performs count updates and checks.
1248     *
1249     * Someday when details settle down a bit more, it might be worth
1250     * some factoring to reduce sprawl.
1251     */
1252    
1253     /** Implementation for put */
1254     private final Object internalPut(Object k, Object v) {
1255 dl 1.1 int h = spread(k.hashCode());
1256 dl 1.38 int count = 0;
1257 dl 1.14 for (Node[] tab = table;;) {
1258 dl 1.38 int i; Node f; int fh; Object fk;
1259 dl 1.1 if (tab == null)
1260 dl 1.24 tab = initTable();
1261     else if ((f = tabAt(tab, i = (tab.length - 1) & h)) == null) {
1262 dl 1.2 if (casTabAt(tab, i, null, new Node(h, k, v, null)))
1263 dl 1.14 break; // no lock when adding to empty bin
1264     }
1265 dl 1.38 else if ((fh = f.hash) == MOVED) {
1266     if ((fk = f.key) instanceof TreeBin) {
1267     TreeBin t = (TreeBin)fk;
1268     Object oldVal = null;
1269     t.acquire(0);
1270     try {
1271     if (tabAt(tab, i) == f) {
1272     count = 2;
1273     TreeNode p = t.putTreeNode(h, k, v);
1274     if (p != null) {
1275     oldVal = p.val;
1276     p.val = v;
1277     }
1278     }
1279     } finally {
1280     t.release(0);
1281     }
1282     if (count != 0) {
1283     if (oldVal != null)
1284     return oldVal;
1285     break;
1286     }
1287     }
1288     else
1289     tab = (Node[])fk;
1290     }
1291 dl 1.27 else if ((fh & LOCKED) != 0) {
1292     checkForResize();
1293     f.tryAwaitLock(tab, i);
1294 dl 1.1 }
1295 dl 1.24 else if (f.casHash(fh, fh | LOCKED)) {
1296 dl 1.27 Object oldVal = null;
1297     try { // needed in case equals() throws
1298 dl 1.24 if (tabAt(tab, i) == f) {
1299 dl 1.38 count = 1;
1300     for (Node e = f;; ++count) {
1301 dl 1.24 Object ek, ev;
1302     if ((e.hash & HASH_BITS) == h &&
1303     (ev = e.val) != null &&
1304     ((ek = e.key) == k || k.equals(ek))) {
1305 dl 1.1 oldVal = ev;
1306 dl 1.27 e.val = v;
1307 dl 1.10 break;
1308 dl 1.1 }
1309 dl 1.10 Node last = e;
1310     if ((e = e.next) == null) {
1311 dl 1.2 last.next = new Node(h, k, v, null);
1312 dl 1.38 if (count >= TREE_THRESHOLD)
1313     replaceWithTreeBin(tab, i, k);
1314 dl 1.10 break;
1315 dl 1.1 }
1316     }
1317     }
1318 dl 1.24 } finally { // unlock and signal if needed
1319     if (!f.casHash(fh | LOCKED, fh)) {
1320     f.hash = fh;
1321 jsr166 1.26 synchronized (f) { f.notifyAll(); };
1322 dl 1.24 }
1323 dl 1.1 }
1324 dl 1.38 if (count != 0) {
1325 dl 1.27 if (oldVal != null)
1326     return oldVal;
1327 dl 1.38 if (tab.length <= 64)
1328     count = 2;
1329 dl 1.1 break;
1330     }
1331     }
1332     }
1333 dl 1.27 counter.add(1L);
1334 dl 1.38 if (count > 1)
1335 dl 1.27 checkForResize();
1336     return null;
1337 dl 1.1 }
1338    
1339 dl 1.27 /** Implementation for putIfAbsent */
1340     private final Object internalPutIfAbsent(Object k, Object v) {
1341 dl 1.1 int h = spread(k.hashCode());
1342 dl 1.38 int count = 0;
1343 dl 1.14 for (Node[] tab = table;;) {
1344 dl 1.27 int i; Node f; int fh; Object fk, fv;
1345     if (tab == null)
1346     tab = initTable();
1347     else if ((f = tabAt(tab, i = (tab.length - 1) & h)) == null) {
1348     if (casTabAt(tab, i, null, new Node(h, k, v, null)))
1349     break;
1350     }
1351 dl 1.38 else if ((fh = f.hash) == MOVED) {
1352     if ((fk = f.key) instanceof TreeBin) {
1353     TreeBin t = (TreeBin)fk;
1354     Object oldVal = null;
1355     t.acquire(0);
1356     try {
1357     if (tabAt(tab, i) == f) {
1358     count = 2;
1359     TreeNode p = t.putTreeNode(h, k, v);
1360     if (p != null)
1361     oldVal = p.val;
1362     }
1363     } finally {
1364     t.release(0);
1365     }
1366     if (count != 0) {
1367     if (oldVal != null)
1368     return oldVal;
1369     break;
1370     }
1371     }
1372     else
1373     tab = (Node[])fk;
1374     }
1375 dl 1.27 else if ((fh & HASH_BITS) == h && (fv = f.val) != null &&
1376     ((fk = f.key) == k || k.equals(fk)))
1377     return fv;
1378     else {
1379     Node g = f.next;
1380     if (g != null) { // at least 2 nodes -- search and maybe resize
1381     for (Node e = g;;) {
1382     Object ek, ev;
1383     if ((e.hash & HASH_BITS) == h && (ev = e.val) != null &&
1384     ((ek = e.key) == k || k.equals(ek)))
1385     return ev;
1386     if ((e = e.next) == null) {
1387     checkForResize();
1388     break;
1389     }
1390     }
1391     }
1392     if (((fh = f.hash) & LOCKED) != 0) {
1393     checkForResize();
1394     f.tryAwaitLock(tab, i);
1395     }
1396     else if (tabAt(tab, i) == f && f.casHash(fh, fh | LOCKED)) {
1397     Object oldVal = null;
1398     try {
1399     if (tabAt(tab, i) == f) {
1400 dl 1.38 count = 1;
1401     for (Node e = f;; ++count) {
1402 dl 1.27 Object ek, ev;
1403     if ((e.hash & HASH_BITS) == h &&
1404     (ev = e.val) != null &&
1405     ((ek = e.key) == k || k.equals(ek))) {
1406 dl 1.1 oldVal = ev;
1407 dl 1.27 break;
1408     }
1409     Node last = e;
1410     if ((e = e.next) == null) {
1411     last.next = new Node(h, k, v, null);
1412 dl 1.38 if (count >= TREE_THRESHOLD)
1413     replaceWithTreeBin(tab, i, k);
1414 dl 1.27 break;
1415 dl 1.1 }
1416     }
1417 dl 1.27 }
1418     } finally {
1419     if (!f.casHash(fh | LOCKED, fh)) {
1420     f.hash = fh;
1421 jsr166 1.30 synchronized (f) { f.notifyAll(); };
1422 dl 1.24 }
1423     }
1424 dl 1.38 if (count != 0) {
1425 dl 1.27 if (oldVal != null)
1426     return oldVal;
1427 dl 1.38 if (tab.length <= 64)
1428     count = 2;
1429 dl 1.27 break;
1430     }
1431     }
1432     }
1433     }
1434     counter.add(1L);
1435 dl 1.38 if (count > 1)
1436     checkForResize();
1437 dl 1.27 return null;
1438     }
1439    
1440     /** Implementation for computeIfAbsent */
1441     private final Object internalComputeIfAbsent(K k,
1442     MappingFunction<? super K, ?> mf) {
1443     int h = spread(k.hashCode());
1444     Object val = null;
1445 dl 1.38 int count = 0;
1446 dl 1.27 for (Node[] tab = table;;) {
1447     Node f; int i, fh; Object fk, fv;
1448     if (tab == null)
1449     tab = initTable();
1450     else if ((f = tabAt(tab, i = (tab.length - 1) & h)) == null) {
1451     Node node = new Node(fh = h | LOCKED, k, null, null);
1452     if (casTabAt(tab, i, null, node)) {
1453 dl 1.38 count = 1;
1454 dl 1.27 try {
1455     if ((val = mf.map(k)) != null)
1456     node.val = val;
1457     } finally {
1458     if (val == null)
1459     setTabAt(tab, i, null);
1460     if (!node.casHash(fh, h)) {
1461     node.hash = h;
1462 jsr166 1.30 synchronized (node) { node.notifyAll(); };
1463 dl 1.27 }
1464 dl 1.1 }
1465     }
1466 dl 1.38 if (count != 0)
1467 dl 1.24 break;
1468 dl 1.27 }
1469 dl 1.38 else if ((fh = f.hash) == MOVED) {
1470     if ((fk = f.key) instanceof TreeBin) {
1471     TreeBin t = (TreeBin)fk;
1472     boolean added = false;
1473     t.acquire(0);
1474     try {
1475     if (tabAt(tab, i) == f) {
1476     count = 1;
1477     TreeNode p = t.getTreeNode(h, k, t.root);
1478     if (p != null)
1479     val = p.val;
1480     else if ((val = mf.map(k)) != null) {
1481     added = true;
1482     count = 2;
1483     t.putTreeNode(h, k, val);
1484     }
1485     }
1486     } finally {
1487     t.release(0);
1488     }
1489     if (count != 0) {
1490     if (!added)
1491     return val;
1492     break;
1493     }
1494     }
1495     else
1496     tab = (Node[])fk;
1497     }
1498 dl 1.27 else if ((fh & HASH_BITS) == h && (fv = f.val) != null &&
1499     ((fk = f.key) == k || k.equals(fk)))
1500     return fv;
1501     else {
1502     Node g = f.next;
1503     if (g != null) {
1504     for (Node e = g;;) {
1505     Object ek, ev;
1506     if ((e.hash & HASH_BITS) == h && (ev = e.val) != null &&
1507     ((ek = e.key) == k || k.equals(ek)))
1508     return ev;
1509     if ((e = e.next) == null) {
1510     checkForResize();
1511     break;
1512     }
1513     }
1514     }
1515     if (((fh = f.hash) & LOCKED) != 0) {
1516     checkForResize();
1517     f.tryAwaitLock(tab, i);
1518     }
1519     else if (tabAt(tab, i) == f && f.casHash(fh, fh | LOCKED)) {
1520 dl 1.38 boolean added = false;
1521 dl 1.27 try {
1522     if (tabAt(tab, i) == f) {
1523 dl 1.38 count = 1;
1524     for (Node e = f;; ++count) {
1525 dl 1.27 Object ek, ev;
1526     if ((e.hash & HASH_BITS) == h &&
1527     (ev = e.val) != null &&
1528     ((ek = e.key) == k || k.equals(ek))) {
1529     val = ev;
1530     break;
1531     }
1532     Node last = e;
1533     if ((e = e.next) == null) {
1534 dl 1.38 if ((val = mf.map(k)) != null) {
1535     added = true;
1536 dl 1.27 last.next = new Node(h, k, val, null);
1537 dl 1.38 if (count >= TREE_THRESHOLD)
1538     replaceWithTreeBin(tab, i, k);
1539     }
1540 dl 1.27 break;
1541     }
1542     }
1543     }
1544     } finally {
1545     if (!f.casHash(fh | LOCKED, fh)) {
1546     f.hash = fh;
1547 jsr166 1.30 synchronized (f) { f.notifyAll(); };
1548 dl 1.27 }
1549     }
1550 dl 1.38 if (count != 0) {
1551     if (!added)
1552     return val;
1553     if (tab.length <= 64)
1554     count = 2;
1555 dl 1.27 break;
1556 dl 1.38 }
1557 dl 1.1 }
1558     }
1559     }
1560 dl 1.41 if (val != null) {
1561     counter.add(1L);
1562     if (count > 1)
1563     checkForResize();
1564     }
1565 dl 1.27 return val;
1566 dl 1.1 }
1567    
1568 dl 1.27 /** Implementation for compute */
1569 dl 1.1 @SuppressWarnings("unchecked")
1570 dl 1.27 private final Object internalCompute(K k,
1571     RemappingFunction<? super K, V> mf) {
1572 dl 1.1 int h = spread(k.hashCode());
1573 dl 1.27 Object val = null;
1574 dl 1.41 int delta = 0;
1575 dl 1.38 int count = 0;
1576 dl 1.27 for (Node[] tab = table;;) {
1577 dl 1.38 Node f; int i, fh; Object fk;
1578 dl 1.1 if (tab == null)
1579 dl 1.24 tab = initTable();
1580     else if ((f = tabAt(tab, i = (tab.length - 1) & h)) == null) {
1581     Node node = new Node(fh = h | LOCKED, k, null, null);
1582     if (casTabAt(tab, i, null, node)) {
1583     try {
1584 dl 1.38 count = 1;
1585 dl 1.27 if ((val = mf.remap(k, null)) != null) {
1586 dl 1.24 node.val = val;
1587 dl 1.41 delta = 1;
1588 dl 1.24 }
1589     } finally {
1590 dl 1.41 if (delta == 0)
1591 dl 1.24 setTabAt(tab, i, null);
1592     if (!node.casHash(fh, h)) {
1593 dl 1.25 node.hash = h;
1594 jsr166 1.26 synchronized (node) { node.notifyAll(); };
1595 dl 1.1 }
1596     }
1597     }
1598 dl 1.38 if (count != 0)
1599 dl 1.10 break;
1600 dl 1.1 }
1601 dl 1.38 else if ((fh = f.hash) == MOVED) {
1602     if ((fk = f.key) instanceof TreeBin) {
1603     TreeBin t = (TreeBin)fk;
1604     t.acquire(0);
1605     try {
1606     if (tabAt(tab, i) == f) {
1607     count = 1;
1608     TreeNode p = t.getTreeNode(h, k, t.root);
1609 jsr166 1.39 Object pv = (p == null) ? null : p.val;
1610 dl 1.38 if ((val = mf.remap(k, (V)pv)) != null) {
1611     if (p != null)
1612     p.val = val;
1613     else {
1614     count = 2;
1615 dl 1.41 delta = 1;
1616 dl 1.38 t.putTreeNode(h, k, val);
1617     }
1618     }
1619 dl 1.41 else if (p != null) {
1620     delta = -1;
1621     t.deleteTreeNode(p);
1622     }
1623 dl 1.38 }
1624     } finally {
1625     t.release(0);
1626     }
1627     if (count != 0)
1628     break;
1629     }
1630     else
1631     tab = (Node[])fk;
1632     }
1633 dl 1.27 else if ((fh & LOCKED) != 0) {
1634     checkForResize();
1635     f.tryAwaitLock(tab, i);
1636 dl 1.14 }
1637 dl 1.24 else if (f.casHash(fh, fh | LOCKED)) {
1638     try {
1639     if (tabAt(tab, i) == f) {
1640 dl 1.38 count = 1;
1641 dl 1.41 for (Node e = f, pred = null;; ++count) {
1642 dl 1.27 Object ek, ev;
1643 dl 1.24 if ((e.hash & HASH_BITS) == h &&
1644     (ev = e.val) != null &&
1645     ((ek = e.key) == k || k.equals(ek))) {
1646 dl 1.27 val = mf.remap(k, (V)ev);
1647     if (val != null)
1648     e.val = val;
1649 dl 1.41 else {
1650     delta = -1;
1651     Node en = e.next;
1652     if (pred != null)
1653     pred.next = en;
1654     else
1655     setTabAt(tab, i, en);
1656     }
1657 dl 1.10 break;
1658 dl 1.1 }
1659 dl 1.41 pred = e;
1660 dl 1.10 if ((e = e.next) == null) {
1661 dl 1.27 if ((val = mf.remap(k, null)) != null) {
1662 dl 1.41 pred.next = new Node(h, k, val, null);
1663     delta = 1;
1664 dl 1.38 if (count >= TREE_THRESHOLD)
1665     replaceWithTreeBin(tab, i, k);
1666 dl 1.1 }
1667 dl 1.10 break;
1668 dl 1.1 }
1669     }
1670     }
1671 dl 1.24 } finally {
1672     if (!f.casHash(fh | LOCKED, fh)) {
1673     f.hash = fh;
1674 jsr166 1.26 synchronized (f) { f.notifyAll(); };
1675 dl 1.24 }
1676 dl 1.1 }
1677 dl 1.38 if (count != 0) {
1678     if (tab.length <= 64)
1679     count = 2;
1680 dl 1.10 break;
1681 dl 1.38 }
1682 dl 1.1 }
1683 dl 1.10 }
1684 dl 1.41 if (delta != 0) {
1685     counter.add((long)delta);
1686 dl 1.38 if (count > 1)
1687 dl 1.27 checkForResize();
1688     }
1689 dl 1.1 return val;
1690     }
1691    
1692 dl 1.27 /** Implementation for putAll */
1693     private final void internalPutAll(Map<?, ?> m) {
1694     tryPresize(m.size());
1695     long delta = 0L; // number of uncommitted additions
1696     boolean npe = false; // to throw exception on exit for nulls
1697     try { // to clean up counts on other exceptions
1698     for (Map.Entry<?, ?> entry : m.entrySet()) {
1699     Object k, v;
1700     if (entry == null || (k = entry.getKey()) == null ||
1701     (v = entry.getValue()) == null) {
1702     npe = true;
1703     break;
1704     }
1705     int h = spread(k.hashCode());
1706     for (Node[] tab = table;;) {
1707 dl 1.38 int i; Node f; int fh; Object fk;
1708 dl 1.27 if (tab == null)
1709     tab = initTable();
1710     else if ((f = tabAt(tab, i = (tab.length - 1) & h)) == null){
1711     if (casTabAt(tab, i, null, new Node(h, k, v, null))) {
1712     ++delta;
1713     break;
1714     }
1715     }
1716 dl 1.38 else if ((fh = f.hash) == MOVED) {
1717     if ((fk = f.key) instanceof TreeBin) {
1718     TreeBin t = (TreeBin)fk;
1719     boolean validated = false;
1720     t.acquire(0);
1721     try {
1722     if (tabAt(tab, i) == f) {
1723     validated = true;
1724     TreeNode p = t.getTreeNode(h, k, t.root);
1725     if (p != null)
1726     p.val = v;
1727     else {
1728     t.putTreeNode(h, k, v);
1729     ++delta;
1730     }
1731     }
1732     } finally {
1733     t.release(0);
1734     }
1735     if (validated)
1736     break;
1737     }
1738     else
1739     tab = (Node[])fk;
1740     }
1741 dl 1.27 else if ((fh & LOCKED) != 0) {
1742     counter.add(delta);
1743     delta = 0L;
1744     checkForResize();
1745     f.tryAwaitLock(tab, i);
1746     }
1747     else if (f.casHash(fh, fh | LOCKED)) {
1748 dl 1.38 int count = 0;
1749 dl 1.27 try {
1750     if (tabAt(tab, i) == f) {
1751 dl 1.38 count = 1;
1752     for (Node e = f;; ++count) {
1753 dl 1.27 Object ek, ev;
1754     if ((e.hash & HASH_BITS) == h &&
1755     (ev = e.val) != null &&
1756     ((ek = e.key) == k || k.equals(ek))) {
1757     e.val = v;
1758     break;
1759     }
1760     Node last = e;
1761     if ((e = e.next) == null) {
1762     ++delta;
1763     last.next = new Node(h, k, v, null);
1764 dl 1.38 if (count >= TREE_THRESHOLD)
1765     replaceWithTreeBin(tab, i, k);
1766 dl 1.27 break;
1767     }
1768     }
1769     }
1770     } finally {
1771     if (!f.casHash(fh | LOCKED, fh)) {
1772     f.hash = fh;
1773 jsr166 1.30 synchronized (f) { f.notifyAll(); };
1774 dl 1.27 }
1775     }
1776 dl 1.38 if (count != 0) {
1777     if (count > 1) {
1778 dl 1.27 counter.add(delta);
1779     delta = 0L;
1780     checkForResize();
1781 dl 1.1 }
1782 dl 1.27 break;
1783 dl 1.24 }
1784     }
1785 dl 1.1 }
1786     }
1787 dl 1.27 } finally {
1788     if (delta != 0)
1789     counter.add(delta);
1790 dl 1.1 }
1791 dl 1.27 if (npe)
1792     throw new NullPointerException();
1793 dl 1.1 }
1794    
1795 dl 1.27 /* ---------------- Table Initialization and Resizing -------------- */
1796 dl 1.24
1797     /**
1798     * Returns a power of two table size for the given desired capacity.
1799     * See Hackers Delight, sec 3.2
1800     */
1801     private static final int tableSizeFor(int c) {
1802     int n = c - 1;
1803     n |= n >>> 1;
1804     n |= n >>> 2;
1805     n |= n >>> 4;
1806     n |= n >>> 8;
1807     n |= n >>> 16;
1808     return (n < 0) ? 1 : (n >= MAXIMUM_CAPACITY) ? MAXIMUM_CAPACITY : n + 1;
1809     }
1810    
1811     /**
1812     * Initializes table, using the size recorded in sizeCtl.
1813     */
1814     private final Node[] initTable() {
1815     Node[] tab; int sc;
1816     while ((tab = table) == null) {
1817     if ((sc = sizeCtl) < 0)
1818     Thread.yield(); // lost initialization race; just spin
1819     else if (UNSAFE.compareAndSwapInt(this, sizeCtlOffset, sc, -1)) {
1820     try {
1821     if ((tab = table) == null) {
1822     int n = (sc > 0) ? sc : DEFAULT_CAPACITY;
1823     tab = table = new Node[n];
1824 dl 1.27 sc = n - (n >>> 2);
1825 dl 1.24 }
1826     } finally {
1827     sizeCtl = sc;
1828     }
1829     break;
1830     }
1831     }
1832     return tab;
1833     }
1834    
1835     /**
1836 dl 1.27 * If table is too small and not already resizing, creates next
1837     * table and transfers bins. Rechecks occupancy after a transfer
1838     * to see if another resize is already needed because resizings
1839     * are lagging additions.
1840     */
1841     private final void checkForResize() {
1842     Node[] tab; int n, sc;
1843     while ((tab = table) != null &&
1844     (n = tab.length) < MAXIMUM_CAPACITY &&
1845     (sc = sizeCtl) >= 0 && counter.sum() >= (long)sc &&
1846     UNSAFE.compareAndSwapInt(this, sizeCtlOffset, sc, -1)) {
1847 dl 1.24 try {
1848 dl 1.27 if (tab == table) {
1849 dl 1.24 table = rebuild(tab);
1850 dl 1.27 sc = (n << 1) - (n >>> 1);
1851 dl 1.24 }
1852     } finally {
1853     sizeCtl = sc;
1854     }
1855     }
1856     }
1857    
1858 dl 1.27 /**
1859     * Tries to presize table to accommodate the given number of elements.
1860     *
1861     * @param size number of elements (doesn't need to be perfectly accurate)
1862     */
1863     private final void tryPresize(int size) {
1864     int c = (size >= (MAXIMUM_CAPACITY >>> 1)) ? MAXIMUM_CAPACITY :
1865     tableSizeFor(size + (size >>> 1) + 1);
1866     int sc;
1867     while ((sc = sizeCtl) >= 0) {
1868     Node[] tab = table; int n;
1869     if (tab == null || (n = tab.length) == 0) {
1870 jsr166 1.30 n = (sc > c) ? sc : c;
1871 dl 1.27 if (UNSAFE.compareAndSwapInt(this, sizeCtlOffset, sc, -1)) {
1872     try {
1873     if (table == tab) {
1874     table = new Node[n];
1875     sc = n - (n >>> 2);
1876     }
1877     } finally {
1878     sizeCtl = sc;
1879     }
1880     }
1881     }
1882     else if (c <= sc || n >= MAXIMUM_CAPACITY)
1883     break;
1884     else if (UNSAFE.compareAndSwapInt(this, sizeCtlOffset, sc, -1)) {
1885     try {
1886     if (table == tab) {
1887     table = rebuild(tab);
1888     sc = (n << 1) - (n >>> 1);
1889     }
1890     } finally {
1891     sizeCtl = sc;
1892     }
1893     }
1894     }
1895     }
1896    
1897 dl 1.24 /*
1898     * Moves and/or copies the nodes in each bin to new table. See
1899     * above for explanation.
1900     *
1901     * @return the new table
1902     */
1903     private static final Node[] rebuild(Node[] tab) {
1904     int n = tab.length;
1905     Node[] nextTab = new Node[n << 1];
1906     Node fwd = new Node(MOVED, nextTab, null, null);
1907     int[] buffer = null; // holds bins to revisit; null until needed
1908     Node rev = null; // reverse forwarder; null until needed
1909     int nbuffered = 0; // the number of bins in buffer list
1910     int bufferIndex = 0; // buffer index of current buffered bin
1911     int bin = n - 1; // current non-buffered bin or -1 if none
1912    
1913     for (int i = bin;;) { // start upwards sweep
1914     int fh; Node f;
1915     if ((f = tabAt(tab, i)) == null) {
1916     if (bin >= 0) { // no lock needed (or available)
1917     if (!casTabAt(tab, i, f, fwd))
1918     continue;
1919     }
1920     else { // transiently use a locked forwarding node
1921 jsr166 1.33 Node g = new Node(MOVED|LOCKED, nextTab, null, null);
1922 dl 1.24 if (!casTabAt(tab, i, f, g))
1923     continue;
1924     setTabAt(nextTab, i, null);
1925     setTabAt(nextTab, i + n, null);
1926     setTabAt(tab, i, fwd);
1927     if (!g.casHash(MOVED|LOCKED, MOVED)) {
1928     g.hash = MOVED;
1929 jsr166 1.26 synchronized (g) { g.notifyAll(); }
1930 dl 1.24 }
1931     }
1932     }
1933 dl 1.38 else if ((fh = f.hash) == MOVED) {
1934     Object fk = f.key;
1935     if (fk instanceof TreeBin) {
1936     TreeBin t = (TreeBin)fk;
1937     boolean validated = false;
1938     t.acquire(0);
1939     try {
1940     if (tabAt(tab, i) == f) {
1941     validated = true;
1942     splitTreeBin(nextTab, i, t);
1943     setTabAt(tab, i, fwd);
1944     }
1945     } finally {
1946     t.release(0);
1947     }
1948     if (!validated)
1949     continue;
1950     }
1951     }
1952     else if ((fh & LOCKED) == 0 && f.casHash(fh, fh|LOCKED)) {
1953 dl 1.24 boolean validated = false;
1954     try { // split to lo and hi lists; copying as needed
1955     if (tabAt(tab, i) == f) {
1956     validated = true;
1957 dl 1.38 splitBin(nextTab, i, f);
1958 dl 1.24 setTabAt(tab, i, fwd);
1959     }
1960     } finally {
1961     if (!f.casHash(fh | LOCKED, fh)) {
1962     f.hash = fh;
1963 jsr166 1.26 synchronized (f) { f.notifyAll(); };
1964 dl 1.24 }
1965     }
1966     if (!validated)
1967     continue;
1968     }
1969     else {
1970     if (buffer == null) // initialize buffer for revisits
1971     buffer = new int[TRANSFER_BUFFER_SIZE];
1972     if (bin < 0 && bufferIndex > 0) {
1973     int j = buffer[--bufferIndex];
1974     buffer[bufferIndex] = i;
1975     i = j; // swap with another bin
1976     continue;
1977     }
1978     if (bin < 0 || nbuffered >= TRANSFER_BUFFER_SIZE) {
1979     f.tryAwaitLock(tab, i);
1980     continue; // no other options -- block
1981     }
1982     if (rev == null) // initialize reverse-forwarder
1983     rev = new Node(MOVED, tab, null, null);
1984     if (tabAt(tab, i) != f || (f.hash & LOCKED) == 0)
1985     continue; // recheck before adding to list
1986     buffer[nbuffered++] = i;
1987     setTabAt(nextTab, i, rev); // install place-holders
1988     setTabAt(nextTab, i + n, rev);
1989     }
1990    
1991     if (bin > 0)
1992     i = --bin;
1993     else if (buffer != null && nbuffered > 0) {
1994     bin = -1;
1995     i = buffer[bufferIndex = --nbuffered];
1996     }
1997     else
1998     return nextTab;
1999     }
2000     }
2001    
2002 dl 1.27 /**
2003 jsr166 1.45 * Splits a normal bin with list headed by e into lo and hi parts;
2004     * installs in given table.
2005 dl 1.38 */
2006     private static void splitBin(Node[] nextTab, int i, Node e) {
2007     int bit = nextTab.length >>> 1; // bit to split on
2008     int runBit = e.hash & bit;
2009     Node lastRun = e, lo = null, hi = null;
2010     for (Node p = e.next; p != null; p = p.next) {
2011     int b = p.hash & bit;
2012     if (b != runBit) {
2013     runBit = b;
2014     lastRun = p;
2015     }
2016     }
2017     if (runBit == 0)
2018     lo = lastRun;
2019     else
2020     hi = lastRun;
2021     for (Node p = e; p != lastRun; p = p.next) {
2022     int ph = p.hash & HASH_BITS;
2023     Object pk = p.key, pv = p.val;
2024     if ((ph & bit) == 0)
2025     lo = new Node(ph, pk, pv, lo);
2026     else
2027     hi = new Node(ph, pk, pv, hi);
2028     }
2029     setTabAt(nextTab, i, lo);
2030     setTabAt(nextTab, i + bit, hi);
2031     }
2032    
2033     /**
2034 jsr166 1.45 * Splits a tree bin into lo and hi parts; installs in given table.
2035 dl 1.38 */
2036     private static void splitTreeBin(Node[] nextTab, int i, TreeBin t) {
2037     int bit = nextTab.length >>> 1;
2038     TreeBin lt = new TreeBin();
2039     TreeBin ht = new TreeBin();
2040     int lc = 0, hc = 0;
2041     for (Node e = t.first; e != null; e = e.next) {
2042     int h = e.hash & HASH_BITS;
2043     Object k = e.key, v = e.val;
2044     if ((h & bit) == 0) {
2045     ++lc;
2046     lt.putTreeNode(h, k, v);
2047     }
2048     else {
2049     ++hc;
2050     ht.putTreeNode(h, k, v);
2051     }
2052     }
2053     Node ln, hn; // throw away trees if too small
2054     if (lc <= (TREE_THRESHOLD >>> 1)) {
2055     ln = null;
2056     for (Node p = lt.first; p != null; p = p.next)
2057     ln = new Node(p.hash, p.key, p.val, ln);
2058     }
2059     else
2060     ln = new Node(MOVED, lt, null, null);
2061     setTabAt(nextTab, i, ln);
2062     if (hc <= (TREE_THRESHOLD >>> 1)) {
2063     hn = null;
2064     for (Node p = ht.first; p != null; p = p.next)
2065     hn = new Node(p.hash, p.key, p.val, hn);
2066     }
2067     else
2068     hn = new Node(MOVED, ht, null, null);
2069     setTabAt(nextTab, i + bit, hn);
2070     }
2071    
2072     /**
2073 dl 1.27 * Implementation for clear. Steps through each bin, removing all
2074     * nodes.
2075     */
2076     private final void internalClear() {
2077     long delta = 0L; // negative number of deletions
2078     int i = 0;
2079     Node[] tab = table;
2080     while (tab != null && i < tab.length) {
2081 dl 1.38 int fh; Object fk;
2082 dl 1.27 Node f = tabAt(tab, i);
2083     if (f == null)
2084     ++i;
2085 dl 1.38 else if ((fh = f.hash) == MOVED) {
2086     if ((fk = f.key) instanceof TreeBin) {
2087     TreeBin t = (TreeBin)fk;
2088     t.acquire(0);
2089     try {
2090     if (tabAt(tab, i) == f) {
2091     for (Node p = t.first; p != null; p = p.next) {
2092     p.val = null;
2093     --delta;
2094     }
2095     t.first = null;
2096     t.root = null;
2097     ++i;
2098     }
2099     } finally {
2100     t.release(0);
2101     }
2102     }
2103     else
2104     tab = (Node[])fk;
2105     }
2106 dl 1.27 else if ((fh & LOCKED) != 0) {
2107     counter.add(delta); // opportunistically update count
2108     delta = 0L;
2109     f.tryAwaitLock(tab, i);
2110     }
2111     else if (f.casHash(fh, fh | LOCKED)) {
2112     try {
2113     if (tabAt(tab, i) == f) {
2114     for (Node e = f; e != null; e = e.next) {
2115 dl 1.38 e.val = null;
2116     --delta;
2117 dl 1.27 }
2118     setTabAt(tab, i, null);
2119 dl 1.38 ++i;
2120 dl 1.27 }
2121     } finally {
2122     if (!f.casHash(fh | LOCKED, fh)) {
2123     f.hash = fh;
2124 jsr166 1.30 synchronized (f) { f.notifyAll(); };
2125 dl 1.27 }
2126     }
2127     }
2128     }
2129     if (delta != 0)
2130     counter.add(delta);
2131     }
2132    
2133 dl 1.14 /* ----------------Table Traversal -------------- */
2134    
2135 dl 1.1 /**
2136 dl 1.14 * Encapsulates traversal for methods such as containsValue; also
2137     * serves as a base class for other iterators.
2138     *
2139     * At each step, the iterator snapshots the key ("nextKey") and
2140     * value ("nextVal") of a valid node (i.e., one that, at point of
2141 jsr166 1.36 * snapshot, has a non-null user value). Because val fields can
2142 dl 1.14 * change (including to null, indicating deletion), field nextVal
2143     * might not be accurate at point of use, but still maintains the
2144     * weak consistency property of holding a value that was once
2145     * valid.
2146     *
2147     * Internal traversals directly access these fields, as in:
2148 dl 1.41 * {@code while (it.advance() != null) { process(it.nextKey); }}
2149 dl 1.14 *
2150 dl 1.41 * Exported iterators must track whether the iterator has advanced
2151     * (in hasNext vs next) (by setting/checking/nulling field
2152     * nextVal), and then extract key, value, or key-value pairs as
2153     * return values of next().
2154 dl 1.14 *
2155 dl 1.27 * The iterator visits once each still-valid node that was
2156     * reachable upon iterator construction. It might miss some that
2157     * were added to a bin after the bin was visited, which is OK wrt
2158     * consistency guarantees. Maintaining this property in the face
2159     * of possible ongoing resizes requires a fair amount of
2160     * bookkeeping state that is difficult to optimize away amidst
2161     * volatile accesses. Even so, traversal maintains reasonable
2162     * throughput.
2163 dl 1.14 *
2164     * Normally, iteration proceeds bin-by-bin traversing lists.
2165     * However, if the table has been resized, then all future steps
2166     * must traverse both the bin at the current index as well as at
2167     * (index + baseSize); and so on for further resizings. To
2168     * paranoically cope with potential sharing by users of iterators
2169     * across threads, iteration terminates if a bounds checks fails
2170     * for a table read.
2171     */
2172 dl 1.41 static class InternalIterator<K,V> {
2173     final ConcurrentHashMapV8<K, V> map;
2174 dl 1.14 Node next; // the next entry to use
2175     Node last; // the last entry used
2176     Object nextKey; // cached key field of next
2177     Object nextVal; // cached val field of next
2178     Node[] tab; // current table; updated if resized
2179     int index; // index of bin to use next
2180     int baseIndex; // current index of initial table
2181 dl 1.41 int baseLimit; // index bound for initial table
2182 dl 1.14 final int baseSize; // initial table size
2183    
2184     /** Creates iterator for all entries in the table. */
2185 dl 1.41 InternalIterator(ConcurrentHashMapV8<K, V> map) {
2186     this.tab = (this.map = map).table;
2187 dl 1.14 baseLimit = baseSize = (tab == null) ? 0 : tab.length;
2188     }
2189    
2190 jsr166 1.47 /** Creates iterator for clone() and split() methods. */
2191 dl 1.41 InternalIterator(InternalIterator<K,V> it, boolean split) {
2192     this.map = it.map;
2193     this.tab = it.tab;
2194     this.baseSize = it.baseSize;
2195     int lo = it.baseIndex;
2196     int hi = this.baseLimit = it.baseLimit;
2197     this.index = this.baseIndex =
2198     (split) ? (it.baseLimit = (lo + hi + 1) >>> 1) : lo;
2199     }
2200    
2201     /**
2202 jsr166 1.48 * Advances next; returns nextVal or null if terminated.
2203 dl 1.41 * See above for explanation.
2204     */
2205     final Object advance() {
2206 dl 1.14 Node e = last = next;
2207 dl 1.41 Object ev = null;
2208 dl 1.14 outer: do {
2209 dl 1.24 if (e != null) // advance past used/skipped node
2210 dl 1.1 e = e.next;
2211 dl 1.24 while (e == null) { // get to next non-null bin
2212 dl 1.38 Node[] t; int b, i, n; Object ek; // checks must use locals
2213 dl 1.14 if ((b = baseIndex) >= baseLimit || (i = index) < 0 ||
2214     (t = tab) == null || i >= (n = t.length))
2215     break outer;
2216 dl 1.38 else if ((e = tabAt(t, i)) != null && e.hash == MOVED) {
2217     if ((ek = e.key) instanceof TreeBin)
2218     e = ((TreeBin)ek).first;
2219     else {
2220     tab = (Node[])ek;
2221     continue; // restarts due to null val
2222     }
2223     } // visit upper slots if present
2224     index = (i += baseSize) < n ? i : (baseIndex = b + 1);
2225 dl 1.1 }
2226 dl 1.14 nextKey = e.key;
2227 dl 1.41 } while ((ev = e.val) == null); // skip deleted or special nodes
2228 dl 1.14 next = e;
2229 dl 1.41 return nextVal = ev;
2230 dl 1.1 }
2231 dl 1.41
2232     public final void remove() {
2233     if (nextVal == null)
2234     advance();
2235     Node e = last;
2236     if (e == null)
2237     throw new IllegalStateException();
2238     last = null;
2239     map.remove(e.key);
2240     }
2241    
2242     public final boolean hasNext() {
2243     return nextVal != null || advance() != null;
2244     }
2245    
2246     public final boolean hasMoreElements() { return hasNext(); }
2247 dl 1.1 }
2248    
2249     /* ---------------- Public operations -------------- */
2250    
2251     /**
2252 jsr166 1.48 * Creates a new, empty map with the default initial table size (16).
2253 dl 1.1 */
2254 dl 1.16 public ConcurrentHashMapV8() {
2255 dl 1.14 this.counter = new LongAdder();
2256 dl 1.1 }
2257    
2258     /**
2259 dl 1.16 * Creates a new, empty map with an initial table size
2260     * accommodating the specified number of elements without the need
2261     * to dynamically resize.
2262 dl 1.1 *
2263     * @param initialCapacity The implementation performs internal
2264     * sizing to accommodate this many elements.
2265     * @throws IllegalArgumentException if the initial capacity of
2266 jsr166 1.18 * elements is negative
2267 dl 1.1 */
2268 dl 1.16 public ConcurrentHashMapV8(int initialCapacity) {
2269     if (initialCapacity < 0)
2270     throw new IllegalArgumentException();
2271     int cap = ((initialCapacity >= (MAXIMUM_CAPACITY >>> 1)) ?
2272     MAXIMUM_CAPACITY :
2273     tableSizeFor(initialCapacity + (initialCapacity >>> 1) + 1));
2274     this.counter = new LongAdder();
2275 dl 1.24 this.sizeCtl = cap;
2276 dl 1.1 }
2277    
2278     /**
2279 dl 1.16 * Creates a new map with the same mappings as the given map.
2280 dl 1.1 *
2281 dl 1.16 * @param m the map
2282 dl 1.1 */
2283 dl 1.16 public ConcurrentHashMapV8(Map<? extends K, ? extends V> m) {
2284     this.counter = new LongAdder();
2285 dl 1.24 this.sizeCtl = DEFAULT_CAPACITY;
2286 dl 1.27 internalPutAll(m);
2287 dl 1.1 }
2288    
2289     /**
2290 dl 1.16 * Creates a new, empty map with an initial table size based on
2291     * the given number of elements ({@code initialCapacity}) and
2292     * initial table density ({@code loadFactor}).
2293     *
2294     * @param initialCapacity the initial capacity. The implementation
2295     * performs internal sizing to accommodate this many elements,
2296     * given the specified load factor.
2297     * @param loadFactor the load factor (table density) for
2298 jsr166 1.18 * establishing the initial table size
2299 dl 1.16 * @throws IllegalArgumentException if the initial capacity of
2300     * elements is negative or the load factor is nonpositive
2301 jsr166 1.22 *
2302     * @since 1.6
2303 dl 1.1 */
2304 dl 1.16 public ConcurrentHashMapV8(int initialCapacity, float loadFactor) {
2305     this(initialCapacity, loadFactor, 1);
2306 dl 1.1 }
2307    
2308     /**
2309 dl 1.16 * Creates a new, empty map with an initial table size based on
2310     * the given number of elements ({@code initialCapacity}), table
2311     * density ({@code loadFactor}), and number of concurrently
2312     * updating threads ({@code concurrencyLevel}).
2313 dl 1.1 *
2314 dl 1.16 * @param initialCapacity the initial capacity. The implementation
2315     * performs internal sizing to accommodate this many elements,
2316     * given the specified load factor.
2317     * @param loadFactor the load factor (table density) for
2318 jsr166 1.18 * establishing the initial table size
2319 dl 1.16 * @param concurrencyLevel the estimated number of concurrently
2320     * updating threads. The implementation may use this value as
2321     * a sizing hint.
2322     * @throws IllegalArgumentException if the initial capacity is
2323     * negative or the load factor or concurrencyLevel are
2324 jsr166 1.18 * nonpositive
2325 dl 1.1 */
2326 dl 1.16 public ConcurrentHashMapV8(int initialCapacity,
2327     float loadFactor, int concurrencyLevel) {
2328     if (!(loadFactor > 0.0f) || initialCapacity < 0 || concurrencyLevel <= 0)
2329     throw new IllegalArgumentException();
2330     if (initialCapacity < concurrencyLevel) // Use at least as many bins
2331     initialCapacity = concurrencyLevel; // as estimated threads
2332     long size = (long)(1.0 + (long)initialCapacity / loadFactor);
2333 jsr166 1.49 int cap = (size >= (long)MAXIMUM_CAPACITY) ?
2334     MAXIMUM_CAPACITY : tableSizeFor((int)size);
2335 dl 1.16 this.counter = new LongAdder();
2336 dl 1.24 this.sizeCtl = cap;
2337 dl 1.1 }
2338    
2339     /**
2340 dl 1.14 * {@inheritDoc}
2341 dl 1.1 */
2342     public boolean isEmpty() {
2343 dl 1.2 return counter.sum() <= 0L; // ignore transient negative values
2344 dl 1.1 }
2345    
2346     /**
2347 dl 1.14 * {@inheritDoc}
2348 dl 1.1 */
2349     public int size() {
2350     long n = counter.sum();
2351 jsr166 1.15 return ((n < 0L) ? 0 :
2352     (n > (long)Integer.MAX_VALUE) ? Integer.MAX_VALUE :
2353 dl 1.14 (int)n);
2354 dl 1.1 }
2355    
2356 dl 1.24 final long longSize() { // accurate version of size needed for views
2357     long n = counter.sum();
2358     return (n < 0L) ? 0L : n;
2359     }
2360    
2361 dl 1.1 /**
2362     * Returns the value to which the specified key is mapped,
2363     * or {@code null} if this map contains no mapping for the key.
2364     *
2365     * <p>More formally, if this map contains a mapping from a key
2366     * {@code k} to a value {@code v} such that {@code key.equals(k)},
2367     * then this method returns {@code v}; otherwise it returns
2368     * {@code null}. (There can be at most one such mapping.)
2369     *
2370     * @throws NullPointerException if the specified key is null
2371     */
2372     @SuppressWarnings("unchecked")
2373     public V get(Object key) {
2374     if (key == null)
2375     throw new NullPointerException();
2376     return (V)internalGet(key);
2377     }
2378    
2379     /**
2380     * Tests if the specified object is a key in this table.
2381     *
2382     * @param key possible key
2383     * @return {@code true} if and only if the specified object
2384     * is a key in this table, as determined by the
2385 jsr166 1.18 * {@code equals} method; {@code false} otherwise
2386 dl 1.1 * @throws NullPointerException if the specified key is null
2387     */
2388     public boolean containsKey(Object key) {
2389     if (key == null)
2390     throw new NullPointerException();
2391     return internalGet(key) != null;
2392     }
2393    
2394     /**
2395     * Returns {@code true} if this map maps one or more keys to the
2396 dl 1.14 * specified value. Note: This method may require a full traversal
2397     * of the map, and is much slower than method {@code containsKey}.
2398 dl 1.1 *
2399     * @param value value whose presence in this map is to be tested
2400     * @return {@code true} if this map maps one or more keys to the
2401     * specified value
2402     * @throws NullPointerException if the specified value is null
2403     */
2404     public boolean containsValue(Object value) {
2405     if (value == null)
2406     throw new NullPointerException();
2407 dl 1.14 Object v;
2408 dl 1.41 InternalIterator<K,V> it = new InternalIterator<K,V>(this);
2409     while ((v = it.advance()) != null) {
2410     if (v == value || value.equals(v))
2411 dl 1.14 return true;
2412     }
2413     return false;
2414 dl 1.1 }
2415    
2416     /**
2417     * Legacy method testing if some key maps into the specified value
2418     * in this table. This method is identical in functionality to
2419     * {@link #containsValue}, and exists solely to ensure
2420     * full compatibility with class {@link java.util.Hashtable},
2421     * which supported this method prior to introduction of the
2422     * Java Collections framework.
2423     *
2424     * @param value a value to search for
2425     * @return {@code true} if and only if some key maps to the
2426     * {@code value} argument in this table as
2427     * determined by the {@code equals} method;
2428     * {@code false} otherwise
2429     * @throws NullPointerException if the specified value is null
2430     */
2431     public boolean contains(Object value) {
2432     return containsValue(value);
2433     }
2434    
2435     /**
2436     * Maps the specified key to the specified value in this table.
2437     * Neither the key nor the value can be null.
2438     *
2439     * <p> The value can be retrieved by calling the {@code get} method
2440     * with a key that is equal to the original key.
2441     *
2442     * @param key key with which the specified value is to be associated
2443     * @param value value to be associated with the specified key
2444     * @return the previous value associated with {@code key}, or
2445     * {@code null} if there was no mapping for {@code key}
2446     * @throws NullPointerException if the specified key or value is null
2447     */
2448     @SuppressWarnings("unchecked")
2449     public V put(K key, V value) {
2450     if (key == null || value == null)
2451     throw new NullPointerException();
2452 dl 1.27 return (V)internalPut(key, value);
2453 dl 1.1 }
2454    
2455     /**
2456     * {@inheritDoc}
2457     *
2458     * @return the previous value associated with the specified key,
2459     * or {@code null} if there was no mapping for the key
2460     * @throws NullPointerException if the specified key or value is null
2461     */
2462     @SuppressWarnings("unchecked")
2463     public V putIfAbsent(K key, V value) {
2464     if (key == null || value == null)
2465     throw new NullPointerException();
2466 dl 1.27 return (V)internalPutIfAbsent(key, value);
2467 dl 1.1 }
2468    
2469     /**
2470     * Copies all of the mappings from the specified map to this one.
2471     * These mappings replace any mappings that this map had for any of the
2472     * keys currently in the specified map.
2473     *
2474     * @param m mappings to be stored in this map
2475     */
2476     public void putAll(Map<? extends K, ? extends V> m) {
2477 dl 1.27 internalPutAll(m);
2478 dl 1.1 }
2479    
2480     /**
2481     * If the specified key is not already associated with a value,
2482 dl 1.41 * computes its value using the given mappingFunction and enters
2483     * it into the map unless null. This is equivalent to
2484 dl 1.27 * <pre> {@code
2485 jsr166 1.13 * if (map.containsKey(key))
2486     * return map.get(key);
2487     * value = mappingFunction.map(key);
2488 dl 1.41 * if (value != null)
2489     * map.put(key, value);
2490 jsr166 1.13 * return value;}</pre>
2491 dl 1.1 *
2492 dl 1.27 * except that the action is performed atomically. If the
2493 dl 1.41 * function returns {@code null} no mapping is recorded. If the
2494     * function itself throws an (unchecked) exception, the exception
2495     * is rethrown to its caller, and no mapping is recorded. Some
2496     * attempted update operations on this map by other threads may be
2497     * blocked while computation is in progress, so the computation
2498     * should be short and simple, and must not attempt to update any
2499     * other mappings of this Map. The most appropriate usage is to
2500     * construct a new object serving as an initial mapped value, or
2501     * memoized result, as in:
2502 dl 1.27 *
2503 jsr166 1.13 * <pre> {@code
2504 dl 1.5 * map.computeIfAbsent(key, new MappingFunction<K, V>() {
2505 jsr166 1.13 * public V map(K k) { return new Value(f(k)); }});}</pre>
2506 dl 1.1 *
2507     * @param key key with which the specified value is to be associated
2508     * @param mappingFunction the function to compute a value
2509     * @return the current (existing or computed) value associated with
2510 dl 1.41 * the specified key, or null if the computed value is null.
2511     * @throws NullPointerException if the specified key or mappingFunction
2512     * is null
2513 dl 1.5 * @throws IllegalStateException if the computation detectably
2514     * attempts a recursive update to this map that would
2515 jsr166 1.18 * otherwise never complete
2516 dl 1.1 * @throws RuntimeException or Error if the mappingFunction does so,
2517 jsr166 1.18 * in which case the mapping is left unestablished
2518 dl 1.1 */
2519 dl 1.27 @SuppressWarnings("unchecked")
2520 dl 1.1 public V computeIfAbsent(K key, MappingFunction<? super K, ? extends V> mappingFunction) {
2521     if (key == null || mappingFunction == null)
2522     throw new NullPointerException();
2523 dl 1.27 return (V)internalComputeIfAbsent(key, mappingFunction);
2524 dl 1.2 }
2525    
2526     /**
2527 dl 1.41 * Computes a new mapping value given a key and
2528 dl 1.27 * its current mapped value (or {@code null} if there is no current
2529     * mapping). This is equivalent to
2530 jsr166 1.13 * <pre> {@code
2531 dl 1.41 * value = remappingFunction.remap(key, map.get(key));
2532     * if (value != null)
2533     * map.put(key, value);
2534     * else
2535     * map.remove(key);
2536 dl 1.27 * }</pre>
2537 dl 1.2 *
2538 dl 1.27 * except that the action is performed atomically. If the
2539 dl 1.41 * function returns {@code null}, the mapping is removed. If the
2540     * function itself throws an (unchecked) exception, the exception
2541     * is rethrown to its caller, and the current mapping is left
2542     * unchanged. Some attempted update operations on this map by
2543     * other threads may be blocked while computation is in progress,
2544     * so the computation should be short and simple, and must not
2545     * attempt to update any other mappings of this Map. For example,
2546     * to either create or append new messages to a value mapping:
2547 dl 1.27 *
2548     * <pre> {@code
2549     * Map<Key, String> map = ...;
2550     * final String msg = ...;
2551     * map.compute(key, new RemappingFunction<Key, String>() {
2552     * public String remap(Key k, String v) {
2553 dl 1.28 * return (v == null) ? msg : v + msg;});}}</pre>
2554 dl 1.2 *
2555     * @param key key with which the specified value is to be associated
2556 dl 1.27 * @param remappingFunction the function to compute a value
2557     * @return the new value associated with
2558 dl 1.41 * the specified key, or null if none.
2559 dl 1.27 * @throws NullPointerException if the specified key or remappingFunction
2560 dl 1.41 * is null
2561 dl 1.5 * @throws IllegalStateException if the computation detectably
2562     * attempts a recursive update to this map that would
2563 jsr166 1.18 * otherwise never complete
2564 dl 1.29 * @throws RuntimeException or Error if the remappingFunction does so,
2565 jsr166 1.18 * in which case the mapping is unchanged
2566 dl 1.2 */
2567 dl 1.27 @SuppressWarnings("unchecked")
2568     public V compute(K key, RemappingFunction<? super K, V> remappingFunction) {
2569     if (key == null || remappingFunction == null)
2570 dl 1.2 throw new NullPointerException();
2571 dl 1.27 return (V)internalCompute(key, remappingFunction);
2572 dl 1.1 }
2573    
2574     /**
2575     * Removes the key (and its corresponding value) from this map.
2576     * This method does nothing if the key is not in the map.
2577     *
2578     * @param key the key that needs to be removed
2579     * @return the previous value associated with {@code key}, or
2580     * {@code null} if there was no mapping for {@code key}
2581     * @throws NullPointerException if the specified key is null
2582     */
2583     @SuppressWarnings("unchecked")
2584     public V remove(Object key) {
2585     if (key == null)
2586     throw new NullPointerException();
2587 jsr166 1.3 return (V)internalReplace(key, null, null);
2588 dl 1.1 }
2589    
2590     /**
2591     * {@inheritDoc}
2592     *
2593     * @throws NullPointerException if the specified key is null
2594     */
2595     public boolean remove(Object key, Object value) {
2596     if (key == null)
2597     throw new NullPointerException();
2598     if (value == null)
2599     return false;
2600     return internalReplace(key, null, value) != null;
2601     }
2602    
2603     /**
2604     * {@inheritDoc}
2605     *
2606     * @throws NullPointerException if any of the arguments are null
2607     */
2608     public boolean replace(K key, V oldValue, V newValue) {
2609     if (key == null || oldValue == null || newValue == null)
2610     throw new NullPointerException();
2611 jsr166 1.3 return internalReplace(key, newValue, oldValue) != null;
2612 dl 1.1 }
2613    
2614     /**
2615     * {@inheritDoc}
2616     *
2617     * @return the previous value associated with the specified key,
2618     * or {@code null} if there was no mapping for the key
2619     * @throws NullPointerException if the specified key or value is null
2620     */
2621     @SuppressWarnings("unchecked")
2622     public V replace(K key, V value) {
2623     if (key == null || value == null)
2624     throw new NullPointerException();
2625 jsr166 1.3 return (V)internalReplace(key, value, null);
2626 dl 1.1 }
2627    
2628     /**
2629     * Removes all of the mappings from this map.
2630     */
2631     public void clear() {
2632     internalClear();
2633     }
2634    
2635     /**
2636     * Returns a {@link Set} view of the keys contained in this map.
2637     * The set is backed by the map, so changes to the map are
2638     * reflected in the set, and vice-versa. The set supports element
2639     * removal, which removes the corresponding mapping from this map,
2640     * via the {@code Iterator.remove}, {@code Set.remove},
2641     * {@code removeAll}, {@code retainAll}, and {@code clear}
2642     * operations. It does not support the {@code add} or
2643     * {@code addAll} operations.
2644     *
2645     * <p>The view's {@code iterator} is a "weakly consistent" iterator
2646     * that will never throw {@link ConcurrentModificationException},
2647     * and guarantees to traverse elements as they existed upon
2648     * construction of the iterator, and may (but is not guaranteed to)
2649     * reflect any modifications subsequent to construction.
2650     */
2651     public Set<K> keySet() {
2652 dl 1.14 KeySet<K,V> ks = keySet;
2653     return (ks != null) ? ks : (keySet = new KeySet<K,V>(this));
2654 dl 1.1 }
2655    
2656     /**
2657     * Returns a {@link Collection} view of the values contained in this map.
2658     * The collection is backed by the map, so changes to the map are
2659     * reflected in the collection, and vice-versa. The collection
2660     * supports element removal, which removes the corresponding
2661     * mapping from this map, via the {@code Iterator.remove},
2662     * {@code Collection.remove}, {@code removeAll},
2663     * {@code retainAll}, and {@code clear} operations. It does not
2664     * support the {@code add} or {@code addAll} operations.
2665     *
2666     * <p>The view's {@code iterator} is a "weakly consistent" iterator
2667     * that will never throw {@link ConcurrentModificationException},
2668     * and guarantees to traverse elements as they existed upon
2669     * construction of the iterator, and may (but is not guaranteed to)
2670     * reflect any modifications subsequent to construction.
2671     */
2672     public Collection<V> values() {
2673 dl 1.14 Values<K,V> vs = values;
2674     return (vs != null) ? vs : (values = new Values<K,V>(this));
2675 dl 1.1 }
2676    
2677     /**
2678     * Returns a {@link Set} view of the mappings contained in this map.
2679     * The set is backed by the map, so changes to the map are
2680     * reflected in the set, and vice-versa. The set supports element
2681     * removal, which removes the corresponding mapping from the map,
2682     * via the {@code Iterator.remove}, {@code Set.remove},
2683     * {@code removeAll}, {@code retainAll}, and {@code clear}
2684     * operations. It does not support the {@code add} or
2685     * {@code addAll} operations.
2686     *
2687     * <p>The view's {@code iterator} is a "weakly consistent" iterator
2688     * that will never throw {@link ConcurrentModificationException},
2689     * and guarantees to traverse elements as they existed upon
2690     * construction of the iterator, and may (but is not guaranteed to)
2691     * reflect any modifications subsequent to construction.
2692     */
2693     public Set<Map.Entry<K,V>> entrySet() {
2694 dl 1.14 EntrySet<K,V> es = entrySet;
2695     return (es != null) ? es : (entrySet = new EntrySet<K,V>(this));
2696 dl 1.1 }
2697    
2698     /**
2699     * Returns an enumeration of the keys in this table.
2700     *
2701     * @return an enumeration of the keys in this table
2702     * @see #keySet()
2703     */
2704     public Enumeration<K> keys() {
2705 dl 1.14 return new KeyIterator<K,V>(this);
2706 dl 1.1 }
2707    
2708     /**
2709     * Returns an enumeration of the values in this table.
2710     *
2711     * @return an enumeration of the values in this table
2712     * @see #values()
2713     */
2714     public Enumeration<V> elements() {
2715 dl 1.14 return new ValueIterator<K,V>(this);
2716 dl 1.1 }
2717    
2718     /**
2719 dl 1.41 * Returns a partionable iterator of the keys in this map.
2720     *
2721     * @return a partionable iterator of the keys in this map
2722     */
2723     public Spliterator<K> keySpliterator() {
2724     return new KeyIterator<K,V>(this);
2725     }
2726    
2727     /**
2728     * Returns a partionable iterator of the values in this map.
2729     *
2730     * @return a partionable iterator of the values in this map
2731     */
2732     public Spliterator<V> valueSpliterator() {
2733     return new ValueIterator<K,V>(this);
2734     }
2735    
2736     /**
2737     * Returns a partionable iterator of the entries in this map.
2738     *
2739     * @return a partionable iterator of the entries in this map
2740     */
2741     public Spliterator<Map.Entry<K,V>> entrySpliterator() {
2742     return new EntryIterator<K,V>(this);
2743     }
2744    
2745     /**
2746 dl 1.2 * Returns the hash code value for this {@link Map}, i.e.,
2747     * the sum of, for each key-value pair in the map,
2748     * {@code key.hashCode() ^ value.hashCode()}.
2749     *
2750     * @return the hash code value for this map
2751 dl 1.1 */
2752     public int hashCode() {
2753 dl 1.14 int h = 0;
2754 dl 1.41 InternalIterator<K,V> it = new InternalIterator<K,V>(this);
2755     Object v;
2756     while ((v = it.advance()) != null) {
2757     h += it.nextKey.hashCode() ^ v.hashCode();
2758 dl 1.14 }
2759     return h;
2760 dl 1.1 }
2761    
2762     /**
2763 dl 1.2 * Returns a string representation of this map. The string
2764     * representation consists of a list of key-value mappings (in no
2765     * particular order) enclosed in braces ("{@code {}}"). Adjacent
2766     * mappings are separated by the characters {@code ", "} (comma
2767     * and space). Each key-value mapping is rendered as the key
2768     * followed by an equals sign ("{@code =}") followed by the
2769     * associated value.
2770     *
2771     * @return a string representation of this map
2772 dl 1.1 */
2773     public String toString() {
2774 dl 1.41 InternalIterator<K,V> it = new InternalIterator<K,V>(this);
2775 dl 1.14 StringBuilder sb = new StringBuilder();
2776     sb.append('{');
2777 dl 1.41 Object v;
2778     if ((v = it.advance()) != null) {
2779 dl 1.14 for (;;) {
2780 dl 1.41 Object k = it.nextKey;
2781 dl 1.14 sb.append(k == this ? "(this Map)" : k);
2782     sb.append('=');
2783     sb.append(v == this ? "(this Map)" : v);
2784 dl 1.41 if ((v = it.advance()) == null)
2785 dl 1.14 break;
2786     sb.append(',').append(' ');
2787     }
2788     }
2789     return sb.append('}').toString();
2790 dl 1.1 }
2791    
2792     /**
2793 dl 1.2 * Compares the specified object with this map for equality.
2794     * Returns {@code true} if the given object is a map with the same
2795     * mappings as this map. This operation may return misleading
2796     * results if either map is concurrently modified during execution
2797     * of this method.
2798     *
2799     * @param o object to be compared for equality with this map
2800     * @return {@code true} if the specified object is equal to this map
2801 dl 1.1 */
2802     public boolean equals(Object o) {
2803 dl 1.14 if (o != this) {
2804     if (!(o instanceof Map))
2805     return false;
2806     Map<?,?> m = (Map<?,?>) o;
2807 dl 1.41 InternalIterator<K,V> it = new InternalIterator<K,V>(this);
2808     Object val;
2809     while ((val = it.advance()) != null) {
2810 dl 1.14 Object v = m.get(it.nextKey);
2811     if (v == null || (v != val && !v.equals(val)))
2812 dl 1.1 return false;
2813 dl 1.14 }
2814 dl 1.1 for (Map.Entry<?,?> e : m.entrySet()) {
2815 dl 1.14 Object mk, mv, v;
2816     if ((mk = e.getKey()) == null ||
2817     (mv = e.getValue()) == null ||
2818     (v = internalGet(mk)) == null ||
2819     (mv != v && !mv.equals(v)))
2820 dl 1.1 return false;
2821     }
2822 dl 1.14 }
2823     return true;
2824     }
2825    
2826     /* ----------------Iterators -------------- */
2827    
2828 dl 1.41 static final class KeyIterator<K,V> extends InternalIterator<K,V>
2829     implements Spliterator<K>, Enumeration<K> {
2830     KeyIterator(ConcurrentHashMapV8<K, V> map) { super(map); }
2831     KeyIterator(InternalIterator<K,V> it, boolean split) {
2832     super(it, split);
2833     }
2834     public KeyIterator<K,V> split() {
2835     if (last != null || (next != null && nextVal == null))
2836     throw new IllegalStateException();
2837     return new KeyIterator<K,V>(this, true);
2838 dl 1.14 }
2839 dl 1.41 public KeyIterator<K,V> clone() {
2840     if (last != null || (next != null && nextVal == null))
2841 dl 1.14 throw new IllegalStateException();
2842 dl 1.41 return new KeyIterator<K,V>(this, false);
2843 dl 1.14 }
2844    
2845     @SuppressWarnings("unchecked")
2846     public final K next() {
2847 dl 1.41 if (nextVal == null && advance() == null)
2848 dl 1.14 throw new NoSuchElementException();
2849     Object k = nextKey;
2850 dl 1.41 nextVal = null;
2851     return (K) k;
2852 dl 1.14 }
2853    
2854     public final K nextElement() { return next(); }
2855     }
2856    
2857 dl 1.41 static final class ValueIterator<K,V> extends InternalIterator<K,V>
2858     implements Spliterator<V>, Enumeration<V> {
2859 dl 1.14 ValueIterator(ConcurrentHashMapV8<K, V> map) { super(map); }
2860 dl 1.41 ValueIterator(InternalIterator<K,V> it, boolean split) {
2861     super(it, split);
2862     }
2863     public ValueIterator<K,V> split() {
2864     if (last != null || (next != null && nextVal == null))
2865     throw new IllegalStateException();
2866     return new ValueIterator<K,V>(this, true);
2867     }
2868    
2869     public ValueIterator<K,V> clone() {
2870     if (last != null || (next != null && nextVal == null))
2871     throw new IllegalStateException();
2872     return new ValueIterator<K,V>(this, false);
2873     }
2874 dl 1.14
2875     @SuppressWarnings("unchecked")
2876     public final V next() {
2877 dl 1.41 Object v;
2878     if ((v = nextVal) == null && (v = advance()) == null)
2879 dl 1.14 throw new NoSuchElementException();
2880 dl 1.41 nextVal = null;
2881     return (V) v;
2882 dl 1.14 }
2883    
2884     public final V nextElement() { return next(); }
2885     }
2886    
2887 dl 1.41 static final class EntryIterator<K,V> extends InternalIterator<K,V>
2888     implements Spliterator<Map.Entry<K,V>> {
2889 dl 1.14 EntryIterator(ConcurrentHashMapV8<K, V> map) { super(map); }
2890 dl 1.41 EntryIterator(InternalIterator<K,V> it, boolean split) {
2891     super(it, split);
2892     }
2893     public EntryIterator<K,V> split() {
2894     if (last != null || (next != null && nextVal == null))
2895     throw new IllegalStateException();
2896     return new EntryIterator<K,V>(this, true);
2897     }
2898     public EntryIterator<K,V> clone() {
2899     if (last != null || (next != null && nextVal == null))
2900     throw new IllegalStateException();
2901     return new EntryIterator<K,V>(this, false);
2902 dl 1.24 }
2903    
2904     @SuppressWarnings("unchecked")
2905     public final Map.Entry<K,V> next() {
2906 dl 1.41 Object v;
2907     if ((v = nextVal) == null && (v = advance()) == null)
2908 dl 1.24 throw new NoSuchElementException();
2909     Object k = nextKey;
2910 dl 1.41 nextVal = null;
2911     return new MapEntry<K,V>((K)k, (V)v, map);
2912 dl 1.1 }
2913     }
2914    
2915     /**
2916 dl 1.41 * Exported Entry for iterators
2917 dl 1.1 */
2918 dl 1.41 static final class MapEntry<K,V> implements Map.Entry<K, V> {
2919 dl 1.14 final K key; // non-null
2920     V val; // non-null
2921 dl 1.41 final ConcurrentHashMapV8<K, V> map;
2922     MapEntry(K key, V val, ConcurrentHashMapV8<K, V> map) {
2923     this.key = key;
2924     this.val = val;
2925     this.map = map;
2926     }
2927 dl 1.14 public final K getKey() { return key; }
2928     public final V getValue() { return val; }
2929     public final int hashCode() { return key.hashCode() ^ val.hashCode(); }
2930     public final String toString(){ return key + "=" + val; }
2931    
2932     public final boolean equals(Object o) {
2933     Object k, v; Map.Entry<?,?> e;
2934     return ((o instanceof Map.Entry) &&
2935     (k = (e = (Map.Entry<?,?>)o).getKey()) != null &&
2936     (v = e.getValue()) != null &&
2937     (k == key || k.equals(key)) &&
2938     (v == val || v.equals(val)));
2939 dl 1.1 }
2940    
2941     /**
2942     * Sets our entry's value and writes through to the map. The
2943 dl 1.41 * value to return is somewhat arbitrary here. Since a we do
2944     * not necessarily track asynchronous changes, the most recent
2945     * "previous" value could be different from what we return (or
2946     * could even have been removed in which case the put will
2947     * re-establish). We do not and cannot guarantee more.
2948 dl 1.1 */
2949 dl 1.14 public final V setValue(V value) {
2950 dl 1.1 if (value == null) throw new NullPointerException();
2951 dl 1.14 V v = val;
2952     val = value;
2953     map.put(key, value);
2954 dl 1.1 return v;
2955     }
2956     }
2957    
2958 dl 1.14 /* ----------------Views -------------- */
2959 dl 1.1
2960 dl 1.24 /**
2961 dl 1.41 * Base class for views.
2962 dl 1.14 */
2963 dl 1.24 static abstract class MapView<K, V> {
2964 dl 1.14 final ConcurrentHashMapV8<K, V> map;
2965 dl 1.24 MapView(ConcurrentHashMapV8<K, V> map) { this.map = map; }
2966 dl 1.14 public final int size() { return map.size(); }
2967     public final boolean isEmpty() { return map.isEmpty(); }
2968     public final void clear() { map.clear(); }
2969 dl 1.24
2970     // implementations below rely on concrete classes supplying these
2971 dl 1.41 abstract public Iterator<?> iterator();
2972 dl 1.24 abstract public boolean contains(Object o);
2973     abstract public boolean remove(Object o);
2974    
2975     private static final String oomeMsg = "Required array size too large";
2976    
2977     public final Object[] toArray() {
2978     long sz = map.longSize();
2979     if (sz > (long)(MAX_ARRAY_SIZE))
2980     throw new OutOfMemoryError(oomeMsg);
2981     int n = (int)sz;
2982     Object[] r = new Object[n];
2983     int i = 0;
2984 dl 1.41 Iterator<?> it = iterator();
2985 dl 1.24 while (it.hasNext()) {
2986     if (i == n) {
2987     if (n >= MAX_ARRAY_SIZE)
2988     throw new OutOfMemoryError(oomeMsg);
2989     if (n >= MAX_ARRAY_SIZE - (MAX_ARRAY_SIZE >>> 1) - 1)
2990     n = MAX_ARRAY_SIZE;
2991     else
2992     n += (n >>> 1) + 1;
2993     r = Arrays.copyOf(r, n);
2994     }
2995     r[i++] = it.next();
2996     }
2997     return (i == n) ? r : Arrays.copyOf(r, i);
2998     }
2999    
3000     @SuppressWarnings("unchecked")
3001     public final <T> T[] toArray(T[] a) {
3002     long sz = map.longSize();
3003     if (sz > (long)(MAX_ARRAY_SIZE))
3004     throw new OutOfMemoryError(oomeMsg);
3005     int m = (int)sz;
3006     T[] r = (a.length >= m) ? a :
3007     (T[])java.lang.reflect.Array
3008     .newInstance(a.getClass().getComponentType(), m);
3009     int n = r.length;
3010     int i = 0;
3011 dl 1.41 Iterator<?> it = iterator();
3012 dl 1.24 while (it.hasNext()) {
3013     if (i == n) {
3014     if (n >= MAX_ARRAY_SIZE)
3015     throw new OutOfMemoryError(oomeMsg);
3016     if (n >= MAX_ARRAY_SIZE - (MAX_ARRAY_SIZE >>> 1) - 1)
3017     n = MAX_ARRAY_SIZE;
3018     else
3019     n += (n >>> 1) + 1;
3020     r = Arrays.copyOf(r, n);
3021     }
3022     r[i++] = (T)it.next();
3023     }
3024     if (a == r && i < n) {
3025     r[i] = null; // null-terminate
3026     return r;
3027     }
3028     return (i == n) ? r : Arrays.copyOf(r, i);
3029     }
3030    
3031     public final int hashCode() {
3032     int h = 0;
3033 dl 1.41 for (Iterator<?> it = iterator(); it.hasNext();)
3034 dl 1.24 h += it.next().hashCode();
3035     return h;
3036     }
3037    
3038     public final String toString() {
3039     StringBuilder sb = new StringBuilder();
3040     sb.append('[');
3041 dl 1.41 Iterator<?> it = iterator();
3042 dl 1.24 if (it.hasNext()) {
3043     for (;;) {
3044     Object e = it.next();
3045     sb.append(e == this ? "(this Collection)" : e);
3046     if (!it.hasNext())
3047     break;
3048     sb.append(',').append(' ');
3049     }
3050     }
3051     return sb.append(']').toString();
3052     }
3053    
3054     public final boolean containsAll(Collection<?> c) {
3055     if (c != this) {
3056     for (Iterator<?> it = c.iterator(); it.hasNext();) {
3057     Object e = it.next();
3058     if (e == null || !contains(e))
3059     return false;
3060     }
3061     }
3062     return true;
3063     }
3064    
3065 jsr166 1.32 public final boolean removeAll(Collection<?> c) {
3066 dl 1.24 boolean modified = false;
3067 dl 1.41 for (Iterator<?> it = iterator(); it.hasNext();) {
3068 dl 1.24 if (c.contains(it.next())) {
3069     it.remove();
3070     modified = true;
3071     }
3072     }
3073     return modified;
3074     }
3075    
3076     public final boolean retainAll(Collection<?> c) {
3077     boolean modified = false;
3078 dl 1.41 for (Iterator<?> it = iterator(); it.hasNext();) {
3079 dl 1.24 if (!c.contains(it.next())) {
3080     it.remove();
3081     modified = true;
3082     }
3083     }
3084     return modified;
3085     }
3086    
3087     }
3088    
3089     static final class KeySet<K,V> extends MapView<K,V> implements Set<K> {
3090     KeySet(ConcurrentHashMapV8<K, V> map) { super(map); }
3091 dl 1.14 public final boolean contains(Object o) { return map.containsKey(o); }
3092     public final boolean remove(Object o) { return map.remove(o) != null; }
3093     public final Iterator<K> iterator() {
3094     return new KeyIterator<K,V>(map);
3095 dl 1.1 }
3096 dl 1.24 public final boolean add(K e) {
3097     throw new UnsupportedOperationException();
3098     }
3099     public final boolean addAll(Collection<? extends K> c) {
3100     throw new UnsupportedOperationException();
3101     }
3102     public boolean equals(Object o) {
3103     Set<?> c;
3104     return ((o instanceof Set) &&
3105     ((c = (Set<?>)o) == this ||
3106     (containsAll(c) && c.containsAll(this))));
3107     }
3108 dl 1.1 }
3109    
3110 dl 1.24 static final class Values<K,V> extends MapView<K,V>
3111 jsr166 1.34 implements Collection<V> {
3112 dl 1.24 Values(ConcurrentHashMapV8<K, V> map) { super(map); }
3113     public final boolean contains(Object o) { return map.containsValue(o); }
3114     public final boolean remove(Object o) {
3115     if (o != null) {
3116     Iterator<V> it = new ValueIterator<K,V>(map);
3117     while (it.hasNext()) {
3118     if (o.equals(it.next())) {
3119     it.remove();
3120     return true;
3121     }
3122     }
3123     }
3124     return false;
3125     }
3126 dl 1.14 public final Iterator<V> iterator() {
3127     return new ValueIterator<K,V>(map);
3128 dl 1.1 }
3129 dl 1.24 public final boolean add(V e) {
3130     throw new UnsupportedOperationException();
3131     }
3132     public final boolean addAll(Collection<? extends V> c) {
3133     throw new UnsupportedOperationException();
3134     }
3135 dl 1.1 }
3136    
3137 jsr166 1.33 static final class EntrySet<K,V> extends MapView<K,V>
3138 dl 1.24 implements Set<Map.Entry<K,V>> {
3139     EntrySet(ConcurrentHashMapV8<K, V> map) { super(map); }
3140 dl 1.14 public final boolean contains(Object o) {
3141     Object k, v, r; Map.Entry<?,?> e;
3142     return ((o instanceof Map.Entry) &&
3143     (k = (e = (Map.Entry<?,?>)o).getKey()) != null &&
3144     (r = map.get(k)) != null &&
3145     (v = e.getValue()) != null &&
3146     (v == r || v.equals(r)));
3147 dl 1.1 }
3148 dl 1.14 public final boolean remove(Object o) {
3149     Object k, v; Map.Entry<?,?> e;
3150     return ((o instanceof Map.Entry) &&
3151     (k = (e = (Map.Entry<?,?>)o).getKey()) != null &&
3152     (v = e.getValue()) != null &&
3153     map.remove(k, v));
3154 dl 1.1 }
3155 dl 1.24 public final Iterator<Map.Entry<K,V>> iterator() {
3156     return new EntryIterator<K,V>(map);
3157     }
3158     public final boolean add(Entry<K,V> e) {
3159     throw new UnsupportedOperationException();
3160     }
3161     public final boolean addAll(Collection<? extends Entry<K,V>> c) {
3162     throw new UnsupportedOperationException();
3163     }
3164     public boolean equals(Object o) {
3165     Set<?> c;
3166     return ((o instanceof Set) &&
3167     ((c = (Set<?>)o) == this ||
3168     (containsAll(c) && c.containsAll(this))));
3169     }
3170 dl 1.1 }
3171    
3172     /* ---------------- Serialization Support -------------- */
3173    
3174     /**
3175 dl 1.14 * Stripped-down version of helper class used in previous version,
3176     * declared for the sake of serialization compatibility
3177 dl 1.1 */
3178 dl 1.14 static class Segment<K,V> implements Serializable {
3179 dl 1.1 private static final long serialVersionUID = 2249069246763182397L;
3180     final float loadFactor;
3181     Segment(float lf) { this.loadFactor = lf; }
3182     }
3183    
3184     /**
3185     * Saves the state of the {@code ConcurrentHashMapV8} instance to a
3186     * stream (i.e., serializes it).
3187     * @param s the stream
3188     * @serialData
3189     * the key (Object) and value (Object)
3190     * for each key-value mapping, followed by a null pair.
3191     * The key-value mappings are emitted in no particular order.
3192     */
3193     @SuppressWarnings("unchecked")
3194     private void writeObject(java.io.ObjectOutputStream s)
3195     throws java.io.IOException {
3196     if (segments == null) { // for serialization compatibility
3197     segments = (Segment<K,V>[])
3198     new Segment<?,?>[DEFAULT_CONCURRENCY_LEVEL];
3199     for (int i = 0; i < segments.length; ++i)
3200 dl 1.16 segments[i] = new Segment<K,V>(LOAD_FACTOR);
3201 dl 1.1 }
3202     s.defaultWriteObject();
3203 dl 1.41 InternalIterator<K,V> it = new InternalIterator<K,V>(this);
3204     Object v;
3205     while ((v = it.advance()) != null) {
3206 dl 1.14 s.writeObject(it.nextKey);
3207 dl 1.41 s.writeObject(v);
3208 dl 1.14 }
3209 dl 1.1 s.writeObject(null);
3210     s.writeObject(null);
3211     segments = null; // throw away
3212     }
3213    
3214     /**
3215 jsr166 1.9 * Reconstitutes the instance from a stream (that is, deserializes it).
3216 dl 1.1 * @param s the stream
3217     */
3218     @SuppressWarnings("unchecked")
3219     private void readObject(java.io.ObjectInputStream s)
3220     throws java.io.IOException, ClassNotFoundException {
3221     s.defaultReadObject();
3222     this.segments = null; // unneeded
3223 jsr166 1.21 // initialize transient final field
3224 dl 1.14 UNSAFE.putObjectVolatile(this, counterOffset, new LongAdder());
3225    
3226     // Create all nodes, then place in table once size is known
3227     long size = 0L;
3228     Node p = null;
3229 dl 1.1 for (;;) {
3230 dl 1.14 K k = (K) s.readObject();
3231     V v = (V) s.readObject();
3232     if (k != null && v != null) {
3233 dl 1.38 int h = spread(k.hashCode());
3234     p = new Node(h, k, v, p);
3235 dl 1.14 ++size;
3236     }
3237     else
3238 dl 1.1 break;
3239 dl 1.14 }
3240     if (p != null) {
3241     boolean init = false;
3242 dl 1.24 int n;
3243     if (size >= (long)(MAXIMUM_CAPACITY >>> 1))
3244     n = MAXIMUM_CAPACITY;
3245     else {
3246     int sz = (int)size;
3247     n = tableSizeFor(sz + (sz >>> 1) + 1);
3248     }
3249     int sc = sizeCtl;
3250 dl 1.38 boolean collide = false;
3251 dl 1.24 if (n > sc &&
3252     UNSAFE.compareAndSwapInt(this, sizeCtlOffset, sc, -1)) {
3253 dl 1.14 try {
3254     if (table == null) {
3255     init = true;
3256     Node[] tab = new Node[n];
3257     int mask = n - 1;
3258     while (p != null) {
3259     int j = p.hash & mask;
3260     Node next = p.next;
3261 dl 1.38 Node q = p.next = tabAt(tab, j);
3262 dl 1.14 setTabAt(tab, j, p);
3263 dl 1.38 if (!collide && q != null && q.hash == p.hash)
3264     collide = true;
3265 dl 1.14 p = next;
3266     }
3267     table = tab;
3268     counter.add(size);
3269 dl 1.29 sc = n - (n >>> 2);
3270 dl 1.14 }
3271     } finally {
3272 dl 1.24 sizeCtl = sc;
3273 dl 1.14 }
3274 dl 1.38 if (collide) { // rescan and convert to TreeBins
3275     Node[] tab = table;
3276     for (int i = 0; i < tab.length; ++i) {
3277     int c = 0;
3278     for (Node e = tabAt(tab, i); e != null; e = e.next) {
3279     if (++c > TREE_THRESHOLD &&
3280     (e.key instanceof Comparable)) {
3281     replaceWithTreeBin(tab, i, e.key);
3282     break;
3283     }
3284     }
3285     }
3286     }
3287 dl 1.14 }
3288     if (!init) { // Can only happen if unsafely published.
3289     while (p != null) {
3290 dl 1.27 internalPut(p.key, p.val);
3291 dl 1.14 p = p.next;
3292     }
3293     }
3294 dl 1.1 }
3295     }
3296    
3297     // Unsafe mechanics
3298     private static final sun.misc.Unsafe UNSAFE;
3299     private static final long counterOffset;
3300 dl 1.24 private static final long sizeCtlOffset;
3301 dl 1.1 private static final long ABASE;
3302     private static final int ASHIFT;
3303    
3304     static {
3305     int ss;
3306     try {
3307     UNSAFE = getUnsafe();
3308     Class<?> k = ConcurrentHashMapV8.class;
3309     counterOffset = UNSAFE.objectFieldOffset
3310     (k.getDeclaredField("counter"));
3311 dl 1.24 sizeCtlOffset = UNSAFE.objectFieldOffset
3312     (k.getDeclaredField("sizeCtl"));
3313 dl 1.1 Class<?> sc = Node[].class;
3314     ABASE = UNSAFE.arrayBaseOffset(sc);
3315     ss = UNSAFE.arrayIndexScale(sc);
3316     } catch (Exception e) {
3317     throw new Error(e);
3318     }
3319     if ((ss & (ss-1)) != 0)
3320     throw new Error("data type scale not a power of two");
3321     ASHIFT = 31 - Integer.numberOfLeadingZeros(ss);
3322     }
3323    
3324     /**
3325     * Returns a sun.misc.Unsafe. Suitable for use in a 3rd party package.
3326     * Replace with a simple call to Unsafe.getUnsafe when integrating
3327     * into a jdk.
3328     *
3329     * @return a sun.misc.Unsafe
3330     */
3331     private static sun.misc.Unsafe getUnsafe() {
3332     try {
3333     return sun.misc.Unsafe.getUnsafe();
3334     } catch (SecurityException se) {
3335     try {
3336     return java.security.AccessController.doPrivileged
3337     (new java.security
3338     .PrivilegedExceptionAction<sun.misc.Unsafe>() {
3339     public sun.misc.Unsafe run() throws Exception {
3340     java.lang.reflect.Field f = sun.misc
3341     .Unsafe.class.getDeclaredField("theUnsafe");
3342     f.setAccessible(true);
3343     return (sun.misc.Unsafe) f.get(null);
3344     }});
3345     } catch (java.security.PrivilegedActionException e) {
3346     throw new RuntimeException("Could not initialize intrinsics",
3347     e.getCause());
3348     }
3349     }
3350     }
3351    
3352     }