Return-Path: X-Original-To: apmail-ignite-commits-archive@minotaur.apache.org Delivered-To: apmail-ignite-commits-archive@minotaur.apache.org Received: from mail.apache.org (hermes.apache.org [140.211.11.3]) by minotaur.apache.org (Postfix) with SMTP id 6D23B179C8 for ; Wed, 10 Jun 2015 15:04:46 +0000 (UTC) Received: (qmail 60363 invoked by uid 500); 10 Jun 2015 15:04:40 -0000 Delivered-To: apmail-ignite-commits-archive@ignite.apache.org Received: (qmail 60327 invoked by uid 500); 10 Jun 2015 15:04:40 -0000 Mailing-List: contact commits-help@ignite.incubator.apache.org; run by ezmlm Precedence: bulk List-Help: List-Unsubscribe: List-Post: List-Id: Reply-To: dev@ignite.incubator.apache.org Delivered-To: mailing list commits@ignite.incubator.apache.org Received: (qmail 60284 invoked by uid 99); 10 Jun 2015 15:04:40 -0000 Received: from Unknown (HELO spamd3-us-west.apache.org) (209.188.14.142) by apache.org (qpsmtpd/0.29) with ESMTP; Wed, 10 Jun 2015 15:04:40 +0000 Received: from localhost (localhost [127.0.0.1]) by spamd3-us-west.apache.org (ASF Mail Server at spamd3-us-west.apache.org) with ESMTP id A3C241827EA for ; Wed, 10 Jun 2015 15:04:39 +0000 (UTC) X-Virus-Scanned: Debian amavisd-new at spamd3-us-west.apache.org X-Spam-Flag: NO X-Spam-Score: 0.971 X-Spam-Level: X-Spam-Status: No, score=0.971 tagged_above=-999 required=6.31 tests=[KAM_LAZY_DOMAIN_SECURITY=1, RCVD_IN_MSPIKE_H3=-0.01, RCVD_IN_MSPIKE_WL=-0.01, T_RP_MATCHES_RCVD=-0.01, URIBL_BLOCKED=0.001] autolearn=disabled Received: from mx1-us-west.apache.org ([10.40.0.8]) by localhost (spamd3-us-west.apache.org [10.40.0.10]) (amavisd-new, port 10024) with ESMTP id fdyI8ro9DadW for ; Wed, 10 Jun 2015 15:04:27 +0000 (UTC) Received: from mail.apache.org (hermes.apache.org [140.211.11.3]) by mx1-us-west.apache.org (ASF Mail Server at mx1-us-west.apache.org) with SMTP id 8FFAC27619 for ; Wed, 10 Jun 2015 15:04:27 +0000 (UTC) Received: (qmail 58936 invoked by uid 99); 10 Jun 2015 15:04:27 -0000 Received: from git1-us-west.apache.org (HELO git1-us-west.apache.org) (140.211.11.23) by apache.org (qpsmtpd/0.29) with ESMTP; Wed, 10 Jun 2015 15:04:27 +0000 Received: by git1-us-west.apache.org (ASF Mail Server at git1-us-west.apache.org, from userid 33) id E1548E0435; Wed, 10 Jun 2015 15:04:26 +0000 (UTC) Content-Type: text/plain; charset="us-ascii" MIME-Version: 1.0 Content-Transfer-Encoding: 7bit From: sboikov@apache.org To: commits@ignite.incubator.apache.org Date: Wed, 10 Jun 2015 15:04:27 -0000 Message-Id: <03d70693d63a4ef483a026331412598e@git.apache.org> In-Reply-To: <1ee8ad76585a4158972dd0fbad0715f8@git.apache.org> References: <1ee8ad76585a4158972dd0fbad0715f8@git.apache.org> X-Mailer: ASF-Git Admin Mailer Subject: [2/7] incubator-ignite git commit: ignite-sprint-6: merge from ignite-545 http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/455b96fc/modules/core/src/main/java/org/jsr166/ConcurrentHashMap8.java ---------------------------------------------------------------------- diff --git a/modules/core/src/main/java/org/jsr166/ConcurrentHashMap8.java b/modules/core/src/main/java/org/jsr166/ConcurrentHashMap8.java index 79b42b2..727db4c 100644 --- a/modules/core/src/main/java/org/jsr166/ConcurrentHashMap8.java +++ b/modules/core/src/main/java/org/jsr166/ConcurrentHashMap8.java @@ -5,16 +5,22 @@ */ /* - * The initial version of this file was copied from JSR-166: - * http://gee.cs.oswego.edu/dl/concurrency-interest/ + * The latest version of the file corresponds to the following CVS commit: + * http://gee.cs.oswego.edu/cgi-bin/viewcvs.cgi/jsr166/src/jdk7/java/util/concurrent/ConcurrentHashMap.java?pathrev=1.43 + * + * Note, that the repository above is JDK 7 based that is kept up-to-date too. + * The main repository (JDK 8 based) uses JDK 8 features significantly that unavailable in JDK 7. */ + package org.jsr166; import java.io.*; import java.util.*; import java.util.concurrent.*; +import java.util.concurrent.atomic.*; import java.util.concurrent.locks.*; +import java.lang.reflect.*; /** * A hash table supporting full concurrency of retrievals and @@ -68,21 +74,15 @@ import java.util.concurrent.locks.*; * expected {@code concurrencyLevel} as an additional hint for * internal sizing. Note that using many keys with exactly the same * {@code hashCode()} is a sure way to slow down performance of any - * hash table. + * hash table. To ameliorate impact, when keys are {@link Comparable}, + * this class may use comparison order among keys to help break ties. * - *

A {@link Set} projection of a ConcurrentHashMapV8 may be created + *

A {@link Set} projection of a ConcurrentHashMap may be created * (using {@link #newKeySet()} or {@link #newKeySet(int)}), or viewed * (using {@link #keySet(Object)} when only keys are of interest, and the * mapped values are (perhaps transiently) not used or all take the * same mapping value. * - *

A ConcurrentHashMapV8 can be used as scalable frequency map (a - * form of histogram or multiset) by using {@link LongAdder8} values - * and initializing via {@link #computeIfAbsent}. For example, to add - * a count to a {@code ConcurrentHashMapV8 freqs}, you - * can use {@code freqs.computeIfAbsent(k -> new - * LongAdder8()).increment();} - * *

This class and its views and iterators implement all of the * optional methods of the {@link Map} and {@link Iterator} * interfaces. @@ -90,90 +90,9 @@ import java.util.concurrent.locks.*; *

Like {@link Hashtable} but unlike {@link HashMap}, this class * does not allow {@code null} to be used as a key or value. * - *

    - *
  • forEach: Perform a given action on each element. - * A variant form applies a given transformation on each element - * before performing the action.
  • - * - *
  • search: Return the first available non-null result of - * applying a given function on each element; skipping further - * search when a result is found.
  • - * - *
  • reduce: Accumulate each element. The supplied reduction - * function cannot rely on ordering (more formally, it should be - * both associative and commutative). There are five variants: - * - *
      - * - *
    • Plain reductions. (There is not a form of this method for - * (key, value) function arguments since there is no corresponding - * return type.)
    • - * - *
    • Mapped reductions that accumulate the results of a given - * function applied to each element.
    • - * - *
    • Reductions to scalar doubles, longs, and ints, using a - * given basis value.
    • - * - * - *
    - *
- * - *

The concurrency properties of bulk operations follow - * from those of ConcurrentHashMapV8: Any non-null result returned - * from {@code get(key)} and related access methods bears a - * happens-before relation with the associated insertion or - * update. The result of any bulk operation reflects the - * composition of these per-element relations (but is not - * necessarily atomic with respect to the map as a whole unless it - * is somehow known to be quiescent). Conversely, because keys - * and values in the map are never null, null serves as a reliable - * atomic indicator of the current lack of any result. To - * maintain this property, null serves as an implicit basis for - * all non-scalar reduction operations. For the double, long, and - * int versions, the basis should be one that, when combined with - * any other value, returns that other value (more formally, it - * should be the identity element for the reduction). Most common - * reductions have these properties; for example, computing a sum - * with basis 0 or a minimum with basis MAX_VALUE. - * - *

Search and transformation functions provided as arguments - * should similarly return null to indicate the lack of any result - * (in which case it is not used). In the case of mapped - * reductions, this also enables transformations to serve as - * filters, returning null (or, in the case of primitive - * specializations, the identity basis) if the element should not - * be combined. You can create compound transformations and - * filterings by composing them yourself under this "null means - * there is nothing there now" rule before using them in search or - * reduce operations. - * - *

Methods accepting and/or returning Entry arguments maintain - * key-value associations. They may be useful for example when - * finding the key for the greatest value. Note that "plain" Entry - * arguments can be supplied using {@code new - * AbstractMap.SimpleEntry(k,v)}. - * - *

Bulk operations may complete abruptly, throwing an - * exception encountered in the application of a supplied - * function. Bear in mind when handling such exceptions that other - * concurrently executing functions could also have thrown - * exceptions, or would have done so if the first exception had - * not occurred. - * - *

Parallel speedups for bulk operations compared to sequential - * processing are common but not guaranteed. Operations involving - * brief functions on small maps may execute more slowly than - * sequential loops if the underlying work to parallelize the - * computation is more expensive than the computation itself. - * Similarly, parallelization may not lead to much actual parallelism - * if all processors are busy performing unrelated tasks. - * - *

All arguments to all task methods must be non-null. - * - *

jsr166e note: During transition, this class - * uses nested functional interfaces with different names but the - * same forms as those expected for JDK8. + *

This class is a member of the + * + * Java Collections Framework. * * @since 1.5 * @author Doug Lea @@ -181,80 +100,9 @@ import java.util.concurrent.locks.*; * @param the type of mapped values */ @SuppressWarnings("ALL") -public class ConcurrentHashMap8 - implements ConcurrentMap, Serializable { +public class ConcurrentHashMap8 implements ConcurrentMap, Serializable { private static final long serialVersionUID = 7249069246763182397L; - /** - * A partitionable iterator. A Spliterator can be traversed - * directly, but can also be partitioned (before traversal) by - * creating another Spliterator that covers a non-overlapping - * portion of the elements, and so may be amenable to parallel - * execution. - * - *

This interface exports a subset of expected JDK8 - * functionality. - * - *

Sample usage: Here is one (of the several) ways to compute - * the sum of the values held in a map using the ForkJoin - * framework. As illustrated here, Spliterators are well suited to - * designs in which a task repeatedly splits off half its work - * into forked subtasks until small enough to process directly, - * and then joins these subtasks. Variants of this style can also - * be used in completion-based designs. - * - *

-     * {@code ConcurrentHashMapV8 m = ...
-     * // split as if have 8 * parallelism, for load balance
-     * int n = m.size();
-     * int p = aForkJoinPool.getParallelism() * 8;
-     * int split = (n < p)? n : p;
-     * long sum = aForkJoinPool.invoke(new SumValues(m.valueSpliterator(), split, null));
-     * // ...
-     * static class SumValues extends RecursiveTask {
-     *   final Spliterator s;
-     *   final int split;             // split while > 1
-     *   final SumValues nextJoin;    // records forked subtasks to join
-     *   SumValues(Spliterator s, int depth, SumValues nextJoin) {
-     *     this.s = s; this.depth = depth; this.nextJoin = nextJoin;
-     *   }
-     *   public Long compute() {
-     *     long sum = 0;
-     *     SumValues subtasks = null; // fork subtasks
-     *     for (int s = split >>> 1; s > 0; s >>>= 1)
-     *       (subtasks = new SumValues(s.split(), s, subtasks)).fork();
-     *     while (s.hasNext())        // directly process remaining elements
-     *       sum += s.next();
-     *     for (SumValues t = subtasks; t != null; t = t.nextJoin)
-     *       sum += t.join();         // collect subtask results
-     *     return sum;
-     *   }
-     * }
-     * }
- */ - public static interface Spliterator extends Iterator { - /** - * Returns a Spliterator covering approximately half of the - * elements, guaranteed not to overlap with those subsequently - * returned by this Spliterator. After invoking this method, - * the current Spliterator will not produce any of - * the elements of the returned Spliterator, but the two - * Spliterators together will produce all of the elements that - * would have been produced by this Spliterator had this - * method not been called. The exact number of elements - * produced by the returned Spliterator is not guaranteed, and - * may be zero (i.e., with {@code hasNext()} reporting {@code - * false}) if this Spliterator cannot be further split. - * - * @return a Spliterator covering approximately half of the - * elements - * @throws IllegalStateException if this Spliterator has - * already commenced traversing elements - */ - Spliterator split(); - } - - /* * Overview: * @@ -265,18 +113,21 @@ public class ConcurrentHashMap8 * the same or better than java.util.HashMap, and to support high * initial insertion rates on an empty table by many threads. * - * Each key-value mapping is held in a Node. Because Node fields - * can contain special values, they are defined using plain Object - * types. Similarly in turn, all internal methods that use them - * work off Object types. And similarly, so do the internal - * methods of auxiliary iterator and view classes. All public - * generic typed methods relay in/out of these internal methods, - * supplying null-checks and casts as needed. This also allows - * many of the public methods to be factored into a smaller number - * of internal methods (although sadly not so for the five - * variants of put-related operations). The validation-based - * approach explained below leads to a lot of code sprawl because - * retry-control precludes factoring into smaller methods. + * This map usually acts as a binned (bucketed) hash table. Each + * key-value mapping is held in a Node. Most nodes are instances + * of the basic Node class with hash, key, value, and next + * fields. However, various subclasses exist: TreeNodes are + * arranged in balanced trees, not lists. TreeBins hold the roots + * of sets of TreeNodes. ForwardingNodes are placed at the heads + * of bins during resizing. ReservationNodes are used as + * placeholders while establishing values in computeIfAbsent and + * related methods. The types TreeBin, ForwardingNode, and + * ReservationNode do not hold normal user keys, values, or + * hashes, and are readily distinguishable during search etc + * because they have negative hash fields and null key and value + * fields. (These special nodes are either uncommon or transient, + * so the impact of carrying around some unused fields is + * insignificant.) * * The table is lazily initialized to a power-of-two size upon the * first insertion. Each bin in the table normally contains a @@ -284,24 +135,12 @@ public class ConcurrentHashMap8 * Table accesses require volatile/atomic reads, writes, and * CASes. Because there is no other way to arrange this without * adding further indirections, we use intrinsics - * (sun.misc.Unsafe) operations. The lists of nodes within bins - * are always accurately traversable under volatile reads, so long - * as lookups check hash code and non-nullness of value before - * checking key equality. - * - * We use the top two bits of Node hash fields for control - * purposes -- they are available anyway because of addressing - * constraints. As explained further below, these top bits are - * used as follows: - * 00 - Normal - * 01 - Locked - * 11 - Locked and may have a thread waiting for lock - * 10 - Node is a forwarding node + * (sun.misc.Unsafe) operations. * - * The lower 30 bits of each Node's hash field contain a - * transformation of the key's hash code, except for forwarding - * nodes, for which the lower bits are zero (and so always have - * hash field == MOVED). + * We use the top (sign) bit of Node hash fields for control + * purposes -- it is available anyway because of addressing + * constraints. Nodes with negative hash fields are specially + * handled or ignored in map methods. * * Insertion (via put or its variants) of the first node in an * empty bin is performed by just CASing it to the bin. This is @@ -310,22 +149,15 @@ public class ConcurrentHashMap8 * delete, and replace) require locks. We do not want to waste * the space required to associate a distinct lock object with * each bin, so instead use the first node of a bin list itself as - * a lock. Blocking support for these locks relies on the builtin - * "synchronized" monitors. However, we also need a tryLock - * construction, so we overlay these by using bits of the Node - * hash field for lock control (see above), and so normally use - * builtin monitors only for blocking and signalling using - * wait/notifyAll constructions. See Node.tryAwaitLock. + * a lock. Locking support for these locks relies on builtin + * "synchronized" monitors. * * Using the first node of a list as a lock does not by itself * suffice though: When a node is locked, any update must first * validate that it is still the first node after locking it, and * retry if not. Because new nodes are always appended to lists, * once a node is first in a bin, it remains first until deleted - * or the bin becomes invalidated (upon resizing). However, - * operations that only conditionally update may inspect nodes - * until the point of update. This is a converse of sorts to the - * lazy locking technique described by Herlihy & Shavit. + * or the bin becomes invalidated (upon resizing). * * The main disadvantage of per-bin locks is that other update * operations on other nodes in a bin list protected by the same @@ -358,15 +190,12 @@ public class ConcurrentHashMap8 * sometimes deviate significantly from uniform randomness. This * includes the case when N > (1<<30), so some keys MUST collide. * Similarly for dumb or hostile usages in which multiple keys are - * designed to have identical hash codes. Also, although we guard - * against the worst effects of this (see method spread), sets of - * hashes may differ only in bits that do not impact their bin - * index for a given power-of-two mask. So we use a secondary - * strategy that applies when the number of nodes in a bin exceeds - * a threshold, and at least one of the keys implements - * Comparable. These TreeBins use a balanced tree to hold nodes - * (a specialized form of red-black trees), bounding search time - * to O(log N). Each search step in a TreeBin is around twice as + * designed to have identical hash codes or ones that differs only + * in masked-out high bits. So we use a secondary strategy that + * applies when the number of nodes in a bin exceeds a + * threshold. These TreeBins use a balanced tree to hold nodes (a + * specialized form of red-black trees), bounding search time to + * O(log N). Each search step in a TreeBin is at least twice as * slow as in a regular list, but given that N cannot exceed * (1<<64) (before running out of addresses) this bounds search * steps, lock hold times, etc, to reasonable constants (roughly @@ -377,43 +206,50 @@ public class ConcurrentHashMap8 * iterators in the same way. * * The table is resized when occupancy exceeds a percentage - * threshold (nominally, 0.75, but see below). Only a single - * thread performs the resize (using field "sizeCtl", to arrange - * exclusion), but the table otherwise remains usable for reads - * and updates. Resizing proceeds by transferring bins, one by - * one, from the table to the next table. Because we are using - * power-of-two expansion, the elements from each bin must either - * stay at same index, or move with a power of two offset. We - * eliminate unnecessary node creation by catching cases where old - * nodes can be reused because their next fields won't change. On - * average, only about one-sixth of them need cloning when a table - * doubles. The nodes they replace will be garbage collectable as - * soon as they are no longer referenced by any reader thread that - * may be in the midst of concurrently traversing table. Upon - * transfer, the old table bin contains only a special forwarding - * node (with hash field "MOVED") that contains the next table as - * its key. On encountering a forwarding node, access and update - * operations restart, using the new table. - * - * Each bin transfer requires its bin lock. However, unlike other - * cases, a transfer can skip a bin if it fails to acquire its - * lock, and revisit it later (unless it is a TreeBin). Method - * rebuild maintains a buffer of TRANSFER_BUFFER_SIZE bins that - * have been skipped because of failure to acquire a lock, and - * blocks only if none are available (i.e., only very rarely). - * The transfer operation must also ensure that all accessible - * bins in both the old and new table are usable by any traversal. - * When there are no lock acquisition failures, this is arranged - * simply by proceeding from the last bin (table.length - 1) up - * towards the first. Upon seeing a forwarding node, traversals - * (see class Iter) arrange to move to the new table - * without revisiting nodes. However, when any node is skipped - * during a transfer, all earlier table bins may have become - * visible, so are initialized with a reverse-forwarding node back - * to the old table until the new ones are established. (This - * sometimes requires transiently locking a forwarding node, which - * is possible under the above encoding.) These more expensive - * mechanics trigger only when necessary. + * threshold (nominally, 0.75, but see below). Any thread + * noticing an overfull bin may assist in resizing after the + * initiating thread allocates and sets up the replacement array. + * However, rather than stalling, these other threads may proceed + * with insertions etc. The use of TreeBins shields us from the + * worst case effects of overfilling while resizes are in + * progress. Resizing proceeds by transferring bins, one by one, + * from the table to the next table. However, threads claim small + * blocks of indices to transfer (via field transferIndex) before + * doing so, reducing contention. A generation stamp in field + * sizeCtl ensures that resizings do not overlap. Because we are + * using power-of-two expansion, the elements from each bin must + * either stay at same index, or move with a power of two + * offset. We eliminate unnecessary node creation by catching + * cases where old nodes can be reused because their next fields + * won't change. On average, only about one-sixth of them need + * cloning when a table doubles. The nodes they replace will be + * garbage collectable as soon as they are no longer referenced by + * any reader thread that may be in the midst of concurrently + * traversing table. Upon transfer, the old table bin contains + * only a special forwarding node (with hash field "MOVED") that + * contains the next table as its key. On encountering a + * forwarding node, access and update operations restart, using + * the new table. + * + * Each bin transfer requires its bin lock, which can stall + * waiting for locks while resizing. However, because other + * threads can join in and help resize rather than contend for + * locks, average aggregate waits become shorter as resizing + * progresses. The transfer operation must also ensure that all + * accessible bins in both the old and new table are usable by any + * traversal. This is arranged in part by proceeding from the + * last bin (table.length - 1) up towards the first. Upon seeing + * a forwarding node, traversals (see class Traverser) arrange to + * move to the new table without revisiting nodes. To ensure that + * no intervening nodes are skipped even when moved out of order, + * a stack (see class TableStack) is created on first encounter of + * a forwarding node during a traversal, to maintain its place if + * later processing the current table. The need for these + * save/restore mechanics is relatively rare, but when one + * forwarding node is encountered, typically many more will be. + * So Traversers use a simple caching scheme to avoid creating so + * many new TableStack nodes. (Thanks to Peter Levart for + * suggesting use of a stack here.) * * The traversal scheme also applies to partial traversals of * ranges of bins (via an alternate Traverser constructor) @@ -428,20 +264,54 @@ public class ConcurrentHashMap8 * These cases attempt to override the initial capacity settings, * but harmlessly fail to take effect in cases of races. * - * The element count is maintained using a LongAdder8, which avoids - * contention on updates but can encounter cache thrashing if read - * too frequently during concurrent access. To avoid reading so - * often, resizing is attempted either when a bin lock is - * contended, or upon adding to a bin already holding two or more - * nodes (checked before adding in the xIfAbsent methods, after - * adding in others). Under uniform hash distributions, the - * probability of this occurring at threshold is around 13%, - * meaning that only about 1 in 8 puts check threshold (and after - * resizing, many fewer do so). But this approximation has high - * variance for small table sizes, so we check on any collision - * for sizes <= 64. The bulk putAll operation further reduces - * contention by only committing count updates upon these size - * checks. + * The element count is maintained using a specialization of + * LongAdder. We need to incorporate a specialization rather than + * just use a LongAdder in order to access implicit + * contention-sensing that leads to creation of multiple + * CounterCells. The counter mechanics avoid contention on + * updates but can encounter cache thrashing if read too + * frequently during concurrent access. To avoid reading so often, + * resizing under contention is attempted only upon adding to a + * bin already holding two or more nodes. Under uniform hash + * distributions, the probability of this occurring at threshold + * is around 13%, meaning that only about 1 in 8 puts check + * threshold (and after resizing, many fewer do so). + * + * TreeBins use a special form of comparison for search and + * related operations (which is the main reason we cannot use + * existing collections such as TreeMaps). TreeBins contain + * Comparable elements, but may contain others, as well as + * elements that are Comparable but not necessarily Comparable for + * the same T, so we cannot invoke compareTo among them. To handle + * this, the tree is ordered primarily by hash value, then by + * Comparable.compareTo order if applicable. On lookup at a node, + * if elements are not comparable or compare as 0 then both left + * and right children may need to be searched in the case of tied + * hash values. (This corresponds to the full list search that + * would be necessary if all elements were non-Comparable and had + * tied hashes.) On insertion, to keep a total ordering (or as + * close as is required here) across rebalancings, we compare + * classes and identityHashCodes as tie-breakers. The red-black + * balancing code is updated from pre-jdk-collections + * (http://gee.cs.oswego.edu/dl/classes/collections/RBCell.java) + * based in turn on Cormen, Leiserson, and Rivest "Introduction to + * Algorithms" (CLR). + * + * TreeBins also require an additional locking mechanism. While + * list traversal is always possible by readers even during + * updates, tree traversal is not, mainly because of tree-rotations + * that may change the root node and/or its linkages. TreeBins + * include a simple read-write lock mechanism parasitic on the + * main bin-synchronization strategy: Structural adjustments + * associated with an insertion or removal are already bin-locked + * (and so cannot conflict with other writers) but must wait for + * ongoing readers to finish. Since there can be only one such + * waiter, we use a simple scheme using a single "waiter" field to + * block writers. However, readers need never block. If the root + * lock is held, they proceed along the slow traversal path (via + * next-pointers) until the lock becomes available or the list is + * exhausted, whichever comes first. These cases are not fast, but + * maximize aggregate expected throughput. * * Maintaining API and serialization compatibility with previous * versions of this class introduces several oddities. Mainly: We @@ -451,8 +321,20 @@ public class ConcurrentHashMap8 * time that we can guarantee to honor it.) We also declare an * unused "Segment" class that is instantiated in minimal form * only when serializing. + * + * Also, solely for compatibility with previous versions of this + * class, it extends AbstractMap, even though all of its methods + * are overridden, so it is just useless baggage. + * + * This file is organized to make things a little easier to follow + * while reading than they might otherwise: First the main static + * declarations and utilities, then fields, then main public + * methods (with a few factorings of multiple public methods into + * internal ones), then sizing methods, trees, traversers, and + * bulk operations. */ + /* ---------------- Constants -------------- */ /** @@ -492,2737 +374,2362 @@ public class ConcurrentHashMap8 private static final float LOAD_FACTOR = 0.75f; /** - * The buffer size for skipped bins during transfers. The - * value is arbitrary but should be large enough to avoid - * most locking stalls during resizes. + * The bin count threshold for using a tree rather than list for a + * bin. Bins are converted to trees when adding an element to a + * bin with at least this many nodes. The value must be greater + * than 2, and should be at least 8 to mesh with assumptions in + * tree removal about conversion back to plain bins upon + * shrinkage. */ - private static final int TRANSFER_BUFFER_SIZE = 32; + static final int TREEIFY_THRESHOLD = 8; /** - * The bin count threshold for using a tree rather than list for a - * bin. The value reflects the approximate break-even point for - * using tree-based operations. + * The bin count threshold for untreeifying a (split) bin during a + * resize operation. Should be less than TREEIFY_THRESHOLD, and at + * most 6 to mesh with shrinkage detection under removal. */ - private static final int TREE_THRESHOLD = 8; + static final int UNTREEIFY_THRESHOLD = 6; - /* - * Encodings for special uses of Node hash fields. See above for - * explanation. + /** + * The smallest table capacity for which bins may be treeified. + * (Otherwise the table is resized if too many nodes in a bin.) + * The value should be at least 4 * TREEIFY_THRESHOLD to avoid + * conflicts between resizing and treeification thresholds. */ - static final int MOVED = 0x80000000; // hash field for forwarding nodes - static final int LOCKED = 0x40000000; // set/tested only as a bit - static final int WAITING = 0xc0000000; // both bits set/tested together - static final int HASH_BITS = 0x3fffffff; // usable bits of normal node hash - - /* ---------------- Fields -------------- */ + static final int MIN_TREEIFY_CAPACITY = 64; /** - * The array of bins. Lazily initialized upon first insertion. - * Size is always a power of two. Accessed directly by iterators. + * Minimum number of rebinnings per transfer step. Ranges are + * subdivided to allow multiple resizer threads. This value + * serves as a lower bound to avoid resizers encountering + * excessive memory contention. The value should be at least + * DEFAULT_CAPACITY. */ - transient volatile Node[] table; + private static final int MIN_TRANSFER_STRIDE = 16; /** - * The counter maintaining number of elements. + * The number of bits used for generation stamp in sizeCtl. + * Must be at least 6 for 32bit arrays. */ - private transient final LongAdder8 counter; + private static int RESIZE_STAMP_BITS = 16; /** - * Table initialization and resizing control. When negative, the - * table is being initialized or resized. Otherwise, when table is - * null, holds the initial table size to use upon creation, or 0 - * for default. After initialization, holds the next element count - * value upon which to resize the table. + * The maximum number of threads that can help resize. + * Must fit in 32 - RESIZE_STAMP_BITS bits. */ - private transient volatile int sizeCtl; - - // views - private transient KeySetView keySet; - private transient ValuesView values; - private transient EntrySetView entrySet; - - /** For serialization compatibility. Null unless serialized; see below */ - private Segment[] segments; + private static final int MAX_RESIZERS = (1 << (32 - RESIZE_STAMP_BITS)) - 1; - /* ---------------- Table element access -------------- */ + /** + * The bit shift for recording size stamp in sizeCtl. + */ + private static final int RESIZE_STAMP_SHIFT = 32 - RESIZE_STAMP_BITS; /* - * Volatile access methods are used for table elements as well as - * elements of in-progress next table while resizing. Uses are - * null checked by callers, and implicitly bounds-checked, relying - * on the invariants that tab arrays have non-zero size, and all - * indices are masked with (tab.length - 1) which is never - * negative and always less than length. Note that, to be correct - * wrt arbitrary concurrency errors by users, bounds checks must - * operate on local variables, which accounts for some odd-looking - * inline assignments below. + * Encodings for Node hash fields. See above for explanation. */ + static final int MOVED = 0x8fffffff; // (-1) hash for forwarding nodes + static final int TREEBIN = 0x80000000; // hash for roots of trees + static final int RESERVED = 0x80000001; // hash for transient reservations + static final int HASH_BITS = 0x7fffffff; // usable bits of normal node hash - static final Node tabAt(Node[] tab, int i) { // used by Iter - return (Node)UNSAFE.getObjectVolatile(tab, ((long)i< implements Map.Entry { + final int hash; + final K key; + volatile V val; + Node next; - Node(int hash, Object key, Object val, Node next) { + Node(int hash, K key, V val, Node next) { this.hash = hash; this.key = key; this.val = val; this.next = next; } - /** CompareAndSet the hash field */ - final boolean casHash(int cmp, int val) { - return UNSAFE.compareAndSwapInt(this, hashOffset, cmp, val); + public final K getKey() { return key; } + public final V getValue() { return val; } + public final int hashCode() { return key.hashCode() ^ val.hashCode(); } + public final String toString(){ return key + "=" + val; } + public final V setValue(V value) { + throw new UnsupportedOperationException(); } - /** The number of spins before blocking for a lock */ - static final int MAX_SPINS = - Runtime.getRuntime().availableProcessors() > 1 ? 64 : 1; + public final boolean equals(Object o) { + Object k, v, u; Map.Entry e; + return ((o instanceof Map.Entry) && + (k = (e = (Map.Entry)o).getKey()) != null && + (v = e.getValue()) != null && + (k == key || k.equals(key)) && + (v == (u = val) || v.equals(u))); + } /** - * Spins a while if LOCKED bit set and this node is the first - * of its bin, and then sets WAITING bits on hash field and - * blocks (once) if they are still set. It is OK for this - * method to return even if lock is not available upon exit, - * which enables these simple single-wait mechanics. - * - * The corresponding signalling operation is performed within - * callers: Upon detecting that WAITING has been set when - * unlocking lock (via a failed CAS from non-waiting LOCKED - * state), unlockers acquire the sync lock and perform a - * notifyAll. - * - * The initial sanity check on tab and bounds is not currently - * necessary in the only usages of this method, but enables - * use in other future contexts. + * Virtualized support for map.get(); overridden in subclasses. */ - final void tryAwaitLock(Node[] tab, int i) { - if (tab != null && i >= 0 && i < tab.length) { // sanity check - int r = ThreadLocalRandom8.current().nextInt(); // randomize spins - int spins = MAX_SPINS, h; - while (tabAt(tab, i) == this && ((h = hash) & LOCKED) != 0) { - if (spins >= 0) { - r ^= r << 1; r ^= r >>> 3; r ^= r << 10; // xorshift - if (r >= 0 && --spins == 0) - Thread.yield(); // yield before block - } - else if (casHash(h, h | WAITING)) { - synchronized (this) { - if (tabAt(tab, i) == this && - (hash & WAITING) == WAITING) { - try { - wait(); - } catch (InterruptedException ie) { - try { - Thread.currentThread().interrupt(); - } catch (SecurityException ignore) { - } - } - } - else - notifyAll(); // possibly won race vs signaller - } - break; - } - } + Node find(int h, Object k) { + Node e = this; + if (k != null) { + do { + K ek; + if (e.hash == h && + ((ek = e.key) == k || (ek != null && k.equals(ek)))) + return e; + } while ((e = e.next) != null); } + return null; } + } - // Unsafe mechanics for casHash - private static final sun.misc.Unsafe UNSAFE; - private static final long hashOffset; + /* ---------------- Static utilities -------------- */ - static { - try { - UNSAFE = getUnsafe(); - Class k = Node.class; - hashOffset = UNSAFE.objectFieldOffset - (k.getDeclaredField("hash")); - } catch (Exception e) { - throw new Error(e); + /** + * Spreads (XORs) higher bits of hash to lower and also forces top + * bit to 0. Because the table uses power-of-two masking, sets of + * hashes that vary only in bits above the current mask will + * always collide. (Among known examples are sets of Float keys + * holding consecutive whole numbers in small tables.) So we + * apply a transform that spreads the impact of higher bits + * downward. There is a tradeoff between speed, utility, and + * quality of bit-spreading. Because many common sets of hashes + * are already reasonably distributed (so don't benefit from + * spreading), and because we use trees to handle large sets of + * collisions in bins, we just XOR some shifted bits in the + * cheapest possible way to reduce systematic lossage, as well as + * to incorporate impact of the highest bits that would otherwise + * never be used in index calculations because of table bounds. + */ + static final int spread(int h) { + return (h ^ (h >>> 16)) & HASH_BITS; + } + + /** + * Returns a power of two table size for the given desired capacity. + * See Hackers Delight, sec 3.2 + */ + private static final int tableSizeFor(int c) { + int n = c - 1; + n |= n >>> 1; + n |= n >>> 2; + n |= n >>> 4; + n |= n >>> 8; + n |= n >>> 16; + return (n < 0) ? 1 : (n >= MAXIMUM_CAPACITY) ? MAXIMUM_CAPACITY : n + 1; + } + + /** + * Returns x's Class if it is of the form "class C implements + * Comparable", else null. + */ + static Class comparableClassFor(Object x) { + if (x instanceof Comparable) { + Class c; Type[] ts, as; Type t; ParameterizedType p; + if ((c = x.getClass()) == String.class) // bypass checks + return c; + if ((ts = c.getGenericInterfaces()) != null) { + for (int i = 0; i < ts.length; ++i) { + if (((t = ts[i]) instanceof ParameterizedType) && + ((p = (ParameterizedType)t).getRawType() == + Comparable.class) && + (as = p.getActualTypeArguments()) != null && + as.length == 1 && as[0] == c) // type arg is c + return c; + } } } + return null; } - /* ---------------- TreeBins -------------- */ - /** - * Nodes for use in TreeBins + * Returns k.compareTo(x) if x matches kc (k's screened comparable + * class), else 0. */ - static final class TreeNode extends Node { - TreeNode parent; // red-black tree links - TreeNode left; - TreeNode right; - TreeNode prev; // needed to unlink next upon deletion - boolean red; + @SuppressWarnings({"rawtypes","unchecked"}) // for cast to Comparable + static int compareComparables(Class kc, Object k, Object x) { + return (x == null || x.getClass() != kc ? 0 : + ((Comparable)k).compareTo(x)); + } - TreeNode(int hash, Object key, Object val, Node next, TreeNode parent) { - super(hash, key, val, next); - this.parent = parent; - } + /* ---------------- Table element access -------------- */ + + /* + * Volatile access methods are used for table elements as well as + * elements of in-progress next table while resizing. All uses of + * the tab arguments must be null checked by callers. All callers + * also paranoically precheck that tab's length is not zero (or an + * equivalent check), thus ensuring that any index argument taking + * the form of a hash value anded with (length - 1) is a valid + * index. Note that, to be correct wrt arbitrary concurrency + * errors by users, these checks must operate on local variables, + * which accounts for some odd-looking inline assignments below. + * Note that calls to setTabAt always occur within locked regions, + * and so do not need full volatile semantics, but still require + * ordering to maintain concurrent readability. + */ + + @SuppressWarnings("unchecked") + static final Node tabAt(Node[] tab, int i) { + return (Node)U.getObjectVolatile(tab, ((long)i << ASHIFT) + ABASE); + } + + static final boolean casTabAt(Node[] tab, int i, + Node c, Node v) { + return U.compareAndSwapObject(tab, ((long)i << ASHIFT) + ABASE, c, v); + } + + static final void setTabAt(Node[] tab, int i, Node v) { + U.putOrderedObject(tab, ((long)i << ASHIFT) + ABASE, v); } + /* ---------------- Fields -------------- */ + /** - * A specialized form of red-black tree for use in bins - * whose size exceeds a threshold. - * - * TreeBins use a special form of comparison for search and - * related operations (which is the main reason we cannot use - * existing collections such as TreeMaps). TreeBins contain - * Comparable elements, but may contain others, as well as - * elements that are Comparable but not necessarily Comparable - * for the same T, so we cannot invoke compareTo among them. To - * handle this, the tree is ordered primarily by hash value, then - * by getClass().getName() order, and then by Comparator order - * among elements of the same class. On lookup at a node, if - * elements are not comparable or compare as 0, both left and - * right children may need to be searched in the case of tied hash - * values. (This corresponds to the full list search that would be - * necessary if all elements were non-Comparable and had tied - * hashes.) The red-black balancing code is updated from - * pre-jdk-collections - * (http://gee.cs.oswego.edu/dl/classes/collections/RBCell.java) - * based in turn on Cormen, Leiserson, and Rivest "Introduction to - * Algorithms" (CLR). - * - * TreeBins also maintain a separate locking discipline than - * regular bins. Because they are forwarded via special MOVED - * nodes at bin heads (which can never change once established), - * we cannot use those nodes as locks. Instead, TreeBin - * extends AbstractQueuedSynchronizer to support a simple form of - * read-write lock. For update operations and table validation, - * the exclusive form of lock behaves in the same way as bin-head - * locks. However, lookups use shared read-lock mechanics to allow - * multiple readers in the absence of writers. Additionally, - * these lookups do not ever block: While the lock is not - * available, they proceed along the slow traversal path (via - * next-pointers) until the lock becomes available or the list is - * exhausted, whichever comes first. (These cases are not fast, - * but maximize aggregate expected throughput.) The AQS mechanics - * for doing this are straightforward. The lock state is held as - * AQS getState(). Read counts are negative; the write count (1) - * is positive. There are no signalling preferences among readers - * and writers. Since we don't need to export full Lock API, we - * just override the minimal AQS methods and use them directly. - */ - static final class TreeBin extends AbstractQueuedSynchronizer { - private static final long serialVersionUID = 2249069246763182397L; - transient TreeNode root; // root of tree - transient TreeNode first; // head of next-pointer list - - /* AQS overrides */ - public final boolean isHeldExclusively() { return getState() > 0; } - public final boolean tryAcquire(int ignore) { - if (compareAndSetState(0, 1)) { - setExclusiveOwnerThread(Thread.currentThread()); - return true; - } - return false; - } - public final boolean tryRelease(int ignore) { - setExclusiveOwnerThread(null); - setState(0); - return true; - } - public final int tryAcquireShared(int ignore) { - for (int c;;) { - if ((c = getState()) > 0) - return -1; - if (compareAndSetState(c, c -1)) - return 1; - } - } - public final boolean tryReleaseShared(int ignore) { - int c; - do {} while (!compareAndSetState(c = getState(), c + 1)); - return c == -1; - } + * The array of bins. Lazily initialized upon first insertion. + * Size is always a power of two. Accessed directly by iterators. + */ + transient volatile Node[] table; - /** From CLR */ - private void rotateLeft(TreeNode p) { - if (p != null) { - TreeNode r = p.right, pp, rl; - if ((rl = p.right = r.left) != null) - rl.parent = p; - if ((pp = r.parent = p.parent) == null) - root = r; - else if (pp.left == p) - pp.left = r; - else - pp.right = r; - r.left = p; - p.parent = r; - } - } + /** + * The next table to use; non-null only while resizing. + */ + private transient volatile Node[] nextTable; - /** From CLR */ - private void rotateRight(TreeNode p) { - if (p != null) { - TreeNode l = p.left, pp, lr; - if ((lr = p.left = l.right) != null) - lr.parent = p; - if ((pp = l.parent = p.parent) == null) - root = l; - else if (pp.right == p) - pp.right = l; - else - pp.left = l; - l.right = p; - p.parent = l; - } - } + /** + * Base counter value, used mainly when there is no contention, + * but also as a fallback during table initialization + * races. Updated via CAS. + */ + private transient volatile long baseCount; - /** - * Returns the TreeNode (or null if not found) for the given key - * starting at given root. - */ - @SuppressWarnings("unchecked") final TreeNode getTreeNode - (int h, Object k, TreeNode p) { - Class c = k.getClass(); - while (p != null) { - int dir, ph; Object pk; Class pc; - if ((ph = p.hash) == h) { - if ((pk = p.key) == k || k.equals(pk)) - return p; - if (c != (pc = pk.getClass()) || - !(k instanceof Comparable) || - (dir = ((Comparable)k).compareTo((Comparable)pk)) == 0) { - dir = (c == pc) ? 0 : c.getName().compareTo(pc.getName()); - TreeNode r = null, s = null, pl, pr; - if (dir >= 0) { - if ((pl = p.left) != null && h <= pl.hash) - s = pl; - } - else if ((pr = p.right) != null && h >= pr.hash) - s = pr; - if (s != null && (r = getTreeNode(h, k, s)) != null) - return r; - } - } - else - dir = (h < ph) ? -1 : 1; - p = (dir > 0) ? p.right : p.left; - } - return null; - } + /** + * Table initialization and resizing control. When negative, the + * table is being initialized or resized: -1 for initialization, + * else -(1 + the number of active resizing threads). Otherwise, + * when table is null, holds the initial table size to use upon + * creation, or 0 for default. After initialization, holds the + * next element count value upon which to resize the table. + */ + private transient volatile int sizeCtl; - /** - * Wrapper for getTreeNode used by CHM.get. Tries to obtain - * read-lock to call getTreeNode, but during failure to get - * lock, searches along next links. - */ - final Object getValue(int h, Object k) { - Node r = null; - int c = getState(); // Must read lock state first - for (Node e = first; e != null; e = e.next) { - if (c <= 0 && compareAndSetState(c, c - 1)) { - try { - r = getTreeNode(h, k, root); - } finally { - releaseShared(0); - } - break; - } - else if ((e.hash & HASH_BITS) == h && k.equals(e.key)) { - r = e; - break; - } - else - c = getState(); - } - return r == null ? null : r.val; - } + /** + * The next table index (plus one) to split while resizing. + */ + private transient volatile int transferIndex; - /** - * Finds or adds a node. - * @return null if added - */ - @SuppressWarnings("unchecked") final TreeNode putTreeNode - (int h, Object k, Object v) { - Class c = k.getClass(); - TreeNode pp = root, p = null; - int dir = 0; - while (pp != null) { // find existing node or leaf to insert at - int ph; Object pk; Class pc; - p = pp; - if ((ph = p.hash) == h) { - if ((pk = p.key) == k || k.equals(pk)) - return p; - if (c != (pc = pk.getClass()) || - !(k instanceof Comparable) || - (dir = ((Comparable)k).compareTo((Comparable)pk)) == 0) { - dir = (c == pc) ? 0 : c.getName().compareTo(pc.getName()); - TreeNode r = null, s = null, pl, pr; - if (dir >= 0) { - if ((pl = p.left) != null && h <= pl.hash) - s = pl; - } - else if ((pr = p.right) != null && h >= pr.hash) - s = pr; - if (s != null && (r = getTreeNode(h, k, s)) != null) - return r; - } - } - else - dir = (h < ph) ? -1 : 1; - pp = (dir > 0) ? p.right : p.left; - } + /** + * Spinlock (locked via CAS) used when resizing and/or creating CounterCells. + */ + private transient volatile int cellsBusy; - TreeNode f = first; - TreeNode x = first = new TreeNode(h, k, v, f, p); - if (p == null) - root = x; - else { // attach and rebalance; adapted from CLR - TreeNode xp, xpp; - if (f != null) - f.prev = x; - if (dir <= 0) - p.left = x; - else - p.right = x; - x.red = true; - while (x != null && (xp = x.parent) != null && xp.red && - (xpp = xp.parent) != null) { - TreeNode xppl = xpp.left; - if (xp == xppl) { - TreeNode y = xpp.right; - if (y != null && y.red) { - y.red = false; - xp.red = false; - xpp.red = true; - x = xpp; - } - else { - if (x == xp.right) { - rotateLeft(x = xp); - xpp = (xp = x.parent) == null ? null : xp.parent; - } - if (xp != null) { - xp.red = false; - if (xpp != null) { - xpp.red = true; - rotateRight(xpp); - } - } - } - } - else { - TreeNode y = xppl; - if (y != null && y.red) { - y.red = false; - xp.red = false; - xpp.red = true; - x = xpp; - } - else { - if (x == xp.left) { - rotateRight(x = xp); - xpp = (xp = x.parent) == null ? null : xp.parent; - } - if (xp != null) { - xp.red = false; - if (xpp != null) { - xpp.red = true; - rotateLeft(xpp); - } - } - } - } - } - TreeNode r = root; - if (r != null && r.red) - r.red = false; - } - return null; - } + /** + * Table of counter cells. When non-null, size is a power of 2. + */ + private transient volatile CounterCell[] counterCells; - /** - * Removes the given node, that must be present before this - * call. This is messier than typical red-black deletion code - * because we cannot swap the contents of an interior node - * with a leaf successor that is pinned by "next" pointers - * that are accessible independently of lock. So instead we - * swap the tree linkages. - */ - final void deleteTreeNode(TreeNode p) { - TreeNode next = (TreeNode)p.next; // unlink traversal pointers - TreeNode pred = p.prev; - if (pred == null) - first = next; - else - pred.next = next; - if (next != null) - next.prev = pred; - TreeNode replacement; - TreeNode pl = p.left; - TreeNode pr = p.right; - if (pl != null && pr != null) { - TreeNode s = pr, sl; - while ((sl = s.left) != null) // find successor - s = sl; - boolean c = s.red; s.red = p.red; p.red = c; // swap colors - TreeNode sr = s.right; - TreeNode pp = p.parent; - if (s == pr) { // p was s's direct parent - p.parent = s; - s.right = p; - } - else { - TreeNode sp = s.parent; - if ((p.parent = sp) != null) { - if (s == sp.left) - sp.left = p; - else - sp.right = p; - } - if ((s.right = pr) != null) - pr.parent = s; - } - p.left = null; - if ((p.right = sr) != null) - sr.parent = p; - if ((s.left = pl) != null) - pl.parent = s; - if ((s.parent = pp) == null) - root = s; - else if (p == pp.left) - pp.left = s; - else - pp.right = s; - replacement = sr; - } - else - replacement = (pl != null) ? pl : pr; - TreeNode pp = p.parent; - if (replacement == null) { - if (pp == null) { - root = null; - return; - } - replacement = p; - } - else { - replacement.parent = pp; - if (pp == null) - root = replacement; - else if (p == pp.left) - pp.left = replacement; - else - pp.right = replacement; - p.left = p.right = p.parent = null; - } - if (!p.red) { // rebalance, from CLR - TreeNode x = replacement; - while (x != null) { - TreeNode xp, xpl; - if (x.red || (xp = x.parent) == null) { - x.red = false; - break; - } - if (x == (xpl = xp.left)) { - TreeNode sib = xp.right; - if (sib != null && sib.red) { - sib.red = false; - xp.red = true; - rotateLeft(xp); - sib = (xp = x.parent) == null ? null : xp.right; - } - if (sib == null) - x = xp; - else { - TreeNode sl = sib.left, sr = sib.right; - if ((sr == null || !sr.red) && - (sl == null || !sl.red)) { - sib.red = true; - x = xp; - } - else { - if (sr == null || !sr.red) { - if (sl != null) - sl.red = false; - sib.red = true; - rotateRight(sib); - sib = (xp = x.parent) == null ? null : xp.right; - } - if (sib != null) { - sib.red = (xp == null) ? false : xp.red; - if ((sr = sib.right) != null) - sr.red = false; - } - if (xp != null) { - xp.red = false; - rotateLeft(xp); - } - x = root; - } - } - } - else { // symmetric - TreeNode sib = xpl; - if (sib != null && sib.red) { - sib.red = false; - xp.red = true; - rotateRight(xp); - sib = (xp = x.parent) == null ? null : xp.left; - } - if (sib == null) - x = xp; - else { - TreeNode sl = sib.left, sr = sib.right; - if ((sl == null || !sl.red) && - (sr == null || !sr.red)) { - sib.red = true; - x = xp; - } - else { - if (sl == null || !sl.red) { - if (sr != null) - sr.red = false; - sib.red = true; - rotateLeft(sib); - sib = (xp = x.parent) == null ? null : xp.left; - } - if (sib != null) { - sib.red = (xp == null) ? false : xp.red; - if ((sl = sib.left) != null) - sl.red = false; - } - if (xp != null) { - xp.red = false; - rotateRight(xp); - } - x = root; - } - } - } - } - } - if (p == replacement && (pp = p.parent) != null) { - if (p == pp.left) // detach pointers - pp.left = null; - else if (p == pp.right) - pp.right = null; - p.parent = null; - } - } + // views + private transient KeySetView keySet; + private transient ValuesView values; + private transient EntrySetView entrySet; + + + /* ---------------- Public operations -------------- */ + + /** + * Creates a new, empty map with the default initial table size (16). + */ + public ConcurrentHashMap8() { } - /* ---------------- Collision reduction methods -------------- */ + /** + * Creates a new, empty map with an initial table size + * accommodating the specified number of elements without the need + * to dynamically resize. + * + * @param initialCapacity The implementation performs internal + * sizing to accommodate this many elements. + * @throws IllegalArgumentException if the initial capacity of + * elements is negative + */ + public ConcurrentHashMap8(int initialCapacity) { + if (initialCapacity < 0) + throw new IllegalArgumentException(); + int cap = ((initialCapacity >= (MAXIMUM_CAPACITY >>> 1)) ? + MAXIMUM_CAPACITY : + tableSizeFor(initialCapacity + (initialCapacity >>> 1) + 1)); + this.sizeCtl = cap; + } /** - * Spreads higher bits to lower, and also forces top 2 bits to 0. - * Because the table uses power-of-two masking, sets of hashes - * that vary only in bits above the current mask will always - * collide. (Among known examples are sets of Float keys holding - * consecutive whole numbers in small tables.) To counter this, - * we apply a transform that spreads the impact of higher bits - * downward. There is a tradeoff between speed, utility, and - * quality of bit-spreading. Because many common sets of hashes - * are already reasonably distributed across bits (so don't benefit - * from spreading), and because we use trees to handle large sets - * of collisions in bins, we don't need excessively high quality. + * Creates a new map with the same mappings as the given map. + * + * @param m the map */ - private static final int spread(int h) { - h ^= (h >>> 18) ^ (h >>> 12); - return (h ^ (h >>> 10)) & HASH_BITS; + public ConcurrentHashMap8(Map m) { + this.sizeCtl = DEFAULT_CAPACITY; + putAll(m); } /** - * Replaces a list bin with a tree bin. Call only when locked. - * Fails to replace if the given key is non-comparable or table - * is, or needs, resizing. + * Creates a new, empty map with an initial table size based on + * the given number of elements ({@code initialCapacity}) and + * initial table density ({@code loadFactor}). + * + * @param initialCapacity the initial capacity. The implementation + * performs internal sizing to accommodate this many elements, + * given the specified load factor. + * @param loadFactor the load factor (table density) for + * establishing the initial table size + * @throws IllegalArgumentException if the initial capacity of + * elements is negative or the load factor is nonpositive + * + * @since 1.6 */ - private final void replaceWithTreeBin(Node[] tab, int index, Object key) { - if ((key instanceof Comparable) && - (tab.length >= MAXIMUM_CAPACITY || counter.sum() < (long)sizeCtl)) { - TreeBin t = new TreeBin(); - for (Node e = tabAt(tab, index); e != null; e = e.next) - t.putTreeNode(e.hash & HASH_BITS, e.key, e.val); - setTabAt(tab, index, new Node(MOVED, t, null, null)); - } + public ConcurrentHashMap8(int initialCapacity, float loadFactor) { + this(initialCapacity, loadFactor, 1); } - /* ---------------- Internal access and update methods -------------- */ + /** + * Creates a new, empty map with an initial table size based on + * the given number of elements ({@code initialCapacity}), table + * density ({@code loadFactor}), and number of concurrently + * updating threads ({@code concurrencyLevel}). + * + * @param initialCapacity the initial capacity. The implementation + * performs internal sizing to accommodate this many elements, + * given the specified load factor. + * @param loadFactor the load factor (table density) for + * establishing the initial table size + * @param concurrencyLevel the estimated number of concurrently + * updating threads. The implementation may use this value as + * a sizing hint. + * @throws IllegalArgumentException if the initial capacity is + * negative or the load factor or concurrencyLevel are + * nonpositive + */ + public ConcurrentHashMap8(int initialCapacity, + float loadFactor, int concurrencyLevel) { + if (!(loadFactor > 0.0f) || initialCapacity < 0 || concurrencyLevel <= 0) + throw new IllegalArgumentException(); + if (initialCapacity < concurrencyLevel) // Use at least as many bins + initialCapacity = concurrencyLevel; // as estimated threads + long size = (long)(1.0 + (long)initialCapacity / loadFactor); + int cap = (size >= (long)MAXIMUM_CAPACITY) ? + MAXIMUM_CAPACITY : tableSizeFor((int)size); + this.sizeCtl = cap; + } - /** Implementation for get and containsKey */ - private final Object internalGet(Object k) { - int h = spread(k.hashCode()); - retry: for (Node[] tab = table; tab != null;) { - Node e, p; Object ek, ev; int eh; // locals to read fields once - for (e = tabAt(tab, (tab.length - 1) & h); e != null; e = e.next) { - if ((eh = e.hash) == MOVED) { - if ((ek = e.key) instanceof TreeBin) // search TreeBin - return ((TreeBin)ek).getValue(h, k); - else { // restart with new table - tab = (Node[])ek; - continue retry; - } - } - else if ((eh & HASH_BITS) == h && (ev = e.val) != null && - ((ek = e.key) == k || k.equals(ek))) - return ev; + // Original (since JDK1.2) Map methods + + /** + * {@inheritDoc} + */ + public int size() { + long n = sumCount(); + return ((n < 0L) ? 0 : + (n > (long)Integer.MAX_VALUE) ? Integer.MAX_VALUE : + (int)n); + } + + /** + * {@inheritDoc} + */ + public boolean isEmpty() { + return sumCount() <= 0L; // ignore transient negative values + } + + /** + * Returns the value to which the specified key is mapped, + * or {@code null} if this map contains no mapping for the key. + * + *

More formally, if this map contains a mapping from a key + * {@code k} to a value {@code v} such that {@code key.equals(k)}, + * then this method returns {@code v}; otherwise it returns + * {@code null}. (There can be at most one such mapping.) + * + * @throws NullPointerException if the specified key is null + */ + public V get(Object key) { + Node[] tab; Node e, p; int n, eh; K ek; + int h = spread(key.hashCode()); + if ((tab = table) != null && (n = tab.length) > 0 && + (e = tabAt(tab, (n - 1) & h)) != null) { + if ((eh = e.hash) == h) { + if ((ek = e.key) == key || (ek != null && key.equals(ek))) + return e.val; + } + else if (eh < 0) + return (p = e.find(h, key)) != null ? p.val : null; + while ((e = e.next) != null) { + if (e.hash == h && + ((ek = e.key) == key || (ek != null && key.equals(ek)))) + return e.val; } - break; } return null; } /** - * Implementation for the four public remove/replace methods: - * Replaces node value with v, conditional upon match of cv if - * non-null. If resulting value is null, delete. + * Tests if the specified object is a key in this table. + * + * @param key possible key + * @return {@code true} if and only if the specified object + * is a key in this table, as determined by the + * {@code equals} method; {@code false} otherwise + * @throws NullPointerException if the specified key is null */ - private final Object internalReplace(Object k, Object v, Object cv) { - int h = spread(k.hashCode()); - Object oldVal = null; - for (Node[] tab = table;;) { - Node f; int i, fh; Object fk; - if (tab == null || - (f = tabAt(tab, i = (tab.length - 1) & h)) == null) - break; - else if ((fh = f.hash) == MOVED) { - if ((fk = f.key) instanceof TreeBin) { - TreeBin t = (TreeBin)fk; - boolean validated = false; - boolean deleted = false; - t.acquire(0); - try { - if (tabAt(tab, i) == f) { - validated = true; - TreeNode p = t.getTreeNode(h, k, t.root); - if (p != null) { - Object pv = p.val; - if (cv == null || cv == pv || cv.equals(pv)) { - oldVal = pv; - if ((p.val = v) == null) { - deleted = true; - t.deleteTreeNode(p); - } - } - } - } - } finally { - t.release(0); - } - if (validated) { - if (deleted) - counter.add(-1L); - break; - } - } - else - tab = (Node[])fk; - } - else if ((fh & HASH_BITS) != h && f.next == null) // precheck - break; // rules out possible existence - else if ((fh & LOCKED) != 0) { - checkForResize(); // try resizing if can't get lock - f.tryAwaitLock(tab, i); - } - else if (f.casHash(fh, fh | LOCKED)) { - boolean validated = false; - boolean deleted = false; - try { - if (tabAt(tab, i) == f) { - validated = true; - for (Node e = f, pred = null;;) { - Object ek, ev; - if ((e.hash & HASH_BITS) == h && - ((ev = e.val) != null) && - ((ek = e.key) == k || k.equals(ek))) { - if (cv == null || cv == ev || cv.equals(ev)) { - oldVal = ev; - if ((e.val = v) == null) { - deleted = true; - Node en = e.next; - if (pred != null) - pred.next = en; - else - setTabAt(tab, i, en); - } - } - break; - } - pred = e; - if ((e = e.next) == null) - break; - } - } - } finally { - if (!f.casHash(fh | LOCKED, fh)) { - f.hash = fh; - synchronized (f) { f.notifyAll(); }; - } - } - if (validated) { - if (deleted) - counter.add(-1L); - break; - } + public boolean containsKey(Object key) { + return get(key) != null; + } + + /** + * Returns {@code true} if this map maps one or more keys to the + * specified value. Note: This method may require a full traversal + * of the map, and is much slower than method {@code containsKey}. + * + * @param value value whose presence in this map is to be tested + * @return {@code true} if this map maps one or more keys to the + * specified value + * @throws NullPointerException if the specified value is null + */ + public boolean containsValue(Object value) { + if (value == null) + throw new NullPointerException(); + Node[] t; + if ((t = table) != null) { + Traverser it = new Traverser(t, t.length, 0, t.length); + for (Node p; (p = it.advance()) != null; ) { + V v; + if ((v = p.val) == value || (v != null && value.equals(v))) + return true; } } - return oldVal; + return false; } - /* - * Internal versions of the six insertion methods, each a - * little more complicated than the last. All have - * the same basic structure as the first (internalPut): - * 1. If table uninitialized, create - * 2. If bin empty, try to CAS new node - * 3. If bin stale, use new table - * 4. if bin converted to TreeBin, validate and relay to TreeBin methods - * 5. Lock and validate; if valid, scan and add or update + /** + * Maps the specified key to the specified value in this table. + * Neither the key nor the value can be null. * - * The others interweave other checks and/or alternative actions: - * * Plain put checks for and performs resize after insertion. - * * putIfAbsent prescans for mapping without lock (and fails to add - * if present), which also makes pre-emptive resize checks worthwhile. - * * computeIfAbsent extends form used in putIfAbsent with additional - * mechanics to deal with, calls, potential exceptions and null - * returns from function call. - * * compute uses the same function-call mechanics, but without - * the prescans - * * merge acts as putIfAbsent in the absent case, but invokes the - * update function if present - * * putAll attempts to pre-allocate enough table space - * and more lazily performs count updates and checks. + *

The value can be retrieved by calling the {@code get} method + * with a key that is equal to the original key. * - * Someday when details settle down a bit more, it might be worth - * some factoring to reduce sprawl. + * @param key key with which the specified value is to be associated + * @param value value to be associated with the specified key + * @return the previous value associated with {@code key}, or + * {@code null} if there was no mapping for {@code key} + * @throws NullPointerException if the specified key or value is null */ + public V put(K key, V value) { + return putVal(key, value, false); + } - /** Implementation for put */ - private final Object internalPut(Object k, Object v) { - int h = spread(k.hashCode()); - int count = 0; - for (Node[] tab = table;;) { - int i; Node f; int fh; Object fk; - if (tab == null) + /** Implementation for put and putIfAbsent */ + final V putVal(K key, V value, boolean onlyIfAbsent) { + if (key == null || value == null) throw new NullPointerException(); + int hash = spread(key.hashCode()); + int binCount = 0; + for (Node[] tab = table;;) { + Node f; int n, i, fh; + if (tab == null || (n = tab.length) == 0) tab = initTable(); - else if ((f = tabAt(tab, i = (tab.length - 1) & h)) == null) { - if (casTabAt(tab, i, null, new Node(h, k, v, null))) + else if ((f = tabAt(tab, i = (n - 1) & hash)) == null) { + if (casTabAt(tab, i, null, + new Node(hash, key, value, null))) break; // no lock when adding to empty bin } - else if ((fh = f.hash) == MOVED) { - if ((fk = f.key) instanceof TreeBin) { - TreeBin t = (TreeBin)fk; - Object oldVal = null; - t.acquire(0); - try { - if (tabAt(tab, i) == f) { - count = 2; - TreeNode p = t.putTreeNode(h, k, v); - if (p != null) { - oldVal = p.val; - p.val = v; - } - } - } finally { - t.release(0); - } - if (count != 0) { - if (oldVal != null) - return oldVal; - break; - } - } - else - tab = (Node[])fk; - } - else if ((fh & LOCKED) != 0) { - checkForResize(); - f.tryAwaitLock(tab, i); - } - else if (f.casHash(fh, fh | LOCKED)) { - Object oldVal = null; - try { // needed in case equals() throws + else if ((fh = f.hash) == MOVED) + tab = helpTransfer(tab, f); + else { + V oldVal = null; + synchronized (f) { if (tabAt(tab, i) == f) { - count = 1; - for (Node e = f;; ++count) { - Object ek, ev; - if ((e.hash & HASH_BITS) == h && - (ev = e.val) != null && - ((ek = e.key) == k || k.equals(ek))) { - oldVal = ev; - e.val = v; - break; + if (fh >= 0) { + binCount = 1; + for (Node e = f;; ++binCount) { + K ek; + if (e.hash == hash && + ((ek = e.key) == key || + (ek != null && key.equals(ek)))) { + oldVal = e.val; + if (!onlyIfAbsent) + e.val = value; + break; + } + Node pred = e; + if ((e = e.next) == null) { + pred.next = new Node(hash, key, + value, null); + break; + } } - Node last = e; - if ((e = e.next) == null) { - last.next = new Node(h, k, v, null); - if (count >= TREE_THRESHOLD) - replaceWithTreeBin(tab, i, k); - break; + } + else if (f instanceof TreeBin) { + Node p; + binCount = 2; + if ((p = ((TreeBin)f).putTreeVal(hash, key, + value)) != null) { + oldVal = p.val; + if (!onlyIfAbsent) + p.val = value; } } } - } finally { // unlock and signal if needed - if (!f.casHash(fh | LOCKED, fh)) { - f.hash = fh; - synchronized (f) { f.notifyAll(); }; - } } - if (count != 0) { + if (binCount != 0) { + if (binCount >= TREEIFY_THRESHOLD) + treeifyBin(tab, i); if (oldVal != null) return oldVal; - if (tab.length <= 64) - count = 2; break; } } } - counter.add(1L); - if (count > 1) - checkForResize(); + addCount(1L, binCount); return null; } - /** Implementation for putIfAbsent */ - private final Object internalPutIfAbsent(Object k, Object v) { - int h = spread(k.hashCode()); - int count = 0; - for (Node[] tab = table;;) { - int i; Node f; int fh; Object fk, fv; - if (tab == null) - tab = initTable(); - else if ((f = tabAt(tab, i = (tab.length - 1) & h)) == null) { - if (casTabAt(tab, i, null, new Node(h, k, v, null))) - break; - } - else if ((fh = f.hash) == MOVED) { - if ((fk = f.key) instanceof TreeBin) { - TreeBin t = (TreeBin)fk; - Object oldVal = null; - t.acquire(0); - try { - if (tabAt(tab, i) == f) { - count = 2; - TreeNode p = t.putTreeNode(h, k, v); - if (p != null) - oldVal = p.val; - } - } finally { - t.release(0); - } - if (count != 0) { - if (oldVal != null) - return oldVal; - break; - } - } - else - tab = (Node[])fk; - } - else if ((fh & HASH_BITS) == h && (fv = f.val) != null && - ((fk = f.key) == k || k.equals(fk))) - return fv; - else { - Node g = f.next; - if (g != null) { // at least 2 nodes -- search and maybe resize - for (Node e = g;;) { - Object ek, ev; - if ((e.hash & HASH_BITS) == h && (ev = e.val) != null && - ((ek = e.key) == k || k.equals(ek))) - return ev; - if ((e = e.next) == null) { - checkForResize(); - break; - } - } - } - if (((fh = f.hash) & LOCKED) != 0) { - checkForResize(); - f.tryAwaitLock(tab, i); - } - else if (tabAt(tab, i) == f && f.casHash(fh, fh | LOCKED)) { - Object oldVal = null; - try { - if (tabAt(tab, i) == f) { - count = 1; - for (Node e = f;; ++count) { - Object ek, ev; - if ((e.hash & HASH_BITS) == h && -