You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@lucene.apache.org by mi...@apache.org on 2021/06/23 22:05:44 UTC

[lucene-solr] branch branch_8x updated: Backport LUCENE-9142 and LUCENE-9983 (#2517)

This is an automated email from the ASF dual-hosted git repository.

mikemccand pushed a commit to branch branch_8x
in repository https://gitbox.apache.org/repos/asf/lucene-solr.git


The following commit(s) were added to refs/heads/branch_8x by this push:
     new c6b9dd9  Backport LUCENE-9142 and LUCENE-9983 (#2517)
c6b9dd9 is described below

commit c6b9dd95c9f4b9ab9bac5904988432bba2ad4bd3
Author: Patrick Zhai <zh...@users.noreply.github.com>
AuthorDate: Wed Jun 23 15:05:26 2021 -0700

    Backport LUCENE-9142 and LUCENE-9983 (#2517)
    
    * LUCENE-9142 Refactor IntSet operations for determinize (#1184)
    
    Co-authored-by: Mike <ma...@users.noreply.github.com>
---
 lucene/CHANGES.txt                                 |   2 +
 .../apache/lucene/util/automaton/FrozenIntSet.java |  51 ++
 .../org/apache/lucene/util/automaton/IntSet.java   |  52 ++
 .../apache/lucene/util/automaton/Operations.java   |  60 +-
 .../apache/lucene/util/automaton/SortedIntSet.java | 276 ------
 .../org/apache/lucene/util/automaton/StateSet.java | 118 +++
 .../java/org/apache/lucene/util/hppc/BitMixer.java | 132 +++
 .../org/apache/lucene/util/hppc/IntIntHashMap.java | 998 +++++++++++++++++++++
 .../org/apache/lucene/util/hppc/package-info.java  |  19 +
 .../apache/lucene/util/automaton/TestIntSet.java   | 104 +++
 .../apache/lucene/util/hppc/TestIntIntHashMap.java | 633 +++++++++++++
 11 files changed, 2139 insertions(+), 306 deletions(-)

diff --git a/lucene/CHANGES.txt b/lucene/CHANGES.txt
index 47b41b7..96867b1 100644
--- a/lucene/CHANGES.txt
+++ b/lucene/CHANGES.txt
@@ -33,6 +33,8 @@ Improvements
   cases like det(rev(regexp("(.*a){2000}"))) that spend lots of effort but
   result in smallish eventual returned automata.  (Robert Muir, Mike McCandless)
 
+* LUCENE-9983: Stop sorting determinize powersets unnecessarily. (Patrick Zhai)
+
 Optimizations
 ---------------------
 (No changes)
diff --git a/lucene/core/src/java/org/apache/lucene/util/automaton/FrozenIntSet.java b/lucene/core/src/java/org/apache/lucene/util/automaton/FrozenIntSet.java
new file mode 100644
index 0000000..1e80171
--- /dev/null
+++ b/lucene/core/src/java/org/apache/lucene/util/automaton/FrozenIntSet.java
@@ -0,0 +1,51 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.lucene.util.automaton;
+
+import java.util.Arrays;
+
+final class FrozenIntSet extends IntSet {
+  final int[] values;
+  final int state;
+  final long hashCode;
+
+  FrozenIntSet(int[] values, long hashCode, int state) {
+    this.values = values;
+    this.state = state;
+    this.hashCode = hashCode;
+  }
+
+  @Override
+  int[] getArray() {
+    return values;
+  }
+
+  @Override
+  int size() {
+    return values.length;
+  }
+
+  @Override
+  long longHashCode() {
+    return hashCode;
+  }
+
+  @Override
+  public String toString() {
+    return Arrays.toString(values);
+  }
+}
diff --git a/lucene/core/src/java/org/apache/lucene/util/automaton/IntSet.java b/lucene/core/src/java/org/apache/lucene/util/automaton/IntSet.java
new file mode 100644
index 0000000..b909300
--- /dev/null
+++ b/lucene/core/src/java/org/apache/lucene/util/automaton/IntSet.java
@@ -0,0 +1,52 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.lucene.util.automaton;
+
+abstract class IntSet {
+  /**
+   * Return an array representation of this int set's values. Values are valid for indices [0,
+   * {@link #size()}). If this is a mutable int set, then changes to the set are not guaranteed to
+   * be visible in this array.
+   *
+   * @return an array containing the values for this set, guaranteed to be at least {@link #size()}
+   *     elements
+   */
+  abstract int[] getArray();
+
+  /**
+   * Guaranteed to be less than or equal to the length of the array returned by {@link #getArray()}.
+   *
+   * @return The number of values in this set.
+   */
+  abstract int size();
+
+  abstract long longHashCode();
+
+  @Override
+  public int hashCode() {
+    return Long.hashCode(longHashCode());
+  }
+
+  @Override
+  public boolean equals(Object o) {
+    if (this == o) return true;
+    if (!(o instanceof IntSet)) return false;
+    IntSet that = (IntSet) o;
+    return longHashCode() == that.longHashCode()
+        && org.apache.lucene.util.FutureArrays.equals(getArray(), 0, size(), that.getArray(), 0, that.size());
+  }
+}
diff --git a/lucene/core/src/java/org/apache/lucene/util/automaton/Operations.java b/lucene/core/src/java/org/apache/lucene/util/automaton/Operations.java
index c931bea..b41abed 100644
--- a/lucene/core/src/java/org/apache/lucene/util/automaton/Operations.java
+++ b/lucene/core/src/java/org/apache/lucene/util/automaton/Operations.java
@@ -1,9 +1,9 @@
 /*
  * dk.brics.automaton
- * 
+ *
  * Copyright (c) 2001-2009 Anders Moeller
  * All rights reserved.
- * 
+ *
  * Redistribution and use in source and binary forms, with or without
  * modification, are permitted provided that the following conditions
  * are met:
@@ -14,7 +14,7 @@
  *    documentation and/or other materials provided with the distribution.
  * 3. The name of the author may not be used to endorse or promote products
  *    derived from this software without specific prior written permission.
- * 
+ *
  * THIS SOFTWARE IS PROVIDED BY THE AUTHOR ``AS IS'' AND ANY EXPRESS OR
  * IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED TO, THE IMPLIED WARRANTIES
  * OF MERCHANTABILITY AND FITNESS FOR A PARTICULAR PURPOSE ARE DISCLAIMED.
@@ -48,10 +48,11 @@ import org.apache.lucene.util.FixedBitSet;
 import org.apache.lucene.util.IntsRef;
 import org.apache.lucene.util.IntsRefBuilder;
 import org.apache.lucene.util.RamUsageEstimator;
+import org.apache.lucene.util.hppc.BitMixer;
 
 /**
  * Automata operations.
- * 
+ *
  * @lucene.experimental
  */
 final public class Operations {
@@ -173,7 +174,7 @@ final public class Operations {
     result.finishState();
     return result;
   }
-  
+
   /**
    * Returns an automaton that accepts the Kleene star (zero or more
    * concatenated repetitions) of the language of the given automaton. Never
@@ -229,7 +230,7 @@ final public class Operations {
     as.add(repeat(a));
     return concatenate(as);
   }
-  
+
   /**
    * Returns an automaton that accepts between <code>min</code> and
    * <code>max</code> (including both) concatenated repetitions of the language
@@ -284,7 +285,7 @@ final public class Operations {
 
     return result;
   }
-  
+
   /**
    * Returns a (deterministic) automaton that accepts the complement of the
    * language of the given automaton.
@@ -303,7 +304,7 @@ final public class Operations {
     }
     return removeDeadStates(a);
   }
-  
+
   /**
    * Returns a (deterministic) automaton that accepts the intersection of the
    * language of <code>a1</code> and the complement of the language of
@@ -328,7 +329,7 @@ final public class Operations {
     }
     return intersection(a1, complement(a2, determinizeWorkLimit));
   }
-  
+
   /**
    * Returns an automaton that accepts the intersection of the languages of the
    * given automata. Never modifies the input automata languages.
@@ -517,7 +518,7 @@ final public class Operations {
     for(Automaton a : l) {
       result.copy(a);
     }
-    
+
     // Add epsilon transition from new initial state
     int stateOffset = 1;
     for(Automaton a : l) {
@@ -691,13 +692,14 @@ final public class Operations {
     //System.out.println("DET:");
     //a.writeDot("/l/la/lucene/core/detin.dot");
 
-    SortedIntSet.FrozenIntSet initialset = new SortedIntSet.FrozenIntSet(0, 0);
+    // Same initial values and state will always have the same hashCode
+    FrozenIntSet initialset = new FrozenIntSet(new int[] { 0 }, BitMixer.mix(0) + 1, 0);
 
     // Create state 0:
     b.createState();
 
-    ArrayDeque<SortedIntSet.FrozenIntSet> worklist = new ArrayDeque<>();
-    Map<SortedIntSet.FrozenIntSet,Integer> newstate = new HashMap<>();
+    ArrayDeque<FrozenIntSet> worklist = new ArrayDeque<>();
+    Map<IntSet,Integer> newstate = new HashMap<>();
 
     worklist.add(initialset);
 
@@ -707,8 +709,8 @@ final public class Operations {
     // like Set<Integer,PointTransitions>
     final PointTransitionSet points = new PointTransitionSet();
 
-    // like SortedMap<Integer,Integer>
-    final SortedIntSet statesSet = new SortedIntSet(5);
+    // like HashMap<Integer,Integer>, maps state to its count
+    final StateSet statesSet = new StateSet(5);
 
     Transition t = new Transition();
 
@@ -723,7 +725,7 @@ final public class Operations {
       // a high (unecessary) price for that!  really we just need a low-overhead Map<int,int>
       // that implements equals/hash based only on the keys (ignores the values).  fixing this
       // might be a bigspeedup for determinizing complex automata
-      SortedIntSet.FrozenIntSet s = worklist.removeFirst();
+      FrozenIntSet s = worklist.removeFirst();
 
       // LUCENE-9981: we more carefully aggregate the net work this automaton is costing us, instead
       // of (overly simplistically) counting number
@@ -760,15 +762,13 @@ final public class Operations {
 
         final int point = points.points[i].point;
 
-        if (statesSet.upto > 0) {
+        if (statesSet.size() > 0) {
           assert lastPoint != -1;
 
-          statesSet.computeHash();
-          
           Integer q = newstate.get(statesSet);
           if (q == null) {
             q = b.createState();
-            final SortedIntSet.FrozenIntSet p = statesSet.freeze(q);
+            final FrozenIntSet p = statesSet.freeze(q);
             //System.out.println("  make new state=" + q + " -> " + p + " accCount=" + accCount);
             worklist.add(p);
             b.setAccept(q, accCount > 0);
@@ -807,7 +807,7 @@ final public class Operations {
         points.points[i].starts.next = 0;
       }
       points.reset();
-      assert statesSet.upto == 0: "upto=" + statesSet.upto;
+      assert statesSet.size() == 0: "size=" + statesSet.size();
     }
 
     Automaton result = b.finish();
@@ -831,7 +831,7 @@ final public class Operations {
       // Apparently common case: it accepts the damned empty string
       return false;
     }
-    
+
     ArrayDeque<Integer> workList = new ArrayDeque<>();
     BitSet seen = new BitSet(a.getNumStates());
     workList.add(0);
@@ -855,7 +855,7 @@ final public class Operations {
 
     return true;
   }
-  
+
   /**
    * Returns true if the given automaton accepts all strings.  The automaton must be minimized.
    */
@@ -877,7 +877,7 @@ final public class Operations {
     }
     return false;
   }
-  
+
   /**
    * Returns true if the given string is accepted by the automaton.  The input must be deterministic.
    * <p>
@@ -1048,7 +1048,7 @@ final public class Operations {
     }
     return isFinite(new Transition(), a, 0, new BitSet(a.getNumStates()), new BitSet(a.getNumStates()), 0);
   }
-  
+
   /**
    * Checks whether there is a loop containing state. (This is sufficient since
    * there are never transitions to dead states.)
@@ -1071,13 +1071,13 @@ final public class Operations {
     visited.set(state);
     return true;
   }
-  
+
   /**
    * Returns the longest string that is a prefix of all accepted strings and
    * visits each state at most once. The automaton must not have dead states.
    * If this automaton has already been converted to UTF-8 (e.g. using
    * {@link UTF32ToUTF8}) then you should use {@link #getCommonPrefixBytesRef} instead.
-   * 
+   *
    * @throws IllegalArgumentException if the automaton has dead states reachable from the initial
    *     state.
    * @return common prefix, which can be an empty (length 0) String (never null)
@@ -1135,11 +1135,11 @@ final public class Operations {
     }
     return builder.toString();
   }
-  
+
   /**
    * Returns the longest BytesRef that is a prefix of all accepted strings and
    * visits each state at most once.
-   * 
+   *
    * @return common prefix, which can be an empty (length 0) BytesRef (never null), and might
    *     possibly include a UTF-8 fragment of a full Unicode character
    */
@@ -1200,7 +1200,7 @@ final public class Operations {
     reverseBytes(ref);
     return ref;
   }
-  
+
   private static void reverseBytes(BytesRef ref) {
     if (ref.length <= 1) return;
     int num = ref.length >> 1;
diff --git a/lucene/core/src/java/org/apache/lucene/util/automaton/SortedIntSet.java b/lucene/core/src/java/org/apache/lucene/util/automaton/SortedIntSet.java
deleted file mode 100644
index 3251aad..0000000
--- a/lucene/core/src/java/org/apache/lucene/util/automaton/SortedIntSet.java
+++ /dev/null
@@ -1,276 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.lucene.util.automaton;
-
-import java.util.TreeMap;
-import java.util.Map;
-import org.apache.lucene.util.ArrayUtil;
-
-// Just holds a set of int[] states, plus a corresponding
-// int[] count per state.  Used by
-// BasicOperations.determinize
-final class SortedIntSet {
-  int[] values;
-  int[] counts;
-  int upto;
-  private int hashCode;
-
-  // If we hold more than this many states, we switch from
-  // O(N^2) linear ops to O(N log(N)) TreeMap
-  private final static int TREE_MAP_CUTOVER = 30;
-
-  private final Map<Integer,Integer> map = new TreeMap<>();
-
-  private boolean useTreeMap;
-
-  int state;
-
-  public SortedIntSet(int capacity) {
-    values = new int[capacity];
-    counts = new int[capacity];
-  }
-
-  // Adds this state to the set
-  public void incr(int num) {
-    if (useTreeMap) {
-      final Integer key = num;
-      Integer val = map.get(key);
-      if (val == null) {
-        map.put(key, 1);
-      } else {
-        map.put(key, 1+val);
-      }
-      return;
-    }
-
-    if (upto == values.length) {
-      values = ArrayUtil.grow(values, 1+upto);
-      counts = ArrayUtil.grow(counts, 1+upto);
-    }
-
-    for(int i=0;i<upto;i++) {
-      if (values[i] == num) {
-        counts[i]++;
-        return;
-      } else if (num < values[i]) {
-        // insert here
-        int j = upto-1;
-        while (j >= i) {
-          values[1+j] = values[j];
-          counts[1+j] = counts[j];
-          j--;
-        }
-        values[i] = num;
-        counts[i] = 1;
-        upto++;
-        return;
-      }
-    }
-
-    // append
-    values[upto] = num;
-    counts[upto] = 1;
-    upto++;
-
-    if (upto == TREE_MAP_CUTOVER) {
-      useTreeMap = true;
-      for(int i=0;i<upto;i++) {
-        map.put(values[i], counts[i]);
-      }
-    }
-  }
-
-  // Removes this state from the set, if count decrs to 0
-  public void decr(int num) {
-
-    if (useTreeMap) {
-      final int count = map.get(num);
-      if (count == 1) {
-        map.remove(num);
-      } else {
-        map.put(num, count-1);
-      }
-      // Fall back to simple arrays once we touch zero again
-      if (map.size() == 0) {
-        useTreeMap = false;
-        upto = 0;
-      }
-      return;
-    }
-
-    for(int i=0;i<upto;i++) {
-      if (values[i] == num) {
-        counts[i]--;
-        if (counts[i] == 0) {
-          final int limit = upto-1;
-          while(i < limit) {
-            values[i] = values[i+1];
-            counts[i] = counts[i+1];
-            i++;
-          }
-          upto = limit;
-        }
-        return;
-      }
-    }
-    assert false;
-  }
-
-  public void computeHash() {
-    if (useTreeMap) {
-      if (map.size() > values.length) {
-        final int size = ArrayUtil.oversize(map.size(), Integer.BYTES);
-        values = new int[size];
-        counts = new int[size];
-      }
-      hashCode = map.size();
-      upto = 0;
-      for(int state : map.keySet()) {
-        hashCode = 683*hashCode + state;
-        values[upto++] = state;
-      }
-    } else {
-      hashCode = upto;
-      for(int i=0;i<upto;i++) {
-        hashCode = 683*hashCode + values[i];
-      }
-    }
-  }
-
-  public FrozenIntSet freeze(int state) {
-    final int[] c = new int[upto];
-    System.arraycopy(values, 0, c, 0, upto);
-    return new FrozenIntSet(c, hashCode, state);
-  }
-
-  @Override
-  public int hashCode() {
-    return hashCode;
-  }
-
-  @Override
-  public boolean equals(Object _other) {
-    if (_other == null) {
-      return false;
-    }
-    if (!(_other instanceof FrozenIntSet)) {
-      return false;
-    }
-    FrozenIntSet other = (FrozenIntSet) _other;
-    if (hashCode != other.hashCode) {
-      return false;
-    }
-    if (other.values.length != upto) {
-      return false;
-    }
-    for(int i=0;i<upto;i++) {
-      if (other.values[i] != values[i]) {
-        return false;
-      }
-    }
-
-    return true;
-  }
-
-  @Override
-  public String toString() {
-    StringBuilder sb = new StringBuilder().append('[');
-    for(int i=0;i<upto;i++) {
-      if (i > 0) {
-        sb.append(' ');
-      }
-      sb.append(values[i]).append(':').append(counts[i]);
-    }
-    sb.append(']');
-    return sb.toString();
-  }
-  
-  public final static class FrozenIntSet {
-    final int[] values;
-    final int hashCode;
-    final int state;
-
-    public FrozenIntSet(int[] values, int hashCode, int state) {
-      this.values = values;
-      this.hashCode = hashCode;
-      this.state = state;
-    }
-
-    public FrozenIntSet(int num, int state) {
-      this.values = new int[] {num};
-      this.state = state;
-      this.hashCode = 683+num;
-    }
-
-    @Override
-    public int hashCode() {
-      return hashCode;
-    }
-
-    @Override
-    public boolean equals(Object _other) {
-      if (_other == null) {
-        return false;
-      }
-      if (_other instanceof FrozenIntSet) {
-        FrozenIntSet other = (FrozenIntSet) _other;
-        if (hashCode != other.hashCode) {
-          return false;
-        }
-        if (other.values.length != values.length) {
-          return false;
-        }
-        for(int i=0;i<values.length;i++) {
-          if (other.values[i] != values[i]) {
-            return false;
-          }
-        }
-        return true;
-      } else if (_other instanceof SortedIntSet) {
-        SortedIntSet other = (SortedIntSet) _other;
-        if (hashCode != other.hashCode) {
-          return false;
-        }
-        if (other.values.length != values.length) {
-          return false;
-        }
-        for(int i=0;i<values.length;i++) {
-          if (other.values[i] != values[i]) {
-            return false;
-          }
-        }
-        return true;
-      }
-
-      return false;
-    }
-
-    @Override
-    public String toString() {
-      StringBuilder sb = new StringBuilder().append('[');
-      for(int i=0;i<values.length;i++) {
-        if (i > 0) {
-          sb.append(' ');
-        }
-        sb.append(values[i]);
-      }
-      sb.append(']');
-      return sb.toString();
-    }
-  }
-}
-  
diff --git a/lucene/core/src/java/org/apache/lucene/util/automaton/StateSet.java b/lucene/core/src/java/org/apache/lucene/util/automaton/StateSet.java
new file mode 100644
index 0000000..6d4a7ea
--- /dev/null
+++ b/lucene/core/src/java/org/apache/lucene/util/automaton/StateSet.java
@@ -0,0 +1,118 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.lucene.util.automaton;
+
+import java.util.Arrays;
+import org.apache.lucene.util.hppc.BitMixer;
+import org.apache.lucene.util.hppc.IntIntHashMap;
+
+/**
+ * A thin wrapper of {@link IntIntHashMap} Maps from state in integer representation to its
+ * reference count Whenever the count of a state is 0, that state will be removed from the set
+ */
+final class StateSet extends IntSet {
+
+  private final IntIntHashMap inner;
+  private long hashCode;
+  private boolean hashUpdated = true;
+  private boolean arrayUpdated = true;
+  private int[] arrayCache = new int[0];
+
+  StateSet(int capacity) {
+    inner = new IntIntHashMap(capacity);
+  }
+
+  /**
+   * Add the state into this set, if it is already there, increase its reference count by 1
+   *
+   * @param state an integer representing this state
+   */
+  void incr(int state) {
+    if (inner.addTo(state, 1) == 1) {
+      keyChanged();
+    }
+  }
+
+  /**
+   * Decrease the reference count of the state, if the count down to 0, remove the state from this
+   * set
+   *
+   * @param state an integer representing this state
+   */
+  void decr(int state) {
+    assert inner.containsKey(state);
+    int keyIndex = inner.indexOf(state);
+    int count = inner.indexGet(keyIndex) - 1;
+    if (count == 0) {
+      inner.indexRemove(keyIndex);
+      keyChanged();
+    } else {
+      inner.indexReplace(keyIndex, count);
+    }
+  }
+
+  /**
+   * Create a snapshot of this int set associated with a given state. The snapshot will not retain
+   * any frequency information about the elements of this set, only existence.
+   *
+   * @param state the state to associate with the frozen set.
+   * @return A new FrozenIntSet with the same values as this set.
+   */
+  FrozenIntSet freeze(int state) {
+    return new FrozenIntSet(getArray(), longHashCode(), state);
+  }
+
+  private void keyChanged() {
+    hashUpdated = false;
+    arrayUpdated = false;
+  }
+
+  @Override
+  int[] getArray() {
+    if (arrayUpdated) {
+      return arrayCache;
+    }
+    arrayCache = new int[inner.size()];
+    int i = 0;
+    for (IntIntHashMap.IntCursor cursor : inner.keys()) {
+      arrayCache[i++] = cursor.value;
+    }
+    // we need to sort this array since "equals" method depend on this
+    Arrays.sort(arrayCache);
+    arrayUpdated = true;
+    return arrayCache;
+  }
+
+  @Override
+  int size() {
+    return inner.size();
+  }
+
+  @Override
+  long longHashCode() {
+    if (hashUpdated) {
+      return hashCode;
+    }
+    hashCode = inner.size();
+    for (IntIntHashMap.IntCursor cursor : inner.keys()) {
+      hashCode += BitMixer.mix(cursor.value);
+    }
+    hashUpdated = true;
+    return hashCode;
+  }
+}
diff --git a/lucene/core/src/java/org/apache/lucene/util/hppc/BitMixer.java b/lucene/core/src/java/org/apache/lucene/util/hppc/BitMixer.java
new file mode 100644
index 0000000..36d239a
--- /dev/null
+++ b/lucene/core/src/java/org/apache/lucene/util/hppc/BitMixer.java
@@ -0,0 +1,132 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.lucene.util.hppc;
+
+/**
+ * Bit mixing utilities. The purpose of these methods is to evenly distribute key space over int32
+ * range.
+ *
+ * <p>Forked from com.carrotsearch.hppc.BitMixer
+ *
+ * <p>github: https://github.com/carrotsearch/hppc release: 0.9.0
+ */
+public final class BitMixer {
+
+  // Don't bother mixing very small key domains much.
+  public static int mix(byte key) {
+    return key * PHI_C32;
+  }
+
+  public static int mix(short key) {
+    return mixPhi(key);
+  }
+
+  public static int mix(char key) {
+    return mixPhi(key);
+  }
+
+  // Better mix for larger key domains.
+  public static int mix(int key) {
+    return mix32(key);
+  }
+
+  public static int mix(float key) {
+    return mix32(Float.floatToIntBits(key));
+  }
+
+  public static int mix(double key) {
+    return (int) mix64(Double.doubleToLongBits(key));
+  }
+
+  public static int mix(long key) {
+    return (int) mix64(key);
+  }
+
+  public static int mix(Object key) {
+    return key == null ? 0 : mix32(key.hashCode());
+  }
+
+  /** MH3's plain finalization step. */
+  public static int mix32(int k) {
+    k = (k ^ (k >>> 16)) * 0x85ebca6b;
+    k = (k ^ (k >>> 13)) * 0xc2b2ae35;
+    return k ^ (k >>> 16);
+  }
+
+  /**
+   * Computes David Stafford variant 9 of 64bit mix function (MH3 finalization step, with different
+   * shifts and constants).
+   *
+   * <p>Variant 9 is picked because it contains two 32-bit shifts which could be possibly optimized
+   * into better machine code.
+   *
+   * @see "http://zimbry.blogspot.com/2011/09/better-bit-mixing-improving-on.html"
+   */
+  public static long mix64(long z) {
+    z = (z ^ (z >>> 32)) * 0x4cd6944c5cc20b6dL;
+    z = (z ^ (z >>> 29)) * 0xfc12c5b19d3259e9L;
+    return z ^ (z >>> 32);
+  }
+
+  /*
+   * Golden ratio bit mixers.
+   */
+
+  private static final int PHI_C32 = 0x9e3779b9;
+  private static final long PHI_C64 = 0x9e3779b97f4a7c15L;
+
+  public static int mixPhi(byte k) {
+    final int h = k * PHI_C32;
+    return h ^ (h >>> 16);
+  }
+
+  public static int mixPhi(char k) {
+    final int h = k * PHI_C32;
+    return h ^ (h >>> 16);
+  }
+
+  public static int mixPhi(short k) {
+    final int h = k * PHI_C32;
+    return h ^ (h >>> 16);
+  }
+
+  public static int mixPhi(int k) {
+    final int h = k * PHI_C32;
+    return h ^ (h >>> 16);
+  }
+
+  public static int mixPhi(float k) {
+    final int h = Float.floatToIntBits(k) * PHI_C32;
+    return h ^ (h >>> 16);
+  }
+
+  public static int mixPhi(double k) {
+    final long h = Double.doubleToLongBits(k) * PHI_C64;
+    return (int) (h ^ (h >>> 32));
+  }
+
+  public static int mixPhi(long k) {
+    final long h = k * PHI_C64;
+    return (int) (h ^ (h >>> 32));
+  }
+
+  public static int mixPhi(Object k) {
+    final int h = (k == null ? 0 : k.hashCode() * PHI_C32);
+    return h ^ (h >>> 16);
+  }
+}
diff --git a/lucene/core/src/java/org/apache/lucene/util/hppc/IntIntHashMap.java b/lucene/core/src/java/org/apache/lucene/util/hppc/IntIntHashMap.java
new file mode 100644
index 0000000..e7876f8
--- /dev/null
+++ b/lucene/core/src/java/org/apache/lucene/util/hppc/IntIntHashMap.java
@@ -0,0 +1,998 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.lucene.util.hppc;
+
+import static org.apache.lucene.util.BitUtil.nextHighestPowerOfTwo;
+
+import java.util.Arrays;
+import java.util.IllegalFormatException;
+import java.util.Iterator;
+import java.util.Locale;
+import java.util.NoSuchElementException;
+import java.util.concurrent.atomic.AtomicInteger;
+
+/**
+ * A hash map of <code>int</code> to <code>int</code>, implemented using open addressing with linear
+ * probing for collision resolution.
+ *
+ * <p>Mostly forked and trimmed from com.carrotsearch.hppc.IntIntHashMap
+ *
+ * <p>github: https://github.com/carrotsearch/hppc release 0.9.0
+ */
+public class IntIntHashMap implements Iterable<IntIntHashMap.IntIntCursor>, Cloneable {
+
+  public static final int DEFAULT_EXPECTED_ELEMENTS = 4;
+
+  public static final float DEFAULT_LOAD_FACTOR = 0.75f;
+
+  private static final AtomicInteger ITERATION_SEED = new AtomicInteger();
+
+  /** Minimal sane load factor (99 empty slots per 100). */
+  public static final float MIN_LOAD_FACTOR = 1 / 100.0f;
+
+  /** Maximum sane load factor (1 empty slot per 100). */
+  public static final float MAX_LOAD_FACTOR = 99 / 100.0f;
+
+  /** Minimum hash buffer size. */
+  public static final int MIN_HASH_ARRAY_LENGTH = 4;
+
+  /**
+   * Maximum array size for hash containers (power-of-two and still allocable in Java, not a
+   * negative int).
+   */
+  public static final int MAX_HASH_ARRAY_LENGTH = 0x80000000 >>> 1;
+
+  /** The array holding keys. */
+  public int[] keys;
+
+  /** The array holding values. */
+  public int[] values;
+
+  /**
+   * The number of stored keys (assigned key slots), excluding the special "empty" key, if any (use
+   * {@link #size()} instead).
+   *
+   * @see #size()
+   */
+  protected int assigned;
+
+  /** Mask for slot scans in {@link #keys}. */
+  protected int mask;
+
+  /** Expand (rehash) {@link #keys} when {@link #assigned} hits this value. */
+  protected int resizeAt;
+
+  /** Special treatment for the "empty slot" key marker. */
+  protected boolean hasEmptyKey;
+
+  /** The load factor for {@link #keys}. */
+  protected double loadFactor;
+
+  /** Seed used to ensure the hash iteration order is different from an iteration to another. */
+  protected int iterationSeed;
+
+  /** New instance with sane defaults. */
+  public IntIntHashMap() {
+    this(DEFAULT_EXPECTED_ELEMENTS);
+  }
+
+  /**
+   * New instance with sane defaults.
+   *
+   * @param expectedElements The expected number of elements guaranteed not to cause buffer
+   *     expansion (inclusive).
+   */
+  public IntIntHashMap(int expectedElements) {
+    this(expectedElements, DEFAULT_LOAD_FACTOR);
+  }
+
+  /**
+   * New instance with the provided defaults.
+   *
+   * @param expectedElements The expected number of elements guaranteed not to cause a rehash
+   *     (inclusive).
+   * @param loadFactor The load factor for internal buffers. Insane load factors (zero, full
+   *     capacity) are rejected by {@link #verifyLoadFactor(double)}.
+   */
+  public IntIntHashMap(int expectedElements, double loadFactor) {
+    this.loadFactor = verifyLoadFactor(loadFactor);
+    iterationSeed = ITERATION_SEED.incrementAndGet();
+    ensureCapacity(expectedElements);
+  }
+
+  /** Create a hash map from all key-value pairs of another container. */
+  public IntIntHashMap(Iterable<? extends IntIntCursor> container) {
+    this();
+    putAll(container);
+  }
+
+  public int put(int key, int value) {
+    assert assigned < mask + 1;
+
+    final int mask = this.mask;
+    if (((key) == 0)) {
+      hasEmptyKey = true;
+      int previousValue = values[mask + 1];
+      values[mask + 1] = value;
+      return previousValue;
+    } else {
+      final int[] keys = this.keys;
+      int slot = hashKey(key) & mask;
+
+      int existing;
+      while (!((existing = keys[slot]) == 0)) {
+        if (((existing) == (key))) {
+          final int previousValue = values[slot];
+          values[slot] = value;
+          return previousValue;
+        }
+        slot = (slot + 1) & mask;
+      }
+
+      if (assigned == resizeAt) {
+        allocateThenInsertThenRehash(slot, key, value);
+      } else {
+        keys[slot] = key;
+        values[slot] = value;
+      }
+
+      assigned++;
+      return 0;
+    }
+  }
+
+  public int putAll(Iterable<? extends IntIntCursor> iterable) {
+    final int count = size();
+    for (IntIntCursor c : iterable) {
+      put(c.key, c.value);
+    }
+    return size() - count;
+  }
+
+  /**
+   * <a href="http://trove4j.sourceforge.net">Trove</a>-inspired API method. An equivalent of the
+   * following code:
+   *
+   * <pre>
+   * if (!map.containsKey(key)) map.put(value);
+   * </pre>
+   *
+   * @param key The key of the value to check.
+   * @param value The value to put if <code>key</code> does not exist.
+   * @return <code>true</code> if <code>key</code> did not exist and <code>value</code> was placed
+   *     in the map.
+   */
+  public boolean putIfAbsent(int key, int value) {
+    int keyIndex = indexOf(key);
+    if (!indexExists(keyIndex)) {
+      indexInsert(keyIndex, key, value);
+      return true;
+    } else {
+      return false;
+    }
+  }
+
+  /**
+   * If <code>key</code> exists, <code>putValue</code> is inserted into the map, otherwise any
+   * existing value is incremented by <code>additionValue</code>.
+   *
+   * @param key The key of the value to adjust.
+   * @param putValue The value to put if <code>key</code> does not exist.
+   * @param incrementValue The value to add to the existing value if <code>key</code> exists.
+   * @return Returns the current value associated with <code>key</code> (after changes).
+   */
+  public int putOrAdd(int key, int putValue, int incrementValue) {
+    assert assigned < mask + 1;
+
+    int keyIndex = indexOf(key);
+    if (indexExists(keyIndex)) {
+      putValue = values[keyIndex] + incrementValue;
+      indexReplace(keyIndex, putValue);
+    } else {
+      indexInsert(keyIndex, key, putValue);
+    }
+    return putValue;
+  }
+
+  /**
+   * Adds <code>incrementValue</code> to any existing value for the given <code>key</code> or
+   * inserts <code>incrementValue</code> if <code>key</code> did not previously exist.
+   *
+   * @param key The key of the value to adjust.
+   * @param incrementValue The value to put or add to the existing value if <code>key</code> exists.
+   * @return Returns the current value associated with <code>key</code> (after changes).
+   */
+  public int addTo(int key, int incrementValue) {
+    return putOrAdd(key, incrementValue, incrementValue);
+  }
+
+  public int remove(int key) {
+    final int mask = this.mask;
+    if (((key) == 0)) {
+      hasEmptyKey = false;
+      int previousValue = values[mask + 1];
+      values[mask + 1] = 0;
+      return previousValue;
+    } else {
+      final int[] keys = this.keys;
+      int slot = hashKey(key) & mask;
+
+      int existing;
+      while (!((existing = keys[slot]) == 0)) {
+        if (((existing) == (key))) {
+          final int previousValue = values[slot];
+          shiftConflictingKeys(slot);
+          return previousValue;
+        }
+        slot = (slot + 1) & mask;
+      }
+
+      return 0;
+    }
+  }
+
+  public int get(int key) {
+    if (((key) == 0)) {
+      return hasEmptyKey ? values[mask + 1] : 0;
+    } else {
+      final int[] keys = this.keys;
+      final int mask = this.mask;
+      int slot = hashKey(key) & mask;
+
+      int existing;
+      while (!((existing = keys[slot]) == 0)) {
+        if (((existing) == (key))) {
+          return values[slot];
+        }
+        slot = (slot + 1) & mask;
+      }
+
+      return 0;
+    }
+  }
+
+  public int getOrDefault(int key, int defaultValue) {
+    if (((key) == 0)) {
+      return hasEmptyKey ? values[mask + 1] : defaultValue;
+    } else {
+      final int[] keys = this.keys;
+      final int mask = this.mask;
+      int slot = hashKey(key) & mask;
+
+      int existing;
+      while (!((existing = keys[slot]) == 0)) {
+        if (((existing) == (key))) {
+          return values[slot];
+        }
+        slot = (slot + 1) & mask;
+      }
+
+      return defaultValue;
+    }
+  }
+
+  public boolean containsKey(int key) {
+    if (((key) == 0)) {
+      return hasEmptyKey;
+    } else {
+      final int[] keys = this.keys;
+      final int mask = this.mask;
+      int slot = hashKey(key) & mask;
+
+      int existing;
+      while (!((existing = keys[slot]) == 0)) {
+        if (((existing) == (key))) {
+          return true;
+        }
+        slot = (slot + 1) & mask;
+      }
+
+      return false;
+    }
+  }
+
+  public int indexOf(int key) {
+    final int mask = this.mask;
+    if (((key) == 0)) {
+      return hasEmptyKey ? mask + 1 : ~(mask + 1);
+    } else {
+      final int[] keys = this.keys;
+      int slot = hashKey(key) & mask;
+
+      int existing;
+      while (!((existing = keys[slot]) == 0)) {
+        if (((existing) == (key))) {
+          return slot;
+        }
+        slot = (slot + 1) & mask;
+      }
+
+      return ~slot;
+    }
+  }
+
+  public boolean indexExists(int index) {
+    assert index < 0 || (index >= 0 && index <= mask) || (index == mask + 1 && hasEmptyKey);
+
+    return index >= 0;
+  }
+
+  public int indexGet(int index) {
+    assert index >= 0 : "The index must point at an existing key.";
+    assert index <= mask || (index == mask + 1 && hasEmptyKey);
+
+    return values[index];
+  }
+
+  public int indexReplace(int index, int newValue) {
+    assert index >= 0 : "The index must point at an existing key.";
+    assert index <= mask || (index == mask + 1 && hasEmptyKey);
+
+    int previousValue = values[index];
+    values[index] = newValue;
+    return previousValue;
+  }
+
+  public void indexInsert(int index, int key, int value) {
+    assert index < 0 : "The index must not point at an existing key.";
+
+    index = ~index;
+    if (((key) == 0)) {
+      assert index == mask + 1;
+      values[index] = value;
+      hasEmptyKey = true;
+    } else {
+      assert ((keys[index]) == 0);
+
+      if (assigned == resizeAt) {
+        allocateThenInsertThenRehash(index, key, value);
+      } else {
+        keys[index] = key;
+        values[index] = value;
+      }
+
+      assigned++;
+    }
+  }
+
+  public int indexRemove(int index) {
+    assert index >= 0 : "The index must point at an existing key.";
+    assert index <= mask || (index == mask + 1 && hasEmptyKey);
+
+    int previousValue = values[index];
+    if (index > mask) {
+      hasEmptyKey = false;
+      values[index] = 0;
+    } else {
+      shiftConflictingKeys(index);
+    }
+    return previousValue;
+  }
+
+  public void clear() {
+    assigned = 0;
+    hasEmptyKey = false;
+
+    Arrays.fill(keys, 0);
+
+    /*  */
+  }
+
+  public void release() {
+    assigned = 0;
+    hasEmptyKey = false;
+
+    keys = null;
+    values = null;
+    ensureCapacity(DEFAULT_EXPECTED_ELEMENTS);
+  }
+
+  public int size() {
+    return assigned + (hasEmptyKey ? 1 : 0);
+  }
+
+  public boolean isEmpty() {
+    return size() == 0;
+  }
+
+  @Override
+  public int hashCode() {
+    int h = hasEmptyKey ? 0xDEADBEEF : 0;
+    for (IntIntCursor c : this) {
+      h += BitMixer.mix(c.key) + BitMixer.mix(c.value);
+    }
+    return h;
+  }
+
+  @Override
+  public boolean equals(Object obj) {
+    return obj != null && getClass() == obj.getClass() && equalElements(getClass().cast(obj));
+  }
+
+  /** Return true if all keys of some other container exist in this container. */
+  protected boolean equalElements(IntIntHashMap other) {
+    if (other.size() != size()) {
+      return false;
+    }
+
+    for (IntIntCursor c : other) {
+      int key = c.key;
+      if (!containsKey(key) || !((get(key)) == (c.value))) {
+        return false;
+      }
+    }
+
+    return true;
+  }
+
+  /**
+   * Ensure this container can hold at least the given number of keys (entries) without resizing its
+   * buffers.
+   *
+   * @param expectedElements The total number of keys, inclusive.
+   */
+  public void ensureCapacity(int expectedElements) {
+    if (expectedElements > resizeAt || keys == null) {
+      final int[] prevKeys = this.keys;
+      final int[] prevValues = this.values;
+      allocateBuffers(minBufferSize(expectedElements, loadFactor));
+      if (prevKeys != null && !isEmpty()) {
+        rehash(prevKeys, prevValues);
+      }
+    }
+  }
+
+  /**
+   * Provides the next iteration seed used to build the iteration starting slot and offset
+   * increment. This method does not need to be synchronized, what matters is that each thread gets
+   * a sequence of varying seeds.
+   */
+  protected int nextIterationSeed() {
+    return iterationSeed = BitMixer.mixPhi(iterationSeed);
+  }
+
+  /** An iterator implementation for {@link #iterator}. */
+  private final class EntryIterator extends AbstractIterator<IntIntCursor> {
+    private final IntIntCursor cursor;
+    private final int increment;
+    private int index;
+    private int slot;
+
+    public EntryIterator() {
+      cursor = new IntIntCursor();
+      int seed = nextIterationSeed();
+      increment = iterationIncrement(seed);
+      slot = seed & mask;
+    }
+
+    @Override
+    protected IntIntCursor fetch() {
+      final int mask = IntIntHashMap.this.mask;
+      while (index <= mask) {
+        int existing;
+        index++;
+        slot = (slot + increment) & mask;
+        if (!((existing = keys[slot]) == 0)) {
+          cursor.index = slot;
+          cursor.key = existing;
+          cursor.value = values[slot];
+          return cursor;
+        }
+      }
+
+      if (index == mask + 1 && hasEmptyKey) {
+        cursor.index = index;
+        cursor.key = 0;
+        cursor.value = values[index++];
+        return cursor;
+      }
+
+      return done();
+    }
+  }
+
+  @Override
+  public Iterator<IntIntCursor> iterator() {
+    return new EntryIterator();
+  }
+
+  /** Returns a specialized view of the keys of this associated container. */
+  public KeysContainer keys() {
+    return new KeysContainer();
+  }
+
+  /** A view of the keys inside this hash map. */
+  public final class KeysContainer extends IntContainer {
+    @Override
+    public Iterator<IntCursor> iterator() {
+      return new KeysIterator();
+    }
+  }
+  ;
+
+  /** An iterator over the set of assigned keys. */
+  private final class KeysIterator extends AbstractIterator<IntCursor> {
+    private final IntCursor cursor;
+    private final int increment;
+    private int index;
+    private int slot;
+
+    public KeysIterator() {
+      cursor = new IntCursor();
+      int seed = nextIterationSeed();
+      increment = iterationIncrement(seed);
+      slot = seed & mask;
+    }
+
+    @Override
+    protected IntCursor fetch() {
+      final int mask = IntIntHashMap.this.mask;
+      while (index <= mask) {
+        int existing;
+        index++;
+        slot = (slot + increment) & mask;
+        if (!((existing = keys[slot]) == 0)) {
+          cursor.index = slot;
+          cursor.value = existing;
+          return cursor;
+        }
+      }
+
+      if (index == mask + 1 && hasEmptyKey) {
+        cursor.index = index++;
+        cursor.value = 0;
+        return cursor;
+      }
+
+      return done();
+    }
+  }
+
+  /** @return Returns a container with all values stored in this map. */
+  public IntContainer values() {
+    return new ValuesContainer();
+  }
+
+  /** A view over the set of values of this map. */
+  private final class ValuesContainer extends IntContainer {
+    @Override
+    public Iterator<IntCursor> iterator() {
+      return new ValuesIterator();
+    }
+  }
+
+  /** IntCursor iterable with size and toArray function implemented */
+  public abstract class IntContainer implements Iterable<IntCursor> {
+
+    public int size() {
+      return IntIntHashMap.this.size();
+    }
+
+    public int[] toArray() {
+      int[] array = new int[size()];
+      int i = 0;
+      for (IntCursor cursor : this) {
+        array[i++] = cursor.value;
+      }
+      return array;
+    }
+  }
+
+  /** An iterator over the set of assigned values. */
+  private final class ValuesIterator extends AbstractIterator<IntCursor> {
+    private final IntCursor cursor;
+    private final int increment;
+    private int index;
+    private int slot;
+
+    public ValuesIterator() {
+      cursor = new IntCursor();
+      int seed = nextIterationSeed();
+      increment = iterationIncrement(seed);
+      slot = seed & mask;
+    }
+
+    @Override
+    protected IntCursor fetch() {
+      final int mask = IntIntHashMap.this.mask;
+      while (index <= mask) {
+        index++;
+        slot = (slot + increment) & mask;
+        if (!((keys[slot]) == 0)) {
+          cursor.index = slot;
+          cursor.value = values[slot];
+          return cursor;
+        }
+      }
+
+      if (index == mask + 1 && hasEmptyKey) {
+        cursor.index = index;
+        cursor.value = values[index++];
+        return cursor;
+      }
+
+      return done();
+    }
+  }
+
+  /** Simplifies the implementation of iterators a bit. Modeled loosely after Google Guava's API. */
+  public abstract static class AbstractIterator<E> implements Iterator<E> {
+    private static final int NOT_CACHED = 0;
+    private static final int CACHED = 1;
+    private static final int AT_END = 2;
+
+    /** Current iterator state. */
+    private int state = NOT_CACHED;
+
+    /** The next element to be returned from {@link #next()} if fetched. */
+    private E nextElement;
+
+    @Override
+    public boolean hasNext() {
+      if (state == NOT_CACHED) {
+        state = CACHED;
+        nextElement = fetch();
+      }
+      return state == CACHED;
+    }
+
+    @Override
+    public E next() {
+      if (!hasNext()) {
+        throw new NoSuchElementException();
+      }
+
+      state = NOT_CACHED;
+      return nextElement;
+    }
+
+    /** Default implementation throws {@link UnsupportedOperationException}. */
+    @Override
+    public void remove() {
+      throw new UnsupportedOperationException();
+    }
+
+    /**
+     * Fetch next element. The implementation must return {@link #done()} when all elements have
+     * been fetched.
+     *
+     * @return Returns the next value for the iterator or chain-calls {@link #done()}.
+     */
+    protected abstract E fetch();
+
+    /**
+     * Call when done.
+     *
+     * @return Returns a unique sentinel value to indicate end-of-iteration.
+     */
+    protected final E done() {
+      state = AT_END;
+      return null;
+    }
+  }
+
+  @Override
+  public IntIntHashMap clone() {
+    try {
+      /*  */
+      IntIntHashMap cloned = (IntIntHashMap) super.clone();
+      cloned.keys = keys.clone();
+      cloned.values = values.clone();
+      cloned.hasEmptyKey = hasEmptyKey;
+      cloned.iterationSeed = nextIterationSeed();
+      return cloned;
+    } catch (CloneNotSupportedException e) {
+      throw new RuntimeException(e);
+    }
+  }
+
+  /** Convert the contents of this map to a human-friendly string. */
+  @Override
+  public String toString() {
+    final StringBuilder buffer = new StringBuilder();
+    buffer.append("[");
+
+    boolean first = true;
+    for (IntIntCursor cursor : this) {
+      if (!first) {
+        buffer.append(", ");
+      }
+      buffer.append(cursor.key);
+      buffer.append("=>");
+      buffer.append(cursor.value);
+      first = false;
+    }
+    buffer.append("]");
+    return buffer.toString();
+  }
+
+  /** Creates a hash map from two index-aligned arrays of key-value pairs. */
+  public static IntIntHashMap from(int[] keys, int[] values) {
+    if (keys.length != values.length) {
+      throw new IllegalArgumentException(
+          "Arrays of keys and values must have an identical length.");
+    }
+
+    IntIntHashMap map = new IntIntHashMap(keys.length);
+    for (int i = 0; i < keys.length; i++) {
+      map.put(keys[i], values[i]);
+    }
+
+    return map;
+  }
+
+  /**
+   * Returns a hash code for the given key.
+   *
+   * <p>The output from this function should evenly distribute keys across the entire integer range.
+   */
+  protected int hashKey(int key) {
+    assert !((key) == 0); // Handled as a special case (empty slot marker).
+    return BitMixer.mixPhi(key);
+  }
+
+  /**
+   * Validate load factor range and return it. Override and suppress if you need insane load
+   * factors.
+   */
+  protected double verifyLoadFactor(double loadFactor) {
+    checkLoadFactor(loadFactor, MIN_LOAD_FACTOR, MAX_LOAD_FACTOR);
+    return loadFactor;
+  }
+
+  /** Rehash from old buffers to new buffers. */
+  protected void rehash(int[] fromKeys, int[] fromValues) {
+    assert fromKeys.length == fromValues.length && checkPowerOfTwo(fromKeys.length - 1);
+
+    // Rehash all stored key/value pairs into the new buffers.
+    final int[] keys = this.keys;
+    final int[] values = this.values;
+    final int mask = this.mask;
+    int existing;
+
+    // Copy the zero element's slot, then rehash everything else.
+    int from = fromKeys.length - 1;
+    keys[keys.length - 1] = fromKeys[from];
+    values[values.length - 1] = fromValues[from];
+    while (--from >= 0) {
+      if (!((existing = fromKeys[from]) == 0)) {
+        int slot = hashKey(existing) & mask;
+        while (!((keys[slot]) == 0)) {
+          slot = (slot + 1) & mask;
+        }
+        keys[slot] = existing;
+        values[slot] = fromValues[from];
+      }
+    }
+  }
+
+  /**
+   * Allocate new internal buffers. This method attempts to allocate and assign internal buffers
+   * atomically (either allocations succeed or not).
+   */
+  protected void allocateBuffers(int arraySize) {
+    assert Integer.bitCount(arraySize) == 1;
+
+    // Ensure no change is done if we hit an OOM.
+    int[] prevKeys = this.keys;
+    int[] prevValues = this.values;
+    try {
+      int emptyElementSlot = 1;
+      this.keys = (new int[arraySize + emptyElementSlot]);
+      this.values = (new int[arraySize + emptyElementSlot]);
+    } catch (OutOfMemoryError e) {
+      this.keys = prevKeys;
+      this.values = prevValues;
+      throw new BufferAllocationException(
+          "Not enough memory to allocate buffers for rehashing: %,d -> %,d",
+          e, this.mask + 1, arraySize);
+    }
+
+    this.resizeAt = expandAtCount(arraySize, loadFactor);
+    this.mask = arraySize - 1;
+  }
+
+  /**
+   * This method is invoked when there is a new key/ value pair to be inserted into the buffers but
+   * there is not enough empty slots to do so.
+   *
+   * <p>New buffers are allocated. If this succeeds, we know we can proceed with rehashing so we
+   * assign the pending element to the previous buffer (possibly violating the invariant of having
+   * at least one empty slot) and rehash all keys, substituting new buffers at the end.
+   */
+  protected void allocateThenInsertThenRehash(int slot, int pendingKey, int pendingValue) {
+    assert assigned == resizeAt && ((keys[slot]) == 0) && !((pendingKey) == 0);
+
+    // Try to allocate new buffers first. If we OOM, we leave in a consistent state.
+    final int[] prevKeys = this.keys;
+    final int[] prevValues = this.values;
+    allocateBuffers(nextBufferSize(mask + 1, size(), loadFactor));
+    assert this.keys.length > prevKeys.length;
+
+    // We have succeeded at allocating new data so insert the pending key/value at
+    // the free slot in the old arrays before rehashing.
+    prevKeys[slot] = pendingKey;
+    prevValues[slot] = pendingValue;
+
+    // Rehash old keys, including the pending key.
+    rehash(prevKeys, prevValues);
+  }
+
+  static int nextBufferSize(int arraySize, int elements, double loadFactor) {
+    assert checkPowerOfTwo(arraySize);
+    if (arraySize == MAX_HASH_ARRAY_LENGTH) {
+      throw new BufferAllocationException(
+          "Maximum array size exceeded for this load factor (elements: %d, load factor: %f)",
+          elements, loadFactor);
+    }
+
+    return arraySize << 1;
+  }
+
+  static int expandAtCount(int arraySize, double loadFactor) {
+    assert checkPowerOfTwo(arraySize);
+    // Take care of hash container invariant (there has to be at least one empty slot to ensure
+    // the lookup loop finds either the element or an empty slot).
+    return Math.min(arraySize - 1, (int) Math.ceil(arraySize * loadFactor));
+  }
+
+  static boolean checkPowerOfTwo(int arraySize) {
+    // These are internals, we can just assert without retrying.
+    assert arraySize > 1;
+    assert nextHighestPowerOfTwo(arraySize) == arraySize;
+    return true;
+  }
+
+  static int minBufferSize(int elements, double loadFactor) {
+    if (elements < 0) {
+      throw new IllegalArgumentException("Number of elements must be >= 0: " + elements);
+    }
+
+    long length = (long) Math.ceil(elements / loadFactor);
+    if (length == elements) {
+      length++;
+    }
+    length = Math.max(MIN_HASH_ARRAY_LENGTH, nextHighestPowerOfTwo(length));
+
+    if (length > MAX_HASH_ARRAY_LENGTH) {
+      throw new BufferAllocationException(
+          "Maximum array size exceeded for this load factor (elements: %d, load factor: %f)",
+          elements, loadFactor);
+    }
+
+    return (int) length;
+  }
+
+  static void checkLoadFactor(
+      double loadFactor, double minAllowedInclusive, double maxAllowedInclusive) {
+    if (loadFactor < minAllowedInclusive || loadFactor > maxAllowedInclusive) {
+      throw new BufferAllocationException(
+          "The load factor should be in range [%.2f, %.2f]: %f",
+          minAllowedInclusive, maxAllowedInclusive, loadFactor);
+    }
+  }
+
+  static int iterationIncrement(int seed) {
+    return 29 + ((seed & 7) << 1); // Small odd integer.
+  }
+
+  /**
+   * Shift all the slot-conflicting keys and values allocated to (and including) <code>slot</code>.
+   */
+  protected void shiftConflictingKeys(int gapSlot) {
+    final int[] keys = this.keys;
+    final int[] values = this.values;
+    final int mask = this.mask;
+
+    // Perform shifts of conflicting keys to fill in the gap.
+    int distance = 0;
+    while (true) {
+      final int slot = (gapSlot + (++distance)) & mask;
+      final int existing = keys[slot];
+      if (((existing) == 0)) {
+        break;
+      }
+
+      final int idealSlot = hashKey(existing);
+      final int shift = (slot - idealSlot) & mask;
+      if (shift >= distance) {
+        // Entry at this position was originally at or before the gap slot.
+        // Move the conflict-shifted entry to the gap's position and repeat the procedure
+        // for any entries to the right of the current position, treating it
+        // as the new gap.
+        keys[gapSlot] = existing;
+        values[gapSlot] = values[slot];
+        gapSlot = slot;
+        distance = 0;
+      }
+    }
+
+    // Mark the last found gap slot without a conflict as empty.
+    keys[gapSlot] = 0;
+    values[gapSlot] = 0;
+    assigned--;
+  }
+
+  /** Forked from HPPC, holding int index,key and value */
+  public final class IntIntCursor {
+    /**
+     * The current key and value's index in the container this cursor belongs to. The meaning of
+     * this index is defined by the container (usually it will be an index in the underlying storage
+     * buffer).
+     */
+    public int index;
+
+    /** The current key. */
+    public int key;
+
+    /** The current value. */
+    public int value;
+
+    @Override
+    public String toString() {
+      return "[cursor, index: " + index + ", key: " + key + ", value: " + value + "]";
+    }
+  }
+
+  /** Forked from HPPC, holding int index and int value */
+  public final class IntCursor {
+    /**
+     * The current value's index in the container this cursor belongs to. The meaning of this index
+     * is defined by the container (usually it will be an index in the underlying storage buffer).
+     */
+    public int index;
+
+    /** The current value. */
+    public int value;
+
+    @Override
+    public String toString() {
+      return "[cursor, index: " + index + ", value: " + value + "]";
+    }
+  }
+
+  /** BufferAllocationException forked from HPPC */
+  @SuppressWarnings("serial")
+  public static class BufferAllocationException extends RuntimeException {
+    public BufferAllocationException(String message) {
+      super(message);
+    }
+
+    public BufferAllocationException(String message, Object... args) {
+      this(message, null, args);
+    }
+
+    public BufferAllocationException(String message, Throwable t, Object... args) {
+      super(formatMessage(message, t, args), t);
+    }
+
+    private static String formatMessage(String message, Throwable t, Object... args) {
+      try {
+        return String.format(Locale.ROOT, message, args);
+      } catch (IllegalFormatException e) {
+        BufferAllocationException substitute =
+            new BufferAllocationException(message + " [ILLEGAL FORMAT, ARGS SUPPRESSED]");
+        if (t != null) {
+          substitute.addSuppressed(t);
+        }
+        substitute.addSuppressed(e);
+        throw substitute;
+      }
+    }
+  }
+}
diff --git a/lucene/core/src/java/org/apache/lucene/util/hppc/package-info.java b/lucene/core/src/java/org/apache/lucene/util/hppc/package-info.java
new file mode 100644
index 0000000..6f395a6
--- /dev/null
+++ b/lucene/core/src/java/org/apache/lucene/util/hppc/package-info.java
@@ -0,0 +1,19 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+/** package holding hppc related classes. */
+package org.apache.lucene.util.hppc;
diff --git a/lucene/core/src/test/org/apache/lucene/util/automaton/TestIntSet.java b/lucene/core/src/test/org/apache/lucene/util/automaton/TestIntSet.java
new file mode 100644
index 0000000..006830f
--- /dev/null
+++ b/lucene/core/src/test/org/apache/lucene/util/automaton/TestIntSet.java
@@ -0,0 +1,104 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.lucene.util.automaton;
+
+import org.apache.lucene.util.LuceneTestCase;
+import org.junit.Test;
+
+public class TestIntSet extends LuceneTestCase {
+    @Test
+    public void testFreezeEqualitySmallSet() {
+        testFreezeEquality(10);
+    }
+
+    @Test
+    public void testFreezeEqualityLargeSet() {
+        testFreezeEquality(100);
+    }
+
+    private void testFreezeEquality(int size) {
+        StateSet stateSet = new StateSet(0);
+
+        for (int i = 0; i < size; i++) {
+            // Some duplicates is nice but not critical
+            stateSet.incr(random().nextInt(i + 1));
+        }
+
+
+        IntSet frozen0 = stateSet.freeze(0);
+
+        assertEquals("Frozen set not equal to origin sorted set.", stateSet, frozen0);
+        assertEquals("Symmetry: Sorted set not equal to frozen set.", frozen0, stateSet);
+
+        IntSet frozen1 = stateSet.freeze(random().nextInt());
+        assertEquals("Sorted set modified while freezing?", stateSet, frozen1);
+        assertEquals("Frozen sets were not equal", frozen0, frozen1);
+    }
+
+    @Test
+    public void testMapCutover() {
+        StateSet set = new StateSet(10);
+        for (int i = 0; i < 35; i++) {
+            // No duplicates so there are enough elements to trigger impl cutover
+            set.incr(i);
+        }
+
+
+        assertTrue(set.size() > 32);
+
+        for (int i = 0; i < 35; i++) {
+            // This is pretty much the worst case, perf wise
+            set.decr(i);
+        }
+
+
+        assertTrue(set.size() == 0);
+    }
+
+    @Test
+    public void testModify() {
+        StateSet set = new StateSet(2);
+        set.incr(1);
+        set.incr(2);
+
+
+        FrozenIntSet set2 = set.freeze(0);
+        assertEquals(set, set2);
+
+        set.incr(1);
+
+        assertEquals(set, set2);
+
+        set.decr(1);
+
+        assertEquals(set, set2);
+
+        set.decr(1);
+        assertNotEquals(set, set2);
+        }
+
+  @Test
+  public void testHashCode() {
+    StateSet set = new StateSet(1000);
+    StateSet set2 = new StateSet(100);
+    for (int i = 0; i < 100; i++) {
+      set.incr(i);
+      set2.incr(99 - i);
+    }
+    assertEquals(set.hashCode(), set2.hashCode());
+    }
+}
\ No newline at end of file
diff --git a/lucene/core/src/test/org/apache/lucene/util/hppc/TestIntIntHashMap.java b/lucene/core/src/test/org/apache/lucene/util/hppc/TestIntIntHashMap.java
new file mode 100644
index 0000000..62efe25
--- /dev/null
+++ b/lucene/core/src/test/org/apache/lucene/util/hppc/TestIntIntHashMap.java
@@ -0,0 +1,633 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.lucene.util.hppc;
+
+import java.util.Arrays;
+import java.util.HashSet;
+import java.util.Random;
+import java.util.concurrent.atomic.AtomicInteger;
+import org.apache.lucene.util.LuceneTestCase;
+import org.junit.After;
+import org.junit.Test;
+
+/**
+ * Tests for {@link IntIntHashMap}.
+ *
+ * <p>Mostly forked and trimmed from com.carrotsearch.hppc.IntIntHashMapTest
+ *
+ * <p>github: https://github.com/carrotsearch/hppc release: 0.9.0
+ */
+public class TestIntIntHashMap extends LuceneTestCase {
+  /* Ready to use key values. */
+
+  protected int keyE = 0;
+  protected int key0 = cast(0), k0 = key0;
+  protected int key1 = cast(1), k1 = key1;
+  protected int key2 = cast(2), k2 = key2;
+  protected int key3 = cast(3), k3 = key3;
+  protected int key4 = cast(4), k4 = key4;
+  protected int key5 = cast(5), k5 = key5;
+  protected int key6 = cast(6), k6 = key6;
+  protected int key7 = cast(7), k7 = key7;
+  protected int key8 = cast(8), k8 = key8;
+  protected int key9 = cast(9), k9 = key9;
+
+  /** Convert to target type from an integer used to test stuff. */
+  public int cast(Integer v) {
+    return v.intValue();
+  }
+
+  public int[] asArray(int... ints) {
+    int[] values = (new int[ints.length]);
+    for (int i = 0; i < ints.length; i++) values[i] = ints[i];
+    return values;
+  }
+
+  /** Create a new array of a given type and copy the arguments to this array. */
+  /*  */
+  public final int[] newArray(int... elements) {
+    return newArray0(elements);
+  }
+
+  /*  */
+  private final int[] newArray0(int... elements) {
+    return elements;
+  }
+
+  public int[] newArray(int v0) {
+    return this.newArray0(v0);
+  }
+
+  public int[] newArray(int v0, int v1) {
+    return this.newArray0(v0, v1);
+  }
+
+  public int[] newArray(int v0, int v1, int v2) {
+    return this.newArray0(v0, v1, v2);
+  }
+
+  public int[] newArray(int v0, int v1, int v2, int v3) {
+    return this.newArray0(v0, v1, v2, v3);
+  }
+
+  public int[] newArray(int v0, int v1, int v2, int v3, int v4, int v5, int v6) {
+    return this.newArray0(v0, v1, v2, v3, v4, v5, v6);
+  }
+
+  public int[] newArray(int v0, int v1, int v2, int v3, int v4, int v5) {
+    return this.newArray0(v0, v1, v2, v3, v4, v5);
+  }
+
+  public int[] newArray(int v0, int v1, int v2, int v3, int v4) {
+    return this.newArray0(v0, v1, v2, v3, v4);
+  }
+
+  public int[] newArray(int v0, int v1, int v2, int v3, int v4, int v5, int v6, int v7) {
+    return this.newArray0(v0, v1, v2, v3, v4, v5, v6, v7);
+  }
+
+  public static int randomIntBetween(int min, int max) {
+    return min + random().nextInt(max + 1 - min);
+  }
+
+  /** Check if the array's content is identical to a given sequence of elements. */
+  public static void assertSortedListEquals(int[] array, int... elements) {
+    assertEquals(elements.length, array.length);
+    Arrays.sort(array);
+    Arrays.sort(elements);
+    assertArrayEquals(elements, array);
+  }
+
+  protected int value0 = vcast(0);
+  protected int value1 = vcast(1);
+  protected int value2 = vcast(2);
+  protected int value3 = vcast(3);
+  protected int value4 = vcast(4);
+
+  /** Per-test fresh initialized instance. */
+  public IntIntHashMap map = newInstance();
+
+  protected IntIntHashMap newInstance() {
+    return new IntIntHashMap();
+  }
+
+  @After
+  public void checkEmptySlotsUninitialized() {
+    if (map != null) {
+      int occupied = 0;
+      for (int i = 0; i <= map.mask; i++) {
+        if (((map.keys[i]) == 0)) {
+
+        } else {
+          occupied++;
+        }
+      }
+      assertEquals(occupied, map.assigned);
+
+      if (!map.hasEmptyKey) {}
+    }
+  }
+
+  /** Convert to target type from an integer used to test stuff. */
+  protected int vcast(int value) {
+    return value;
+  }
+
+  /** Create a new array of a given type and copy the arguments to this array. */
+  /*  */
+  protected final int[] newvArray(int... elements) {
+    return elements;
+  }
+
+  private void assertSameMap(final IntIntHashMap c1, final IntIntHashMap c2) {
+    assertEquals(c1.size(), c2.size());
+
+    for (IntIntHashMap.IntIntCursor entry : c1) {
+      assertTrue(c2.containsKey(entry.key));
+      assertEquals(entry.value, c2.get(entry.key));
+    }
+  }
+
+  /* */
+  @Test
+  public void testEnsureCapacity() {
+    final AtomicInteger expands = new AtomicInteger();
+    IntIntHashMap map =
+        new IntIntHashMap(0) {
+          @Override
+          protected void allocateBuffers(int arraySize) {
+            super.allocateBuffers(arraySize);
+            expands.incrementAndGet();
+          }
+        };
+
+    // Add some elements.
+    final int max = rarely() ? 0 : randomIntBetween(0, 250);
+    for (int i = 0; i < max; i++) {
+      map.put(cast(i), value0);
+    }
+
+    final int additions = randomIntBetween(max, max + 5000);
+    map.ensureCapacity(additions + map.size());
+    final int before = expands.get();
+    for (int i = 0; i < additions; i++) {
+      map.put(cast(i), value0);
+    }
+    assertEquals(before, expands.get());
+  }
+
+  @Test
+  public void testCursorIndexIsValid() {
+    map.put(keyE, value1);
+    map.put(key1, value2);
+    map.put(key2, value3);
+
+    for (IntIntHashMap.IntIntCursor c : map) {
+      assertTrue(map.indexExists(c.index));
+      assertEquals(c.value, map.indexGet(c.index));
+    }
+  }
+
+  @Test
+  public void testIndexMethods() {
+    map.put(keyE, value1);
+    map.put(key1, value2);
+
+    assertTrue(map.indexOf(keyE) >= 0);
+    assertTrue(map.indexOf(key1) >= 0);
+    assertTrue(map.indexOf(key2) < 0);
+
+    assertTrue(map.indexExists(map.indexOf(keyE)));
+    assertTrue(map.indexExists(map.indexOf(key1)));
+    assertFalse(map.indexExists(map.indexOf(key2)));
+
+    assertEquals(value1, map.indexGet(map.indexOf(keyE)));
+    assertEquals(value2, map.indexGet(map.indexOf(key1)));
+
+    expectThrows(
+        AssertionError.class,
+        () -> {
+          map.indexGet(map.indexOf(key2));
+          fail();
+        });
+
+    assertEquals(value1, map.indexReplace(map.indexOf(keyE), value3));
+    assertEquals(value2, map.indexReplace(map.indexOf(key1), value4));
+    assertEquals(value3, map.indexGet(map.indexOf(keyE)));
+    assertEquals(value4, map.indexGet(map.indexOf(key1)));
+
+    map.indexInsert(map.indexOf(key2), key2, value1);
+    assertEquals(value1, map.indexGet(map.indexOf(key2)));
+    assertEquals(3, map.size());
+
+    assertEquals(value3, map.indexRemove(map.indexOf(keyE)));
+    assertEquals(2, map.size());
+    assertEquals(value1, map.indexRemove(map.indexOf(key2)));
+    assertEquals(1, map.size());
+    assertTrue(map.indexOf(keyE) < 0);
+    assertTrue(map.indexOf(key1) >= 0);
+    assertTrue(map.indexOf(key2) < 0);
+  }
+
+  /* */
+  @Test
+  public void testCloningConstructor() {
+    map.put(key1, value1);
+    map.put(key2, value2);
+    map.put(key3, value3);
+
+    assertSameMap(map, new IntIntHashMap(map));
+  }
+
+  /* */
+  @Test
+  public void testFromArrays() {
+    map.put(key1, value1);
+    map.put(key2, value2);
+    map.put(key3, value3);
+
+    IntIntHashMap map2 =
+        IntIntHashMap.from(newArray(key1, key2, key3), newvArray(value1, value2, value3));
+
+    assertSameMap(map, map2);
+  }
+
+  @Test
+  public void testGetOrDefault() {
+    map.put(key2, value2);
+    assertTrue(map.containsKey(key2));
+
+    map.put(key1, value1);
+    assertEquals(value1, map.getOrDefault(key1, value3));
+    assertEquals(value3, map.getOrDefault(key3, value3));
+    map.remove(key1);
+    assertEquals(value3, map.getOrDefault(key1, value3));
+  }
+
+  /* */
+  @Test
+  public void testPut() {
+    map.put(key1, value1);
+
+    assertTrue(map.containsKey(key1));
+    assertEquals(value1, map.get(key1));
+  }
+
+  /* */
+  @Test
+  public void testPutOverExistingKey() {
+    map.put(key1, value1);
+    assertEquals(value1, map.put(key1, value3));
+    assertEquals(value3, map.get(key1));
+  }
+
+  /* */
+  @Test
+  public void testPutWithExpansions() {
+    final int COUNT = 10000;
+    final Random rnd = new Random(random().nextLong());
+    final HashSet<Object> values = new HashSet<Object>();
+
+    for (int i = 0; i < COUNT; i++) {
+      final int v = rnd.nextInt();
+      final boolean hadKey = values.contains(cast(v));
+      values.add(cast(v));
+
+      assertEquals(hadKey, map.containsKey(cast(v)));
+      map.put(cast(v), vcast(v));
+      assertEquals(values.size(), map.size());
+    }
+    assertEquals(values.size(), map.size());
+  }
+
+  /* */
+  @Test
+  public void testPutAll() {
+    map.put(key1, value1);
+    map.put(key2, value1);
+
+    IntIntHashMap map2 = newInstance();
+
+    map2.put(key2, value2);
+    map2.put(keyE, value1);
+
+    // One new key (keyE).
+    assertEquals(1, map.putAll(map2));
+
+    // Assert the value under key2 has been replaced.
+    assertEquals(value2, map.get(key2));
+
+    // And key3 has been added.
+    assertEquals(value1, map.get(keyE));
+    assertEquals(3, map.size());
+  }
+
+  /* */
+  @Test
+  public void testPutIfAbsent() {
+    assertTrue(map.putIfAbsent(key1, value1));
+    assertFalse(map.putIfAbsent(key1, value2));
+    assertEquals(value1, map.get(key1));
+  }
+
+  @Test
+  public void testPutOrAdd() {
+    assertEquals(value1, map.putOrAdd(key1, value1, value2));
+    assertEquals(value3, map.putOrAdd(key1, value1, value2));
+  }
+
+  @Test
+  public void testAddTo() {
+    assertEquals(value1, map.addTo(key1, value1));
+    assertEquals(value3, map.addTo(key1, value2));
+  }
+
+  /* */
+  @Test
+  public void testRemove() {
+    map.put(key1, value1);
+    assertEquals(value1, map.remove(key1));
+    assertEquals(0, map.remove(key1));
+    assertEquals(0, map.size());
+
+    // These are internals, but perhaps worth asserting too.
+    assertEquals(0, map.assigned);
+  }
+
+  /* */
+  @Test
+  public void testEmptyKey() {
+    final int empty = 0;
+
+    map.put(empty, value1);
+    assertEquals(1, map.size());
+    assertEquals(false, map.isEmpty());
+    assertEquals(value1, map.get(empty));
+    assertEquals(value1, map.getOrDefault(empty, value2));
+    assertEquals(true, map.iterator().hasNext());
+    assertEquals(empty, map.iterator().next().key);
+    assertEquals(value1, map.iterator().next().value);
+
+    map.remove(empty);
+    assertEquals(0, map.get(empty));
+  }
+
+  /* */
+  @Test
+  public void testMapKeySet() {
+    map.put(key1, value3);
+    map.put(key2, value2);
+    map.put(key3, value1);
+
+    assertSortedListEquals(map.keys().toArray(), key1, key2, key3);
+  }
+
+  /* */
+  @Test
+  public void testMapKeySetIterator() {
+    map.put(key1, value3);
+    map.put(key2, value2);
+    map.put(key3, value1);
+
+    int counted = 0;
+    for (IntIntHashMap.IntCursor c : map.keys()) {
+      assertEquals(map.keys[c.index], c.value);
+      counted++;
+    }
+    assertEquals(counted, map.size());
+  }
+
+  /* */
+  @Test
+  public void testClear() {
+    map.put(key1, value1);
+    map.put(key2, value1);
+    map.clear();
+    assertEquals(0, map.size());
+
+    // These are internals, but perhaps worth asserting too.
+    assertEquals(0, map.assigned);
+
+    // Check if the map behaves properly upon subsequent use.
+    testPutWithExpansions();
+  }
+
+  /* */
+  @Test
+  public void testRelease() {
+    map.put(key1, value1);
+    map.put(key2, value1);
+    map.release();
+    assertEquals(0, map.size());
+
+    // These are internals, but perhaps worth asserting too.
+    assertEquals(0, map.assigned);
+
+    // Check if the map behaves properly upon subsequent use.
+    testPutWithExpansions();
+  }
+
+  /* */
+  @Test
+  public void testIterable() {
+    map.put(key1, value1);
+    map.put(key2, value2);
+    map.put(key3, value3);
+    map.remove(key2);
+
+    int count = 0;
+    for (IntIntHashMap.IntIntCursor cursor : map) {
+      count++;
+      assertTrue(map.containsKey(cursor.key));
+      assertEquals(cursor.value, map.get(cursor.key));
+
+      assertEquals(cursor.value, map.values[cursor.index]);
+      assertEquals(cursor.key, map.keys[cursor.index]);
+    }
+    assertEquals(count, map.size());
+
+    map.clear();
+    assertFalse(map.iterator().hasNext());
+  }
+
+  /* */
+  @Test
+  public void testBug_HPPC73_FullCapacityGet() {
+    final AtomicInteger reallocations = new AtomicInteger();
+    final int elements = 0x7F;
+    map =
+        new IntIntHashMap(elements, 1f) {
+          @Override
+          protected double verifyLoadFactor(double loadFactor) {
+            // Skip load factor sanity range checking.
+            return loadFactor;
+          }
+
+          @Override
+          protected void allocateBuffers(int arraySize) {
+            super.allocateBuffers(arraySize);
+            reallocations.incrementAndGet();
+          }
+        };
+
+    int reallocationsBefore = reallocations.get();
+    assertEquals(reallocationsBefore, 1);
+    for (int i = 1; i <= elements; i++) {
+      map.put(cast(i), value1);
+    }
+
+    // Non-existent key.
+    int outOfSet = cast(elements + 1);
+    map.remove(outOfSet);
+    assertFalse(map.containsKey(outOfSet));
+    assertEquals(reallocationsBefore, reallocations.get());
+
+    // Should not expand because we're replacing an existing element.
+    map.put(k1, value2);
+    assertEquals(reallocationsBefore, reallocations.get());
+
+    // Remove from a full map.
+    map.remove(k1);
+    assertEquals(reallocationsBefore, reallocations.get());
+    map.put(k1, value2);
+
+    // Check expand on "last slot of a full map" condition.
+    map.put(outOfSet, value1);
+    assertEquals(reallocationsBefore + 1, reallocations.get());
+  }
+
+  @Test
+  public void testHashCodeEquals() {
+    IntIntHashMap l0 = newInstance();
+    assertEquals(0, l0.hashCode());
+    assertEquals(l0, newInstance());
+
+    IntIntHashMap l1 =
+        IntIntHashMap.from(newArray(key1, key2, key3), newvArray(value1, value2, value3));
+
+    IntIntHashMap l2 =
+        IntIntHashMap.from(newArray(key2, key1, key3), newvArray(value2, value1, value3));
+
+    IntIntHashMap l3 = IntIntHashMap.from(newArray(key1, key2), newvArray(value2, value1));
+
+    assertEquals(l1.hashCode(), l2.hashCode());
+    assertEquals(l1, l2);
+
+    assertFalse(l1.equals(l3));
+    assertFalse(l2.equals(l3));
+  }
+
+  @Test
+  public void testBug_HPPC37() {
+    IntIntHashMap l1 = IntIntHashMap.from(newArray(key1), newvArray(value1));
+
+    IntIntHashMap l2 = IntIntHashMap.from(newArray(key2), newvArray(value1));
+
+    assertFalse(l1.equals(l2));
+    assertFalse(l2.equals(l1));
+  }
+
+  /*
+   *
+   */
+  @Test
+  public void testClone() {
+    this.map.put(key1, value1);
+    this.map.put(key2, value2);
+    this.map.put(key3, value3);
+
+    IntIntHashMap cloned = map.clone();
+    cloned.remove(key1);
+
+    assertSortedListEquals(map.keys().toArray(), key1, key2, key3);
+    assertSortedListEquals(cloned.keys().toArray(), key2, key3);
+  }
+
+  /* */
+  @Test
+  public void testMapValues() {
+    map.put(key1, value3);
+    map.put(key2, value2);
+    map.put(key3, value1);
+    assertSortedListEquals(map.values().toArray(), value1, value2, value3);
+
+    map.clear();
+    map.put(key1, value1);
+    map.put(key2, value2);
+    map.put(key3, value2);
+    assertSortedListEquals(map.values().toArray(), value1, value2, value2);
+  }
+
+  /* */
+  @Test
+  public void testMapValuesIterator() {
+    map.put(key1, value3);
+    map.put(key2, value2);
+    map.put(key3, value1);
+
+    int counted = 0;
+    for (IntIntHashMap.IntCursor c : map.values()) {
+      assertEquals(map.values[c.index], c.value);
+      counted++;
+    }
+    assertEquals(counted, map.size());
+  }
+
+  /* */
+  @Test
+  public void testEqualsSameClass() {
+    IntIntHashMap l1 = newInstance();
+    l1.put(k1, value0);
+    l1.put(k2, value1);
+    l1.put(k3, value2);
+
+    IntIntHashMap l2 = new IntIntHashMap(l1);
+    l2.putAll(l1);
+
+    IntIntHashMap l3 = new IntIntHashMap(l2);
+    l3.putAll(l2);
+    l3.put(k4, value0);
+
+    assertEquals(l2, l1);
+    assertEquals(l2.hashCode(), l1.hashCode());
+    assertNotEquals(l1, l3);
+  }
+
+  /* */
+  @Test
+  public void testEqualsSubClass() {
+    class Sub extends IntIntHashMap {}
+    ;
+
+    IntIntHashMap l1 = newInstance();
+    l1.put(k1, value0);
+    l1.put(k2, value1);
+    l1.put(k3, value2);
+
+    IntIntHashMap l2 = new Sub();
+    l2.putAll(l1);
+    l2.put(k4, value3);
+
+    IntIntHashMap l3 = new Sub();
+    l3.putAll(l2);
+
+    assertNotEquals(l1, l2);
+    assertEquals(l3.hashCode(), l2.hashCode());
+    assertEquals(l3, l2);
+  }
+}