You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@baremaps.apache.org by "github-code-scanning[bot] (via GitHub)" <gi...@apache.org> on 2023/07/12 13:41:48 UTC

[GitHub] [incubator-baremaps] github-code-scanning[bot] commented on a diff in pull request #715: Add new data structures

github-code-scanning[bot] commented on code in PR #715:
URL: https://github.com/apache/incubator-baremaps/pull/715#discussion_r1261199795


##########
baremaps-core/src/main/java/org/apache/baremaps/database/collection/DataMap.java:
##########
@@ -12,124 +12,39 @@
 
 package org.apache.baremaps.database.collection;
 
-
-
 import com.google.common.collect.Streams;
-import java.util.*;
+import java.util.List;
+import java.util.Map;
 
 /**
- * An abstract map of data elements that can hold a large number of elements.
+ * A map of longs to values that can be stored in memory or on disk and has a size that can exceed
+ * the maximum value of an integer.
  *
- * @param <E> The type of the elements.
+ * @param <E>
  */
-public abstract class DataMap<E> implements Map<Long, E> {
-
-  /** {@inheritDoc} */
-  @Override
-  public void putAll(Map<? extends Long, ? extends E> m) {
-    m.forEach(this::put);
-  }
-
-  /**
-   * Returns the value associated with the specified key or null if the key is not present.
-   *
-   * @param keys the keys
-   * @return the values
-   */
-  public List<E> getAll(List<Long> keys) {
-    return Streams.stream(keys).map(this::get).toList();
-  }
-
-  /** {@inheritDoc} */
-  @Override
-  public boolean isEmpty() {
-    return size() == 0;
-  }
+public interface DataMap<E> extends Map<Long, E> {
 
   /**
    * Returns the size of the map as a long.
    *
    * @return the size of the map
    */
-  public abstract long sizeAsLong();
-
-  /** {@inheritDoc} */
-  public int size() {
-    return (int) Math.min(sizeAsLong(), Integer.MAX_VALUE);
-  }
-
-  /**
-   * Returns an iterator over the keys of the map.
-   *
-   * @return an iterator
-   */
-  protected abstract Iterator<Long> keyIterator();
+  long sizeAsLong();
 
   /** {@inheritDoc} */
   @Override
-  public Set<Long> keySet() {
-    return new KeySet();
-  }
-
-  private class KeySet extends AbstractSet<Long> {
-    @Override
-    public Iterator<Long> iterator() {
-      return keyIterator();
-    }
-
-    @Override
-    public int size() {
-      return DataMap.this.size();
-    }
-  }
-
-  /**
-   * Returns an iterator over the values of the map.
-   *
-   * @return an iterator
-   */
-  protected abstract Iterator<E> valueIterator();
-
-  /** {@inheritDoc} */
-  @Override
-  public Collection<E> values() {
-    return new ValueCollection();
-  }
-
-  private class ValueCollection extends AbstractCollection<E> {
-    @Override
-    public Iterator<E> iterator() {
-      return valueIterator();
-    }
-
-    @Override
-    public int size() {
-      return DataMap.this.size();
-    }
+  default int size() {
+    return (int) Math.min(sizeAsLong(), Integer.MAX_VALUE);
   }
 
   /**
-   * Returns an iterator over the entries of the map.
+   * Returns the value associated with the specified key or null if the key is not present.
    *
-   * @return an iterator
+   * @param keys the keys
+   * @return the values
    */
-  protected abstract Iterator<Entry<Long, E>> entryIterator();
-
-  /** {@inheritDoc} */
-  @Override
-  public Set<Entry<Long, E>> entrySet() {
-    return new EntrySet();
+  default List<E> getAll(List<Long> keys) {
+    return Streams.stream(keys).map(this::get).toList();

Review Comment:
   ## Deprecated method or constructor invocation
   
   Invoking [Streams.stream](1) should be avoided because it has been deprecated.
   
   [Show more details](https://github.com/apache/incubator-baremaps/security/code-scanning/769)



##########
baremaps-core/src/main/java/org/apache/baremaps/database/collection/AppendOnlyBuffer.java:
##########
@@ -119,7 +120,7 @@
   }
 
   /** {@inheritDoc} */
-  public long sizeAsLong() {
+  public long size64() {

Review Comment:
   ## Missing Override annotation
   
   This method overrides [DataCollection<E>.size64](1); it is advisable to add an Override annotation.
   
   [Show more details](https://github.com/apache/incubator-baremaps/security/code-scanning/774)



##########
baremaps-core/src/main/java/org/apache/baremaps/database/collection/Long2LongOpenHashDataMap.java:
##########
@@ -0,0 +1,1553 @@
+/*
+ * Licensed 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.baremaps.database.collection;
+
+import static it.unimi.dsi.fastutil.HashCommon.*;
+
+import it.unimi.dsi.fastutil.Hash;
+import it.unimi.dsi.fastutil.HashCommon;
+import it.unimi.dsi.fastutil.longs.*;
+import it.unimi.dsi.fastutil.objects.*;
+import java.util.Map;
+import java.util.NoSuchElementException;
+import java.util.concurrent.atomic.AtomicLong;
+import java.util.function.Consumer;
+import java.util.function.LongConsumer;
+import java.util.function.Supplier;
+
+/**
+ * An open addressed hash map of long keys and long values derived from fastutil's
+ * {@link Long2LongOpenHashMap}. This implementation allows for the use of on-heap, off-heap, or
+ * memory mapped memory.
+ */
+public class Long2LongOpenHashDataMap extends AbstractLong2LongMap
+    implements DataMap<Long>, Hash {
+
+  protected Supplier<AbstractDataList<Long>> keySupplier;
+
+  protected Supplier<AbstractDataList<Long>> valueSupplier;
+
+  /**
+   * The array of keys.
+   */
+  protected AbstractDataList<Long> key;
+  /**
+   * The array of values.
+   */
+  protected AbstractDataList<Long> value;
+  /**
+   * The mask for wrapping a position counter.
+   */
+  protected long mask;
+  /**
+   * Whether this map contains the key zero.
+   */
+  protected boolean containsNullKey;
+  /**
+   * The current table size.
+   */
+  protected long n;
+  /**
+   * Threshold after which we rehash. It must be the table size times {@link #f}.
+   */
+  protected long maxFill;
+  /**
+   * We never resize below this threshold, which is the construction-time {#n}.
+   */
+  protected final long minN;
+  /**
+   * Number of entries in the set (including the key zero, if present).
+   */
+  protected AtomicLong size = new AtomicLong();
+  /**
+   * The acceptable load factor.
+   */
+  protected final float f;
+  /**
+   * Cached set of entries.
+   */
+  protected FastEntrySet entries;
+
+  /**
+   * Cached set of keys.
+   */
+  protected LongSet keys;
+  /**
+   * Cached collection of values.
+   */
+  protected LongCollection values;
+
+  /**
+   * Creates a new hash map.
+   *
+   * <p>
+   * The actual table size will be the least power of two greater than {@code expected}/{@code f}.
+   *
+   * @param expected the expected number of elements in the hash map.
+   * @param f the load factor.
+   */
+  public Long2LongOpenHashDataMap(
+      long expected,
+      float f,
+      Supplier<AbstractDataList<Long>> keySupplier,
+      Supplier<AbstractDataList<Long>> valueSupplier) {
+    if (f <= 0 || f >= 1) {
+      throw new IllegalArgumentException("Load factor must be greater than 0 and smaller than 1");
+    }
+    if (expected < 0) {
+      throw new IllegalArgumentException("The expected number of elements must be non-negative");
+    }
+    this.f = f;
+    this.minN = n = bigArraySize(expected, f);
+    this.mask = n - 1;
+    this.maxFill = maxFill(n, f);
+    this.keySupplier = keySupplier;
+    this.valueSupplier = valueSupplier;
+    this.key = keySupplier.get();
+    this.value = valueSupplier.get();
+  }
+
+  private long realSize() {
+    return containsNullKey ? size.get() - 1 : size.get();
+  }
+
+  private void ensureCapacity(final long capacity) {
+    final long needed = bigArraySize(capacity, f);
+    if (needed > n) {
+      rehash(needed);
+    }
+  }
+
+  private void tryCapacity(final long capacity) {
+    final long needed =
+        Math.min(1 << 30, Math.max(2, HashCommon.nextPowerOfTwo((long) Math.ceil(capacity / f))));
+    if (needed > n) {
+      rehash(needed);
+    }
+  }
+
+  private long removeEntry(final long pos) {
+    final long oldValue = value.get(pos);
+    size.decrementAndGet();
+    shiftKeys(pos);
+    if (n > minN && size.get() < maxFill / 4 && n > DEFAULT_INITIAL_SIZE) {
+      rehash(n / 2);
+    }
+    return oldValue;
+  }
+
+  private long removeNullEntry() {
+    containsNullKey = false;
+    final long oldValue = value.get(n);
+    size.decrementAndGet();
+    if (n > minN && size.get() < maxFill / 4 && n > DEFAULT_INITIAL_SIZE) {
+      rehash(n / 2);
+    }
+    return oldValue;
+  }
+
+  @Override
+  public void putAll(Map<? extends Long, ? extends Long> m) {
+    if (f <= .5) {
+      ensureCapacity(m.size());
+    } else {
+      tryCapacity(sizeAsLong() + m.size());
+    }
+    super.putAll(m);
+  }
+
+  private long find(final long k) {
+    if (((k) == 0)) {
+      return containsNullKey ? n : -(n + 1);
+    }
+    long curr;
+    long pos;
+    // The starting point.
+    if (((curr = key.get(pos = HashCommon.mix((k)) & mask)) == 0)) {
+      return -(pos + 1);
+    }
+    if (((k) == (curr))) {
+      return pos;
+    }
+    // There's always an unused entry.
+    while (true) {
+      if (((curr = key.get(pos = (pos + 1) & mask)) == 0)) {
+        return -(pos + 1);
+      }
+      if (((k) == (curr))) {
+        return pos;
+      }
+    }
+  }
+
+  private void insert(final long pos, final long k, final long v) {
+    if (pos == n) {
+      containsNullKey = true;
+    }
+    key.set(pos, k);
+    value.set(pos, v);
+    if (size.getAndIncrement() >= maxFill) {
+      rehash(bigArraySize(size.get() + 1, f));
+    }
+  }
+
+  @Override
+  public long put(final long k, final long v) {
+    final long pos = find(k);
+    if (pos < 0) {
+      insert(-pos - 1, k, v);
+      return defRetValue;
+    }
+    final long oldValue = value.get(pos);
+    value.set(pos, v);
+    return oldValue;
+  }
+
+  private long addToValue(final long pos, final long incr) {
+    final long oldValue = value.get(pos);
+    value.set(pos, oldValue + incr);
+    return oldValue;
+  }
+
+  /**
+   * Adds an increment to value currently associated with a key.
+   *
+   * <p>
+   * Note that this method respects the {@linkplain #defaultReturnValue() default return value}
+   * semantics: when called with a key that does not currently appears in the map, the key will be
+   * associated with the default return value plus the given increment.
+   *
+   * @param k the key.
+   * @param incr the increment.
+   * @return the old value, or the {@linkplain #defaultReturnValue() default return value} if no
+   *         value was present for the given key.
+   */
+  public long addTo(final long k, final long incr) {
+    long pos;
+    if (((k) == 0)) {
+      if (containsNullKey) {
+        return addToValue(n, incr);
+      }
+      pos = n;
+      containsNullKey = true;
+    } else {
+      long curr;
+      // The starting point.
+      if (!((curr = key.get(pos = HashCommon.mix((k)) & mask)) == 0)) {
+        if (((curr) == (k))) {
+          return addToValue(pos, incr);
+        }
+        while (!((curr = key.get(pos = (pos + 1) & mask)) == 0)) {
+          if (((curr) == (k))) {
+            return addToValue(pos, incr);
+          }
+        }
+      }
+    }
+    key.set(pos, k);
+    value.set(pos, defRetValue + incr);
+    if (size.incrementAndGet() >= maxFill)
+      rehash(bigArraySize(size.get() + 1, f));
+    return defRetValue;
+  }
+
+  /**
+   * Shifts left entries with the specified hash code, starting at the specified position, and
+   * empties the resulting free entry.
+   *
+   * @param pos a starting position.
+   */
+  protected final void shiftKeys(long pos) {
+    // Shift entries with the same hash.
+    long last, slot;
+    long curr;
+    for (;;) {
+      pos = ((last = pos) + 1) & mask;
+      for (;;) {
+        if (((curr = key.get(pos)) == 0)) {
+          key.set(last, 0L);
+          return;
+        }
+        slot = HashCommon.mix((curr)) & mask;
+        if (last <= pos ? last >= slot || slot > pos : last >= slot && slot > pos) {
+          break;
+        }
+        pos = (pos + 1) & mask;
+      }
+      key.set(last, curr);
+      value.set(last, value.get(pos));
+    }
+  }
+
+  @Override
+
+  public long remove(final long k) {
+    if (((k) == 0)) {
+      if (containsNullKey) {
+        return removeNullEntry();
+      }
+      return defRetValue;
+    }
+    long curr;
+    long pos;
+    // The starting point.
+    if (((curr = key.get(pos = HashCommon.mix((k)) & mask)) == 0)) {
+      return defRetValue;
+    }
+    if (((k) == (curr))) {
+      return removeEntry(pos);
+    }
+    while (true) {
+      if (((curr = key.get(pos = (pos + 1) & mask)) == 0)) {
+        return defRetValue;
+      }
+      if (((k) == (curr))) {
+        return removeEntry(pos);
+      }
+    }
+  }
+
+  @Override
+
+  public long get(final long k) {
+    if (((k) == 0)) {
+      return containsNullKey ? value.get(n) : defRetValue;
+    }
+    long curr;
+    long pos;
+    // The starting point.
+    if (((curr = key.get(pos = HashCommon.mix((k)) & mask)) == 0)) {
+      return defRetValue;
+    }
+    if (((k) == (curr))) {
+      return value.get(pos);
+    }
+    // There's always an unused entry.
+    while (true) {
+      if (((curr = key.get(pos = (pos + 1) & mask)) == 0)) {
+        return defRetValue;
+      }
+      if (((k) == (curr))) {
+        return value.get(pos);
+      }
+    }
+  }
+
+  @Override
+
+  public boolean containsKey(final long k) {
+    if (((k) == 0)) {
+      return containsNullKey;
+    }
+    long curr;
+    long pos;
+    // The starting point.
+    if (((curr = key.get(pos = HashCommon.mix((k)) & mask)) == 0)) {
+      return false;
+    }
+    if (((k) == (curr))) {
+      return true;
+    }
+    // There's always an unused entry.
+    while (true) {
+      if (((curr = key.get(pos = (pos + 1) & mask)) == 0)) {
+        return false;
+      }
+      if (((k) == (curr))) {
+        return true;
+      }
+    }
+  }
+
+  @Override
+  public boolean containsValue(final long v) {
+    if (containsNullKey && ((value.get(n)) == (v))) {
+      return true;
+    }
+    for (long i = n; i-- != 0;) {
+      if (!((key.get(i)) == 0) && ((value.get(i)) == (v))) {
+        return true;
+      }
+    }
+    return false;
+  }
+
+  /**
+   * {@inheritDoc}
+   */
+  @Override
+
+  public long getOrDefault(final long k, final long defaultValue) {
+    if (((k) == 0)) {
+      return containsNullKey ? value.get(n) : defaultValue;
+    }
+    long curr;
+    long pos;
+    // The starting point.
+    if (((curr = key.get(pos = HashCommon.mix((k)) & mask)) == 0)) {
+      return defaultValue;
+    }
+    if (((k) == (curr))) {
+      return value.get(pos);
+    }
+    // There's always an unused entry.
+    while (true) {
+      if (((curr = key.get(pos = (pos + 1) & mask)) == 0)) {
+        return defaultValue;
+      }
+      if (((k) == (curr))) {
+        return value.get(pos);
+      }
+    }
+  }
+
+  /**
+   * {@inheritDoc}
+   */
+  @Override
+  public long putIfAbsent(final long k, final long v) {
+    final long pos = find(k);
+    if (pos >= 0) {
+      return value.get(pos);
+    }
+    insert(-pos - 1, k, v);
+    return defRetValue;
+  }
+
+  /**
+   * {@inheritDoc}
+   */
+  @Override
+
+  public boolean remove(final long k, final long v) {
+    if (((k) == 0)) {
+      if (containsNullKey && ((v) == (value.get(n)))) {
+        removeNullEntry();
+        return true;
+      }
+      return false;
+    }
+    long curr;
+    long pos;
+    // The starting point.
+    if (((curr = key.get(pos = HashCommon.mix((k)) & mask)) == 0)) {
+      return false;
+    }
+    if (((k) == (curr)) && ((v) == (value.get(pos)))) {
+      removeEntry(pos);
+      return true;
+    }
+    while (true) {
+      if (((curr = key.get(pos = (pos + 1) & mask)) == 0)) {
+        return false;
+      }
+      if (((k) == (curr)) && ((v) == (value.get(pos)))) {
+        removeEntry(pos);
+        return true;
+      }
+    }
+  }
+
+  /**
+   * {@inheritDoc}
+   */
+  @Override
+  public boolean replace(final long k, final long oldValue, final long v) {
+    final long pos = find(k);
+    if (pos < 0 || !(oldValue == (value.get(pos)))) {
+      return false;
+    }
+    value.set(pos, v);
+    return true;
+  }
+
+  /**
+   * {@inheritDoc}
+   */
+  @Override
+  public long replace(final long k, final long v) {
+    final long pos = find(k);
+    if (pos < 0) {
+      return defRetValue;
+    }
+    final long oldValue = value.get(pos);
+    value.set(pos, v);
+    return oldValue;
+  }
+
+  /**
+   * {@inheritDoc}
+   */
+  @Override
+  public long computeIfAbsent(final long k,
+      final java.util.function.LongUnaryOperator mappingFunction) {
+    java.util.Objects.requireNonNull(mappingFunction);
+    final long pos = find(k);
+    if (pos >= 0) {
+      return value.get(pos);
+    }
+    final long newValue = mappingFunction.applyAsLong(k);
+    insert(-pos - 1, k, newValue);
+    return newValue;
+  }
+
+  /**
+   * {@inheritDoc}
+   */
+  @Override
+  public long computeIfAbsent(final long key, final Long2LongFunction mappingFunction) {

Review Comment:
   ## Confusing overloading of methods
   
   Method Long2LongOpenHashDataMap.computeIfAbsent(..) could be confused with overloaded method [computeIfAbsent](1), since dispatch depends on static types.
   
   [Show more details](https://github.com/apache/incubator-baremaps/security/code-scanning/757)



##########
baremaps-core/src/main/java/org/apache/baremaps/database/collection/Long2ObjectOpenHashDataMap.java:
##########
@@ -0,0 +1,1468 @@
+/*
+ * Licensed 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.baremaps.database.collection;
+
+import static it.unimi.dsi.fastutil.HashCommon.*;
+
+import it.unimi.dsi.fastutil.Hash;
+import it.unimi.dsi.fastutil.HashCommon;
+import it.unimi.dsi.fastutil.longs.*;
+import it.unimi.dsi.fastutil.objects.AbstractObjectCollection;
+import it.unimi.dsi.fastutil.objects.AbstractObjectSet;
+import it.unimi.dsi.fastutil.objects.ObjectCollection;
+import it.unimi.dsi.fastutil.objects.ObjectIterator;
+import it.unimi.dsi.fastutil.objects.ObjectSpliterator;
+import it.unimi.dsi.fastutil.objects.ObjectSpliterators;
+import java.util.Map;
+import java.util.NoSuchElementException;
+import java.util.concurrent.atomic.AtomicLong;
+import java.util.function.Consumer;
+import java.util.function.Supplier;
+
+/**
+ * An open addressed hash map of long keys and long values derived from fastutil's
+ * {@link Long2ObjectOpenHashMap}. This implementation allows for the use of on-heap, off-heap, or
+ * memory mapped memory.
+ */
+public class Long2ObjectOpenHashDataMap<V> extends AbstractLong2ObjectMap<V>
+    implements DataMap<V>, Hash {
+
+  private final Supplier<DataMap<Long>> keySupplier;
+
+  private final Supplier<DataMap<V>> valueSupplier;
+
+  /**
+   * The array of keys.
+   */
+  protected DataMap<Long> key;
+  /**
+   * The array of values.
+   */
+  protected DataMap<V> value;
+  /**
+   * The mask for wrapping a position counter.
+   */
+  protected long mask;
+  /**
+   * Whether this map contains the key zero.
+   */
+  protected boolean containsNullKey;
+  /**
+   * The current table size.
+   */
+  protected long n;
+  /**
+   * Threshold after which we rehash. It must be the table size times {@link #f}.
+   */
+  protected long maxFill;
+  /**
+   * We never resize below this threshold, which is the construction-time {#n}.
+   */
+  protected final long minN;
+  /**
+   * Number of entries in the set (including the key zero, if present).
+   */
+  protected AtomicLong size = new AtomicLong();;
+  /**
+   * The acceptable load factor.
+   */
+  protected final float f;
+  /**
+   * Cached set of entries.
+   */
+  protected FastEntrySet<V> entries;
+  /**
+   * Cached set of keys.
+   */
+  protected LongSet keys;
+  /**
+   * Cached collection of values.
+   */
+  protected ObjectCollection<V> values;
+
+  /**
+   * Creates a new hash map.
+   *
+   * <p>
+   * The actual table size will be the least power of two greater than {@code expected}/{@code f}.
+   *
+   * @param expected the expected number of elements in the hash map.
+   * @param f the load factor.
+   */
+  public Long2ObjectOpenHashDataMap(
+      final long expected,
+      final float f,
+      final Supplier<DataMap<Long>> keySupplier,
+      final Supplier<DataMap<V>> valueSupplier) {
+    if (f <= 0 || f >= 1) {
+      throw new IllegalArgumentException("Load factor must be greater than 0 and smaller than 1");
+    }
+    if (expected < 0) {
+      throw new IllegalArgumentException("The expected number of elements must be nonnegative");
+    }
+    this.f = f;
+    this.minN = n = bigArraySize(expected, f);
+    this.mask = n - 1;
+    this.maxFill = maxFill(n, f);
+    this.keySupplier = keySupplier;
+    this.valueSupplier = valueSupplier;
+    this.key = keySupplier.get();
+    this.value = valueSupplier.get();
+  }
+
+  private long realSize() {
+    return containsNullKey ? size.get() - 1 : size.get();
+  }
+
+  private void ensureCapacity(final long capacity) {
+    final long needed = bigArraySize(capacity, f);
+    if (needed > n) {
+      rehash(needed);
+    }
+  }
+
+  private void tryCapacity(final long capacity) {
+    final long needed = Math.min(1 << 30,
+        Math.max(2, HashCommon.nextPowerOfTwo((long) Math.ceil(capacity / f))));
+    if (needed > n) {
+      rehash(needed);
+    }
+  }
+
+  private V removeEntry(final long pos) {
+    final V oldValue = value.get(pos);
+    value.put(pos, null);
+    size.decrementAndGet();
+    shiftKeys(pos);
+    if (n > minN && size.get() < maxFill / 4 && n > DEFAULT_INITIAL_SIZE) {
+      rehash(n / 2);
+    }
+    return oldValue;
+  }
+
+  private V removeNullEntry() {
+    containsNullKey = false;
+    final V oldValue = value.get(n);
+    value.put(n, null);
+    size.decrementAndGet();
+    if (n > minN && size.get() < maxFill / 4 && n > DEFAULT_INITIAL_SIZE) {
+      rehash(n / 2);
+    }
+    return oldValue;
+  }
+
+  @Override
+  public void putAll(Map<? extends Long, ? extends V> m) {
+    if (f <= .5) {
+      ensureCapacity(m.size()); // The resulting map will be sized for m.size() elements
+    } else {
+      tryCapacity(size() + m.size()); // The resulting map will be tentatively sized for size() +
+    }
+    // m.size()
+    // elements
+    super.putAll(m);
+  }
+
+  private long find(final long k) {
+    if (((k) == 0)) {
+      return containsNullKey ? n : -(n + 1);
+    }
+    long curr;
+    long pos;
+    // The starting point.
+    if (((curr = key.get(pos = HashCommon.mix((k)) & mask)) == 0)) {
+      return -(pos + 1);
+    }
+    if (((k) == (curr))) {
+      return pos;
+    }
+    // There's always an unused entry.
+    while (true) {
+      if (((curr = key.get(pos = (pos + 1) & mask)) == 0)) {
+        return -(pos + 1);
+      }
+      if (((k) == (curr))) {
+        return pos;
+      }
+    }
+  }
+
+  private void insert(final long pos, final long k, final V v) {
+    if (pos == n) {
+      containsNullKey = true;
+    }
+    key.put(pos, k);
+    value.put(pos, v);
+    if (size.getAndIncrement() >= maxFill) {
+      rehash(bigArraySize(size.get() + 1, f));
+    }
+  }
+
+  @Override
+  public V put(final long k, final V v) {
+    final long pos = find(k);
+    if (pos < 0) {
+      insert(-pos - 1, k, v);
+      return defRetValue;
+    }
+    final V oldValue = value.get(pos);
+    value.put(pos, v);
+    return oldValue;
+  }
+
+  /**
+   * Shifts left entries with the specified hash code, starting at the specified position, and
+   * empties the resulting free entry.
+   *
+   * @param pos a starting position.
+   */
+  protected final void shiftKeys(long pos) {
+    // Shift entries with the same hash.
+    long last, slot;
+    long curr;
+    for (;;) {
+      pos = ((last = pos) + 1) & mask;
+      for (;;) {
+        if (((curr = key.get(pos)) == 0)) {
+          key.put(last, 0L);
+          value.put(last, null);
+          return;
+        }
+        slot = HashCommon.mix((curr)) & mask;
+        if (last <= pos ? last >= slot || slot > pos : last >= slot && slot > pos) {
+          break;
+        }
+        pos = (pos + 1) & mask;
+      }
+      key.put(last, curr);
+      value.put(last, value.get(pos));
+    }
+  }
+
+  @Override
+
+  public V remove(final long k) {
+    if (((k) == 0)) {
+      if (containsNullKey) {
+        return removeNullEntry();
+      }
+      return defRetValue;
+    }
+    long curr;
+    long pos;
+    // The starting point.
+    if (((curr = key.get(pos = HashCommon.mix((k)) & mask)) == 0)) {
+      return defRetValue;
+    }
+    if (((k) == (curr))) {
+      return removeEntry(pos);
+    }
+    while (true) {
+      if (((curr = key.get(pos = (pos + 1) & mask)) == 0)) {
+        return defRetValue;
+      }
+      if (((k) == (curr))) {
+        return removeEntry(pos);
+      }
+    }
+  }
+
+  @Override
+
+  public V get(final long k) {
+    if (((k) == 0)) {
+      return containsNullKey ? value.get(n) : defRetValue;
+    }
+    long curr;
+    long pos;
+    // The starting point.
+    if (((curr = key.get(pos = HashCommon.mix((k)) & mask)) == 0)) {
+      return defRetValue;
+    }
+    if (((k) == (curr))) {
+      return value.get(pos);
+    }
+    // There's always an unused entry.
+    while (true) {
+      if (((curr = key.get(pos = (pos + 1) & mask)) == 0)) {
+        return defRetValue;
+      }
+      if (((k) == (curr))) {
+        return value.get(pos);
+      }
+    }
+  }
+
+  @Override
+
+  public boolean containsKey(final long k) {
+    if (((k) == 0)) {
+      return containsNullKey;
+    }
+    long curr;
+    long pos;
+    // The starting point.
+    if (((curr = key.get(pos = HashCommon.mix((k)) & mask)) == 0)) {
+      return false;
+    }
+    if (((k) == (curr))) {
+      return true;
+    }
+    // There's always an unused entry.
+    while (true) {
+      if (((curr = key.get(pos = (pos + 1) & mask)) == 0)) {
+        return false;
+      }
+      if (((k) == (curr))) {
+        return true;
+      }
+    }
+  }
+
+  @Override
+  public boolean containsValue(final Object v) {
+    if (containsNullKey && java.util.Objects.equals(value.get(n), v)) {
+      return true;
+    }
+    for (long i = n; i-- != 0;) {
+      if (!((key.get(i)) == 0) && java.util.Objects.equals(value.get(i), v)) {
+        return true;
+      }
+    }
+    return false;
+  }
+
+  /**
+   * {@inheritDoc}
+   */
+  @Override
+
+  public V getOrDefault(final long k, final V defaultValue) {
+    if (((k) == 0)) {
+      return containsNullKey ? value.get(n) : defaultValue;
+    }
+    long curr;
+    long pos;
+    // The starting point.
+    if (((curr = key.get(pos = HashCommon.mix((k)) & mask)) == 0)) {
+      return defaultValue;
+    }
+    if (((k) == (curr))) {
+      return value.get(pos);
+    }
+    // There's always an unused entry.
+    while (true) {
+      if (((curr = key.get(pos = (pos + 1) & mask)) == 0)) {
+        return defaultValue;
+      }
+      if (((k) == (curr))) {
+        return value.get(pos);
+      }
+    }
+  }
+
+  /**
+   * {@inheritDoc}
+   */
+  @Override
+  public V putIfAbsent(final long k, final V v) {
+    final long pos = find(k);
+    if (pos >= 0) {
+      return value.get(pos);
+    }
+    insert(-pos - 1, k, v);
+    return defRetValue;
+  }
+
+  /**
+   * {@inheritDoc}
+   */
+  @Override
+
+  public boolean remove(final long k, final Object v) {
+    if (((k) == 0)) {
+      if (containsNullKey && java.util.Objects.equals(v, value.get(n))) {
+        removeNullEntry();
+        return true;
+      }
+      return false;
+    }
+    long curr;
+    long pos;
+    // The starting point.
+    if (((curr = key.get(pos = HashCommon.mix((k)) & mask)) == 0)) {
+      return false;
+    }
+    if (((k) == (curr)) && java.util.Objects.equals(v, value.get(pos))) {
+      removeEntry(pos);
+      return true;
+    }
+    while (true) {
+      if (((curr = key.get(pos = (pos + 1) & mask)) == 0)) {
+        return false;
+      }
+      if (((k) == (curr)) && java.util.Objects.equals(v, value.get(pos))) {
+        removeEntry(pos);
+        return true;
+      }
+    }
+  }
+
+  /**
+   * {@inheritDoc}
+   */
+  @Override
+  public boolean replace(final long k, final V oldValue, final V v) {
+    final long pos = find(k);
+    if (pos < 0 || !java.util.Objects.equals(oldValue, value.get(pos))) {
+      return false;
+    }
+    value.put(pos, v);
+    return true;
+  }
+
+  /**
+   * {@inheritDoc}
+   */
+  @Override
+  public V replace(final long k, final V v) {
+    final long pos = find(k);
+    if (pos < 0) {
+      return defRetValue;
+    }
+    final V oldValue = value.get(pos);
+    value.put(pos, v);
+    return oldValue;
+  }
+
+  /**
+   * {@inheritDoc}
+   */
+  @Override
+  public V computeIfAbsent(final long k,
+      final java.util.function.LongFunction<? extends V> mappingFunction) {
+    java.util.Objects.requireNonNull(mappingFunction);
+    final long pos = find(k);
+    if (pos >= 0) {
+      return value.get(pos);
+    }
+    final V newValue = mappingFunction.apply(k);
+    insert(-pos - 1, k, newValue);
+    return newValue;
+  }
+
+  /**
+   * {@inheritDoc}
+   */
+  @Override
+  public V computeIfAbsent(final long key, final Long2ObjectFunction<? extends V> mappingFunction) {

Review Comment:
   ## Confusing overloading of methods
   
   Method Long2ObjectOpenHashDataMap.computeIfAbsent(..) could be confused with overloaded method [computeIfAbsent](1), since dispatch depends on static types.
   
   [Show more details](https://github.com/apache/incubator-baremaps/security/code-scanning/759)



##########
baremaps-core/src/main/java/org/apache/baremaps/database/collection/Long2LongPackedOpenHashDataMap.java:
##########
@@ -0,0 +1,1555 @@
+/*
+ * Licensed 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.baremaps.database.collection;
+
+import static it.unimi.dsi.fastutil.HashCommon.bigArraySize;
+import static it.unimi.dsi.fastutil.HashCommon.maxFill;
+
+import it.unimi.dsi.fastutil.Hash;
+import it.unimi.dsi.fastutil.HashCommon;
+import it.unimi.dsi.fastutil.longs.*;
+import it.unimi.dsi.fastutil.objects.*;
+import java.util.Map;
+import java.util.NoSuchElementException;
+import java.util.concurrent.atomic.AtomicLong;
+import java.util.function.Consumer;
+import java.util.function.LongConsumer;
+import java.util.function.Supplier;
+import org.apache.baremaps.database.type.PairDataType.Pair;
+
+/**
+ * An open addressed hash map of long keys and long values derived from fastutil's
+ * {@link Long2LongOpenHashMap}. This implementation allows for the use of on-heap, off-heap, or
+ * memory mapped memory. The keys and values are packed in the same memory.
+ */
+public class Long2LongPackedOpenHashDataMap extends AbstractLong2LongMap
+    implements DataMap<Long>, Hash {
+
+  private final Supplier<DataMap<Pair<Long, Long>>> indexSupplier;
+
+  /**
+   * The hash index.
+   */
+  protected DataMap<Pair<Long, Long>> index;
+  /**
+   * The mask for wrapping a position counter.
+   */
+  protected long mask;
+  /**
+   * Whether this map contains the key zero.
+   */
+  protected boolean containsNullKey;
+  /**
+   * The current table size.
+   */
+  protected long n;
+  /**
+   * Threshold after which we rehash. It must be the table size times {@link #f}.
+   */
+  protected long maxFill;
+  /**
+   * We never resize below this threshold, which is the construction-time {#n}.
+   */
+  protected final long minN;
+  /**
+   * Number of entries in the set (including the key zero, if present).
+   */
+  protected AtomicLong size = new AtomicLong();
+  /**
+   * The acceptable load factor.
+   */
+  protected final float f;
+  /**
+   * Cached set of entries.
+   */
+  protected FastEntrySet entries;
+  /**
+   * Cached set of keys.
+   */
+  protected LongSet keys;
+  /**
+   * Cached collection of values.
+   */
+  protected LongCollection values;
+
+  /**
+   * Creates a new hash map.
+   *
+   * <p>
+   * The actual table size will be the least power of two greater than {@code expected}/{@code f}.
+   *
+   * @param expected the expected number of elements in the hash map.
+   * @param f the load factor.
+   */
+  public Long2LongPackedOpenHashDataMap(final long expected, final float f,
+      final Supplier<DataMap<Pair<Long, Long>>> indexSupplier) {
+    if (f <= 0 || f >= 1) {
+      throw new IllegalArgumentException("Load factor must be greater than 0 and smaller than 1");
+    }
+    if (expected < 0) {
+      throw new IllegalArgumentException("The expected number of elements must be nonnegative");
+    }
+    this.f = f;
+    this.minN = n = bigArraySize(expected, f);
+    this.mask = n - 1;
+    this.maxFill = maxFill(n, f);
+    this.indexSupplier = indexSupplier;
+    this.index = indexSupplier.get();
+  }
+
+  private long realSize() {
+    return containsNullKey ? size.get() - 1 : size.get();
+  }
+
+  private void ensureCapacity(final long capacity) {
+    final long needed = bigArraySize(capacity, f);
+    if (needed > n) {
+      rehash(needed);
+    }
+  }
+
+  private void tryCapacity(final long capacity) {
+    final long needed =
+        Math.min(1 << 30, Math.max(2, HashCommon.nextPowerOfTwo((long) Math.ceil(capacity / f))));
+    if (needed > n) {
+      rehash(needed);
+    }
+  }
+
+  private long removeEntry(final long pos) {
+    final long oldValue = index.get(pos).right();
+    size.decrementAndGet();
+    shiftKeys(pos);
+    if (n > minN && size.get() < maxFill / 4 && n > DEFAULT_INITIAL_SIZE) {
+      rehash(n / 2);
+    }
+    return oldValue;
+  }
+
+  private long removeNullEntry() {
+    containsNullKey = false;
+    final long oldValue = index.get(n).right();
+    size.decrementAndGet();
+    if (n > minN && size.get() < maxFill / 4 && n > DEFAULT_INITIAL_SIZE) {
+      rehash(n / 2);
+    }
+    return oldValue;
+  }
+
+  @Override
+  public void putAll(Map<? extends Long, ? extends Long> m) {
+    if (f <= .5) {
+      ensureCapacity(m.size());
+    } else {
+      tryCapacity(size() + m.size());
+    }
+    super.putAll(m);
+  }
+
+  private long find(final long k) {
+    if (((k) == 0)) {
+      return containsNullKey ? n : -(n + 1);
+    }
+    long curr;
+    long pos;
+    if (((curr = index.get(pos = HashCommon.mix((k)) & mask).left()) == 0)) {
+      return -(pos + 1);
+    }
+    if (((k) == (curr))) {
+      return pos;
+    }
+    while (true) {
+      if (((curr = index.get(pos = (pos + 1) & mask).left()) == 0)) {
+        return -(pos + 1);
+      }
+      if (((k) == (curr))) {
+        return pos;
+      }
+    }
+  }
+
+  private void insert(final long pos, final long k, final long v) {
+    if (pos == n) {
+      containsNullKey = true;
+    }
+    index.put(pos, new Pair<>(k, v));
+    if (size.getAndIncrement() >= maxFill) {
+      rehash(bigArraySize(size.get() + 1, f));
+    }
+  }
+
+  @Override
+  public long put(final long k, final long v) {
+    final long pos = find(k);
+    if (pos < 0) {
+      insert(-pos - 1, k, v);
+      return defRetValue;
+    }
+    final long oldValue = index.get(pos).right();
+    index.put(pos, new Pair<>(pos, v));
+    return oldValue;
+  }
+
+  private long addToValue(final long pos, final long incr) {
+    final long oldValue = index.get(pos).right();
+    index.put(pos, new Pair<>(pos, oldValue + incr));
+    return oldValue;
+  }
+
+  /**
+   * Adds an increment to value currently associated with a key.
+   *
+   * <p>
+   * Note that this method respects the {@linkplain #defaultReturnValue() default return value}
+   * semantics: when called with a key that does not currently appears in the map, the key will be
+   * associated with the default return value plus the given increment.
+   *
+   * @param k the key.
+   * @param incr the increment.
+   * @return the old value, or the {@linkplain #defaultReturnValue() default return value} if no
+   *         value was present for the given key.
+   */
+  public long addTo(final long k, final long incr) {
+    long pos;
+    if (((k) == 0)) {
+      if (containsNullKey) {
+        return addToValue(n, incr);
+      }
+      pos = n;
+      containsNullKey = true;
+    } else {
+      long curr;
+      // The starting point.
+      if (!((curr = index.get(pos = HashCommon.mix((k)) & mask).left()) == 0)) {
+        if ((curr == k)) {
+          return addToValue(pos, incr);
+        }
+        while (!((curr = index.get(pos = (pos + 1) & mask).left()) == 0)) {
+          if ((curr == k)) {
+            return addToValue(pos, incr);
+          }
+        }
+      }
+    }
+    index.put(pos, new Pair<>(k, defRetValue + incr));
+    if (size.incrementAndGet() >= maxFill) {
+      rehash(bigArraySize(size.get() + 1, f));
+    }
+    return defRetValue;
+  }
+
+  /**
+   * Shifts left entries with the specified hash code, starting at the specified position, and
+   * empties the resulting free entry.
+   *
+   * @param pos a starting position.
+   */
+  protected final void shiftKeys(long pos) {
+    // Shift entries with the same hash.
+    long last, slot;
+    long curr;
+    for (;;) {
+      pos = ((last = pos) + 1) & mask;
+      for (;;) {
+        if (((curr = index.get(pos).left()) == 0)) {
+          index.put(last, new Pair<>(curr, 0L));
+          return;
+        }
+        slot = HashCommon.mix((curr)) & mask;
+        if (last <= pos ? last >= slot || slot > pos : last >= slot && slot > pos) {
+          break;
+        }
+        pos = (pos + 1) & mask;
+      }
+      index.put(last, new Pair<>(curr, index.get(pos).right()));
+    }
+  }
+
+  @Override
+
+  public long remove(final long k) {
+    if (((k) == 0)) {
+      if (containsNullKey) {
+        return removeNullEntry();
+      }
+      return defRetValue;
+    }
+    long curr;
+    long pos;
+    // The starting point.
+    if (((curr = index.get(pos = HashCommon.mix((k)) & mask).left()) == 0)) {
+      return defRetValue;
+    }
+    if (((k) == (curr))) {
+      return removeEntry(pos);
+    }
+    while (true) {
+      if (((curr = index.get(pos = (pos + 1) & mask).left()) == 0)) {
+        return defRetValue;
+      }
+      if (((k) == (curr))) {
+        return removeEntry(pos);
+      }
+    }
+  }
+
+  @Override
+
+  public long get(final long k) {
+    if (((k) == 0)) {
+      return containsNullKey ? index.get(n).right() : defRetValue;
+    }
+    long curr;
+    long pos;
+    // The starting point.
+    if (((curr = index.get(pos = HashCommon.mix((k)) & mask).left()) == 0)) {
+      return defRetValue;
+    }
+    if (((k) == (curr))) {
+      return index.get(pos).right();
+    }
+    // There's always an unused entry.
+    while (true) {
+      if (((curr = index.get(pos = (pos + 1) & mask).left()) == 0)) {
+        return defRetValue;
+      }
+      if (((k) == (curr))) {
+        return index.get(pos).right();
+      }
+    }
+  }
+
+  @Override
+
+  public boolean containsKey(final long k) {
+    if (((k) == 0)) {
+      return containsNullKey;
+    }
+    long curr;
+    long pos;
+    // The starting point.
+    if (((curr = index.get(pos = HashCommon.mix((k)) & mask).left()) == 0)) {
+      return false;
+    }
+    if (((k) == (curr))) {
+      return true;
+    }
+    // There's always an unused entry.
+    while (true) {
+      if (((curr = index.get(pos = (pos + 1) & mask).left()) == 0)) {
+        return false;
+      }
+      if (((k) == (curr))) {
+        return true;
+      }
+    }
+  }
+
+  @Override
+  public boolean containsValue(final long v) {
+    if (containsNullKey && ((index.get(n).right()) == (v))) {
+      return true;
+    }
+    for (long i = n; i-- != 0;) {
+      if (!((index.get(i).left()) == 0) && ((index.get(i).right()) == (v))) {
+        return true;
+      }
+    }
+    return false;
+  }
+
+  /**
+   * {@inheritDoc}
+   */
+  @Override
+
+  public long getOrDefault(final long k, final long defaultValue) {
+    if (((k) == 0)) {
+      return containsNullKey ? index.get(n).right() : defaultValue;
+    }
+    long curr;
+    long pos;
+    // The starting point.
+    if (((curr = index.get(pos = HashCommon.mix((k)) & mask).left()) == 0)) {
+      return defaultValue;
+    }
+    if (((k) == (curr))) {
+      return index.get(pos).right();
+    }
+    // There's always an unused entry.
+    while (true) {
+      if (((curr = index.get(pos = (pos + 1) & mask).left()) == 0)) {
+        return defaultValue;
+      }
+      if (((k) == (curr))) {
+        return index.get(pos).right();
+      }
+    }
+  }
+
+  /**
+   * {@inheritDoc}
+   */
+  @Override
+  public long putIfAbsent(final long k, final long v) {
+    final long pos = find(k);
+    if (pos >= 0) {
+      return index.get(pos).right();
+    }
+    insert(-pos - 1, k, v);
+    return defRetValue;
+  }
+
+  /**
+   * {@inheritDoc}
+   */
+  @Override
+
+  public boolean remove(final long k, final long v) {
+    if (((k) == 0)) {
+      if (containsNullKey && ((v) == (index.get(n).right()))) {
+        removeNullEntry();
+        return true;
+      }
+      return false;
+    }
+    long curr;
+    long pos;
+    // The starting point.
+    if (((curr = index.get(pos = HashCommon.mix((k)) & mask).left()) == 0)) {
+      return false;
+    }
+    if (((k) == (curr)) && ((v) == (index.get(pos).right()))) {
+      removeEntry(pos);
+      return true;
+    }
+    while (true) {
+      if (((curr = index.get(pos = (pos + 1) & mask).left()) == 0)) {
+        return false;
+      }
+      if (((k) == (curr)) && ((v) == (index.get(pos).right()))) {
+        removeEntry(pos);
+        return true;
+      }
+    }
+  }
+
+  /**
+   * {@inheritDoc}
+   */
+  @Override
+  public boolean replace(final long k, final long oldValue, final long v) {
+    final long pos = find(k);
+    if (pos < 0 || !((oldValue) == (index.get(pos).right()))) {
+      return false;
+    }
+    index.put(pos, new Pair<>(k, v));
+    return true;
+  }
+
+  /**
+   * {@inheritDoc}
+   */
+  @Override
+  public long replace(final long k, final long v) {
+    final long pos = find(k);
+    if (pos < 0) {
+      return defRetValue;
+    }
+    final long oldValue = index.get(pos).right();
+    index.put(pos, new Pair<>(k, v));
+    return oldValue;
+  }
+
+  /**
+   * {@inheritDoc}
+   */
+  @Override
+  public long computeIfAbsent(final long k,
+      final java.util.function.LongUnaryOperator mappingFunction) {
+    java.util.Objects.requireNonNull(mappingFunction);
+    final long pos = find(k);
+    if (pos >= 0) {
+      return index.get(pos).right();
+    }
+    final long newValue = mappingFunction.applyAsLong(k);
+    insert(-pos - 1, k, newValue);
+    return newValue;
+  }
+
+  /**
+   * {@inheritDoc}
+   */
+  @Override
+  public long computeIfAbsent(final long key, final Long2LongFunction mappingFunction) {

Review Comment:
   ## Confusing overloading of methods
   
   Method Long2LongPackedOpenHashDataMap.computeIfAbsent(..) could be confused with overloaded method [computeIfAbsent](1), since dispatch depends on static types.
   
   [Show more details](https://github.com/apache/incubator-baremaps/security/code-scanning/758)



##########
baremaps-core/src/main/java/org/apache/baremaps/database/collection/Long2LongOpenHashDataMap.java:
##########
@@ -0,0 +1,1553 @@
+/*
+ * Licensed 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.baremaps.database.collection;
+
+import static it.unimi.dsi.fastutil.HashCommon.*;
+
+import it.unimi.dsi.fastutil.Hash;
+import it.unimi.dsi.fastutil.HashCommon;
+import it.unimi.dsi.fastutil.longs.*;
+import it.unimi.dsi.fastutil.objects.*;
+import java.util.Map;
+import java.util.NoSuchElementException;
+import java.util.concurrent.atomic.AtomicLong;
+import java.util.function.Consumer;
+import java.util.function.LongConsumer;
+import java.util.function.Supplier;
+
+/**
+ * An open addressed hash map of long keys and long values derived from fastutil's
+ * {@link Long2LongOpenHashMap}. This implementation allows for the use of on-heap, off-heap, or
+ * memory mapped memory.
+ */
+public class Long2LongOpenHashDataMap extends AbstractLong2LongMap
+    implements DataMap<Long>, Hash {
+
+  protected Supplier<AbstractDataList<Long>> keySupplier;
+
+  protected Supplier<AbstractDataList<Long>> valueSupplier;
+
+  /**
+   * The array of keys.
+   */
+  protected AbstractDataList<Long> key;
+  /**
+   * The array of values.
+   */
+  protected AbstractDataList<Long> value;
+  /**
+   * The mask for wrapping a position counter.
+   */
+  protected long mask;
+  /**
+   * Whether this map contains the key zero.
+   */
+  protected boolean containsNullKey;
+  /**
+   * The current table size.
+   */
+  protected long n;
+  /**
+   * Threshold after which we rehash. It must be the table size times {@link #f}.
+   */
+  protected long maxFill;
+  /**
+   * We never resize below this threshold, which is the construction-time {#n}.
+   */
+  protected final long minN;
+  /**
+   * Number of entries in the set (including the key zero, if present).
+   */
+  protected AtomicLong size = new AtomicLong();
+  /**
+   * The acceptable load factor.
+   */
+  protected final float f;
+  /**
+   * Cached set of entries.
+   */
+  protected FastEntrySet entries;
+
+  /**
+   * Cached set of keys.
+   */
+  protected LongSet keys;
+  /**
+   * Cached collection of values.
+   */
+  protected LongCollection values;
+
+  /**
+   * Creates a new hash map.
+   *
+   * <p>
+   * The actual table size will be the least power of two greater than {@code expected}/{@code f}.
+   *
+   * @param expected the expected number of elements in the hash map.
+   * @param f the load factor.
+   */
+  public Long2LongOpenHashDataMap(
+      long expected,
+      float f,
+      Supplier<AbstractDataList<Long>> keySupplier,
+      Supplier<AbstractDataList<Long>> valueSupplier) {
+    if (f <= 0 || f >= 1) {
+      throw new IllegalArgumentException("Load factor must be greater than 0 and smaller than 1");
+    }
+    if (expected < 0) {
+      throw new IllegalArgumentException("The expected number of elements must be non-negative");
+    }
+    this.f = f;
+    this.minN = n = bigArraySize(expected, f);
+    this.mask = n - 1;
+    this.maxFill = maxFill(n, f);
+    this.keySupplier = keySupplier;
+    this.valueSupplier = valueSupplier;
+    this.key = keySupplier.get();
+    this.value = valueSupplier.get();
+  }
+
+  private long realSize() {
+    return containsNullKey ? size.get() - 1 : size.get();
+  }
+
+  private void ensureCapacity(final long capacity) {
+    final long needed = bigArraySize(capacity, f);
+    if (needed > n) {
+      rehash(needed);
+    }
+  }
+
+  private void tryCapacity(final long capacity) {
+    final long needed =
+        Math.min(1 << 30, Math.max(2, HashCommon.nextPowerOfTwo((long) Math.ceil(capacity / f))));
+    if (needed > n) {
+      rehash(needed);
+    }
+  }
+
+  private long removeEntry(final long pos) {
+    final long oldValue = value.get(pos);
+    size.decrementAndGet();
+    shiftKeys(pos);
+    if (n > minN && size.get() < maxFill / 4 && n > DEFAULT_INITIAL_SIZE) {
+      rehash(n / 2);
+    }
+    return oldValue;
+  }
+
+  private long removeNullEntry() {
+    containsNullKey = false;
+    final long oldValue = value.get(n);
+    size.decrementAndGet();
+    if (n > minN && size.get() < maxFill / 4 && n > DEFAULT_INITIAL_SIZE) {
+      rehash(n / 2);
+    }
+    return oldValue;
+  }
+
+  @Override
+  public void putAll(Map<? extends Long, ? extends Long> m) {
+    if (f <= .5) {
+      ensureCapacity(m.size());
+    } else {
+      tryCapacity(sizeAsLong() + m.size());
+    }
+    super.putAll(m);
+  }
+
+  private long find(final long k) {
+    if (((k) == 0)) {
+      return containsNullKey ? n : -(n + 1);
+    }
+    long curr;
+    long pos;
+    // The starting point.
+    if (((curr = key.get(pos = HashCommon.mix((k)) & mask)) == 0)) {
+      return -(pos + 1);
+    }
+    if (((k) == (curr))) {
+      return pos;
+    }
+    // There's always an unused entry.
+    while (true) {
+      if (((curr = key.get(pos = (pos + 1) & mask)) == 0)) {
+        return -(pos + 1);
+      }
+      if (((k) == (curr))) {
+        return pos;
+      }
+    }
+  }
+
+  private void insert(final long pos, final long k, final long v) {
+    if (pos == n) {
+      containsNullKey = true;
+    }
+    key.set(pos, k);
+    value.set(pos, v);
+    if (size.getAndIncrement() >= maxFill) {
+      rehash(bigArraySize(size.get() + 1, f));
+    }
+  }
+
+  @Override
+  public long put(final long k, final long v) {
+    final long pos = find(k);
+    if (pos < 0) {
+      insert(-pos - 1, k, v);
+      return defRetValue;
+    }
+    final long oldValue = value.get(pos);
+    value.set(pos, v);
+    return oldValue;
+  }
+
+  private long addToValue(final long pos, final long incr) {
+    final long oldValue = value.get(pos);
+    value.set(pos, oldValue + incr);
+    return oldValue;
+  }
+
+  /**
+   * Adds an increment to value currently associated with a key.
+   *
+   * <p>
+   * Note that this method respects the {@linkplain #defaultReturnValue() default return value}
+   * semantics: when called with a key that does not currently appears in the map, the key will be
+   * associated with the default return value plus the given increment.
+   *
+   * @param k the key.
+   * @param incr the increment.
+   * @return the old value, or the {@linkplain #defaultReturnValue() default return value} if no
+   *         value was present for the given key.
+   */
+  public long addTo(final long k, final long incr) {
+    long pos;
+    if (((k) == 0)) {
+      if (containsNullKey) {
+        return addToValue(n, incr);
+      }
+      pos = n;
+      containsNullKey = true;
+    } else {
+      long curr;
+      // The starting point.
+      if (!((curr = key.get(pos = HashCommon.mix((k)) & mask)) == 0)) {
+        if (((curr) == (k))) {
+          return addToValue(pos, incr);
+        }
+        while (!((curr = key.get(pos = (pos + 1) & mask)) == 0)) {
+          if (((curr) == (k))) {
+            return addToValue(pos, incr);
+          }
+        }
+      }
+    }
+    key.set(pos, k);
+    value.set(pos, defRetValue + incr);
+    if (size.incrementAndGet() >= maxFill)
+      rehash(bigArraySize(size.get() + 1, f));
+    return defRetValue;
+  }
+
+  /**
+   * Shifts left entries with the specified hash code, starting at the specified position, and
+   * empties the resulting free entry.
+   *
+   * @param pos a starting position.
+   */
+  protected final void shiftKeys(long pos) {
+    // Shift entries with the same hash.
+    long last, slot;
+    long curr;
+    for (;;) {
+      pos = ((last = pos) + 1) & mask;
+      for (;;) {
+        if (((curr = key.get(pos)) == 0)) {
+          key.set(last, 0L);
+          return;
+        }
+        slot = HashCommon.mix((curr)) & mask;
+        if (last <= pos ? last >= slot || slot > pos : last >= slot && slot > pos) {
+          break;
+        }
+        pos = (pos + 1) & mask;
+      }
+      key.set(last, curr);
+      value.set(last, value.get(pos));
+    }
+  }
+
+  @Override
+
+  public long remove(final long k) {
+    if (((k) == 0)) {
+      if (containsNullKey) {
+        return removeNullEntry();
+      }
+      return defRetValue;
+    }
+    long curr;
+    long pos;
+    // The starting point.
+    if (((curr = key.get(pos = HashCommon.mix((k)) & mask)) == 0)) {
+      return defRetValue;
+    }
+    if (((k) == (curr))) {
+      return removeEntry(pos);
+    }
+    while (true) {
+      if (((curr = key.get(pos = (pos + 1) & mask)) == 0)) {
+        return defRetValue;
+      }
+      if (((k) == (curr))) {
+        return removeEntry(pos);
+      }
+    }
+  }
+
+  @Override
+
+  public long get(final long k) {
+    if (((k) == 0)) {
+      return containsNullKey ? value.get(n) : defRetValue;
+    }
+    long curr;
+    long pos;
+    // The starting point.
+    if (((curr = key.get(pos = HashCommon.mix((k)) & mask)) == 0)) {
+      return defRetValue;
+    }
+    if (((k) == (curr))) {
+      return value.get(pos);
+    }
+    // There's always an unused entry.
+    while (true) {
+      if (((curr = key.get(pos = (pos + 1) & mask)) == 0)) {
+        return defRetValue;
+      }
+      if (((k) == (curr))) {
+        return value.get(pos);
+      }
+    }
+  }
+
+  @Override
+
+  public boolean containsKey(final long k) {
+    if (((k) == 0)) {
+      return containsNullKey;
+    }
+    long curr;
+    long pos;
+    // The starting point.
+    if (((curr = key.get(pos = HashCommon.mix((k)) & mask)) == 0)) {
+      return false;
+    }
+    if (((k) == (curr))) {
+      return true;
+    }
+    // There's always an unused entry.
+    while (true) {
+      if (((curr = key.get(pos = (pos + 1) & mask)) == 0)) {
+        return false;
+      }
+      if (((k) == (curr))) {
+        return true;
+      }
+    }
+  }
+
+  @Override
+  public boolean containsValue(final long v) {
+    if (containsNullKey && ((value.get(n)) == (v))) {
+      return true;
+    }
+    for (long i = n; i-- != 0;) {
+      if (!((key.get(i)) == 0) && ((value.get(i)) == (v))) {
+        return true;
+      }
+    }
+    return false;
+  }
+
+  /**
+   * {@inheritDoc}
+   */
+  @Override
+
+  public long getOrDefault(final long k, final long defaultValue) {
+    if (((k) == 0)) {
+      return containsNullKey ? value.get(n) : defaultValue;
+    }
+    long curr;
+    long pos;
+    // The starting point.
+    if (((curr = key.get(pos = HashCommon.mix((k)) & mask)) == 0)) {
+      return defaultValue;
+    }
+    if (((k) == (curr))) {
+      return value.get(pos);
+    }
+    // There's always an unused entry.
+    while (true) {
+      if (((curr = key.get(pos = (pos + 1) & mask)) == 0)) {
+        return defaultValue;
+      }
+      if (((k) == (curr))) {
+        return value.get(pos);
+      }
+    }
+  }
+
+  /**
+   * {@inheritDoc}
+   */
+  @Override
+  public long putIfAbsent(final long k, final long v) {
+    final long pos = find(k);
+    if (pos >= 0) {
+      return value.get(pos);
+    }
+    insert(-pos - 1, k, v);
+    return defRetValue;
+  }
+
+  /**
+   * {@inheritDoc}
+   */
+  @Override
+
+  public boolean remove(final long k, final long v) {
+    if (((k) == 0)) {
+      if (containsNullKey && ((v) == (value.get(n)))) {
+        removeNullEntry();
+        return true;
+      }
+      return false;
+    }
+    long curr;
+    long pos;
+    // The starting point.
+    if (((curr = key.get(pos = HashCommon.mix((k)) & mask)) == 0)) {
+      return false;
+    }
+    if (((k) == (curr)) && ((v) == (value.get(pos)))) {
+      removeEntry(pos);
+      return true;
+    }
+    while (true) {
+      if (((curr = key.get(pos = (pos + 1) & mask)) == 0)) {
+        return false;
+      }
+      if (((k) == (curr)) && ((v) == (value.get(pos)))) {
+        removeEntry(pos);
+        return true;
+      }
+    }
+  }
+
+  /**
+   * {@inheritDoc}
+   */
+  @Override
+  public boolean replace(final long k, final long oldValue, final long v) {
+    final long pos = find(k);
+    if (pos < 0 || !(oldValue == (value.get(pos)))) {
+      return false;
+    }
+    value.set(pos, v);
+    return true;
+  }
+
+  /**
+   * {@inheritDoc}
+   */
+  @Override
+  public long replace(final long k, final long v) {
+    final long pos = find(k);
+    if (pos < 0) {
+      return defRetValue;
+    }
+    final long oldValue = value.get(pos);
+    value.set(pos, v);
+    return oldValue;
+  }
+
+  /**
+   * {@inheritDoc}
+   */
+  @Override
+  public long computeIfAbsent(final long k,
+      final java.util.function.LongUnaryOperator mappingFunction) {
+    java.util.Objects.requireNonNull(mappingFunction);
+    final long pos = find(k);
+    if (pos >= 0) {
+      return value.get(pos);
+    }
+    final long newValue = mappingFunction.applyAsLong(k);
+    insert(-pos - 1, k, newValue);
+    return newValue;
+  }
+
+  /**
+   * {@inheritDoc}
+   */
+  @Override
+  public long computeIfAbsent(final long key, final Long2LongFunction mappingFunction) {
+    java.util.Objects.requireNonNull(mappingFunction);
+    final long pos = find(key);
+    if (pos >= 0) {
+      return value.get(pos);
+    }
+    if (!mappingFunction.containsKey(key)) {
+      return defRetValue;
+    }
+    final long newValue = mappingFunction.get(key);
+    insert(-pos - 1, key, newValue);
+    return newValue;
+  }
+
+  /**
+   * {@inheritDoc}
+   */
+  @Override
+  public long computeIfAbsentNullable(final long k,
+      final java.util.function.LongFunction<? extends Long> mappingFunction) {
+    java.util.Objects.requireNonNull(mappingFunction);
+    final long pos = find(k);
+    if (pos >= 0) {
+      return value.get(pos);
+    }
+    final Long newValue = mappingFunction.apply(k);
+    if (newValue == null) {
+      return defRetValue;
+    }
+    final long v = newValue;
+    insert(-pos - 1, k, v);
+    return v;
+  }
+
+  /**
+   * {@inheritDoc}
+   */
+  @Override
+  public long computeIfPresent(
+      final long k,
+      final java.util.function.BiFunction<? super Long, ? super Long, ? extends Long> remappingFunction) {
+    java.util.Objects.requireNonNull(remappingFunction);
+    final long pos = find(k);
+    if (pos < 0) {
+      return defRetValue;
+    }
+    final Long newValue = remappingFunction.apply(k, value.get(pos));
+    if (newValue == null) {
+      if (((k) == 0)) {
+        removeNullEntry();
+      } else {
+        removeEntry(pos);
+      }
+      return defRetValue;
+    }
+    long newVal = newValue;
+    value.set(pos, newVal);
+    return newVal;
+  }
+
+  /**
+   * {@inheritDoc}
+   */
+  @Override
+  public long compute(
+      final long k,
+      final java.util.function.BiFunction<? super Long, ? super Long, ? extends Long> remappingFunction) {
+    java.util.Objects.requireNonNull(remappingFunction);
+    final long pos = find(k);
+    final Long newValue =
+        remappingFunction.apply(k, pos >= 0 ? value.get(pos) : null);
+    if (newValue == null) {
+      if (pos >= 0) {
+        if (((k) == 0)) {
+          removeNullEntry();
+        } else {
+          removeEntry(pos);
+        }
+      }
+      return defRetValue;
+    }
+    long newVal = newValue;
+    if (pos < 0) {
+      insert(-pos - 1, k, newVal);
+      return newVal;
+    }
+    value.set(pos, newVal);
+    return newVal;
+  }
+
+  /**
+   * {@inheritDoc}
+   */
+  @Override
+  public long merge(final long k, final long v,
+      final java.util.function.BiFunction<? super Long, ? super Long, ? extends Long> remappingFunction) {
+    java.util.Objects.requireNonNull(remappingFunction);
+    final long pos = find(k);
+    if (pos < 0) {
+      insert(-pos - 1, k, v);
+      return v;
+    }
+    final Long newValue = remappingFunction.apply(value.get(pos), v);
+    if (newValue == null) {
+      if (((k) == 0)) {
+        removeNullEntry();
+      } else {
+        removeEntry(pos);
+      }
+      return defRetValue;
+    }
+    long newVal = newValue;
+    value.set(pos, newVal);
+    return newVal;
+  }
+
+  /*
+   * Removes all elements from this map.
+   *
+   * <p>To increase object reuse, this method does not change the table size. If you want to reduce
+   * the table size, you must use {@link #trim()}.
+   *
+   */
+  @Override
+  public void clear() {
+    if (size.get() == 0) {
+      return;
+    }
+    size.set(0);
+    containsNullKey = false;
+  }
+
+  @Override
+  public boolean isEmpty() {
+    return size.get() == 0;
+  }
+
+  @Override
+  public long sizeAsLong() {
+    return size.get();
+  }
+
+  @Override
+  public int size() {
+    return (int) Math.min(sizeAsLong(), Integer.MAX_VALUE);
+  }
+
+  /**
+   * The entry class for a hash map does not record key and value, but rather the position in the
+   * hash table of the corresponding entry. This is necessary so that calls to
+   * {@link java.util.Map.Entry#setValue(Object)} are reflected in the map
+   */
+  final class MapEntry implements Long2LongMap.Entry, Map.Entry<Long, Long>, LongLongPair {
+    // The table index this entry refers to, or -1 if this entry has been deleted.
+    long index;
+
+    MapEntry(final long index) {
+      this.index = index;
+    }
+
+    MapEntry() {}
+
+    @Override
+    public long getLongKey() {
+      return key.get(index);
+    }
+
+    @Override
+    public long leftLong() {
+      return key.get(index);
+    }
+
+    @Override
+    public long getLongValue() {
+      return value.get(index);
+    }
+
+    @Override
+    public long rightLong() {
+      return value.get(index);
+    }
+
+    @Override
+    public long setValue(final long v) {
+      final long oldValue = value.get(index);
+      value.set(index, v);
+      return oldValue;
+    }
+
+    @Override
+    public LongLongPair right(final long v) {
+      value.set(index, v);
+      return this;
+    }
+
+    /**
+     * {@inheritDoc}
+     *
+     * @deprecated Please use the corresponding type-specific method instead.
+     */
+    @Deprecated
+    @Override
+    public Long getKey() {
+      return key.get(index);
+    }
+
+    /**
+     * {@inheritDoc}
+     *
+     * @deprecated Please use the corresponding type-specific method instead.
+     */
+    @Deprecated
+    @Override
+    public Long getValue() {
+      return value.get(index);
+    }
+
+    /**
+     * {@inheritDoc}
+     *
+     * @deprecated Please use the corresponding type-specific method instead.
+     */
+    @Deprecated
+    @Override
+    public Long setValue(final Long v) {
+      return setValue((v).longValue());
+    }
+
+    @Override
+    public boolean equals(final Object o) {
+      if (!(o instanceof Map.Entry)) {
+        return false;
+      }
+      Map.Entry<Long, Long> e = (Map.Entry<Long, Long>) o;
+      return ((key.get(index)) == ((e.getKey()).longValue()))
+          && ((value.get(index)) == ((e.getValue()).longValue()));
+    }
+
+    @Override
+    public int hashCode() {
+      return HashCommon.long2int(key.get(index)) ^ HashCommon.long2int(value.get(index));
+    }
+
+    @Override
+    public String toString() {
+      return key.get(index) + "=>" + value.get(index);
+    }
+  }
+
+  /**
+   * An iterator over a hash map.
+   */
+  private abstract class MapIterator<ConsumerType> {
+    /**
+     * The index of the last entry returned, if positive or zero; initially, {@link #n}. If
+     * negative, the last entry returned was that of the key of index {@code - pos - 1} from the
+     * {@link #wrapped} list.
+     */
+    long pos = n;
+    /**
+     * The index of the last entry that has been returned (more precisely, the value of {@link #pos}
+     * if {@link #pos} is positive, or {@link Integer#MIN_VALUE} if {@link #pos} is negative). It is
+     * -1 if either we did not return an entry yet, or the last returned entry has been removed.
+     */
+    long last = -1;
+    /**
+     * A downward counter measuring how many entries must still be returned.
+     */
+    long c = size.get();
+    /**
+     * A boolean telling us whether we should return the entry with the null key.
+     */
+    boolean mustReturnNullKey = Long2LongOpenHashDataMap.this.containsNullKey;
+    /**
+     * A lazily allocated list containing keys of entries that have wrapped around the table because
+     * of removals.
+     */
+    LongArrayList wrapped;
+
+    abstract void acceptOnIndex(final ConsumerType action, final long index);
+
+    public boolean hasNext() {
+      return c != 0;
+    }
+
+    public long nextEntry() {
+      if (!hasNext()) {
+        throw new NoSuchElementException();
+      }
+      c--;
+      if (mustReturnNullKey) {
+        mustReturnNullKey = false;
+        return last = n;
+      }
+      for (;;) {
+        if (--pos < 0) {
+          // We are just enumerating elements from the wrapped list.
+          last = Integer.MIN_VALUE;
+          final long k = wrapped.getLong((int) -pos - 1);
+          long p = (int) HashCommon.mix((k)) & mask;
+          while (!(k == (key.get(p)))) {
+            p = (p + 1) & mask;
+          }
+          return p;
+        }
+        if (!(key.get(pos) == 0)) {
+          return last = pos;
+        }
+      }
+    }
+
+    public void forEachRemaining(final ConsumerType action) {
+      if (mustReturnNullKey) {
+        mustReturnNullKey = false;
+        acceptOnIndex(action, last = n);
+        c--;
+      }
+      while (c != 0) {
+        if (--pos < 0) {
+          // We are just enumerating elements from the wrapped list.
+          last = Integer.MIN_VALUE;
+          final long k = wrapped.getLong((int) -pos - 1); // TODO: check if -pos - 1 is correct
+          long p = (int) HashCommon.mix((k)) & mask;
+          while (!(k == (key.get(p)))) {
+            p = (p + 1) & mask;
+          }
+          acceptOnIndex(action, p);
+          c--;
+        } else if (!(key.get(pos) == 0)) {
+          acceptOnIndex(action, last = pos);
+          c--;
+        }
+      }
+    }
+
+    /**
+     * Shifts left entries with the specified hash code, starting at the specified position, and
+     * empties the resulting free entry.
+     *
+     * @param pos a starting position.
+     */
+    private void shiftKeys(long pos) {
+      // Shift entries with the same hash.
+      long last, slot;
+      long curr;
+      for (;;) {
+        pos = ((last = pos) + 1) & mask;
+        for (;;) {
+          if (((curr = key.get(pos)) == 0)) {
+            key.set(last, 0L);
+            return;
+          }
+          slot = (int) HashCommon.mix((curr)) & mask;
+          if (last <= pos ? last >= slot || slot > pos : last >= slot && slot > pos) {
+            break;
+          }
+          pos = (pos + 1) & mask;
+        }
+        if (pos < last) { // Wrapped entry.
+          if (wrapped == null) {
+            wrapped = new LongArrayList(2);
+          }
+          wrapped.add(key.get(pos));

Review Comment:
   ## Deprecated method or constructor invocation
   
   Invoking [AbstractLongCollection.add](1) should be avoided because it has been deprecated.
   
   [Show more details](https://github.com/apache/incubator-baremaps/security/code-scanning/771)



##########
baremaps-core/src/main/java/org/apache/baremaps/database/collection/Long2ObjectOpenHashDataMap.java:
##########
@@ -0,0 +1,1468 @@
+/*
+ * Licensed 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.baremaps.database.collection;
+
+import static it.unimi.dsi.fastutil.HashCommon.*;
+
+import it.unimi.dsi.fastutil.Hash;
+import it.unimi.dsi.fastutil.HashCommon;
+import it.unimi.dsi.fastutil.longs.*;
+import it.unimi.dsi.fastutil.objects.AbstractObjectCollection;
+import it.unimi.dsi.fastutil.objects.AbstractObjectSet;
+import it.unimi.dsi.fastutil.objects.ObjectCollection;
+import it.unimi.dsi.fastutil.objects.ObjectIterator;
+import it.unimi.dsi.fastutil.objects.ObjectSpliterator;
+import it.unimi.dsi.fastutil.objects.ObjectSpliterators;
+import java.util.Map;
+import java.util.NoSuchElementException;
+import java.util.concurrent.atomic.AtomicLong;
+import java.util.function.Consumer;
+import java.util.function.Supplier;
+
+/**
+ * An open addressed hash map of long keys and long values derived from fastutil's
+ * {@link Long2ObjectOpenHashMap}. This implementation allows for the use of on-heap, off-heap, or
+ * memory mapped memory.
+ */
+public class Long2ObjectOpenHashDataMap<V> extends AbstractLong2ObjectMap<V>
+    implements DataMap<V>, Hash {
+
+  private final Supplier<DataMap<Long>> keySupplier;
+
+  private final Supplier<DataMap<V>> valueSupplier;
+
+  /**
+   * The array of keys.
+   */
+  protected DataMap<Long> key;
+  /**
+   * The array of values.
+   */
+  protected DataMap<V> value;
+  /**
+   * The mask for wrapping a position counter.
+   */
+  protected long mask;
+  /**
+   * Whether this map contains the key zero.
+   */
+  protected boolean containsNullKey;
+  /**
+   * The current table size.
+   */
+  protected long n;
+  /**
+   * Threshold after which we rehash. It must be the table size times {@link #f}.
+   */
+  protected long maxFill;
+  /**
+   * We never resize below this threshold, which is the construction-time {#n}.
+   */
+  protected final long minN;
+  /**
+   * Number of entries in the set (including the key zero, if present).
+   */
+  protected AtomicLong size = new AtomicLong();;
+  /**
+   * The acceptable load factor.
+   */
+  protected final float f;
+  /**
+   * Cached set of entries.
+   */
+  protected FastEntrySet<V> entries;
+  /**
+   * Cached set of keys.
+   */
+  protected LongSet keys;
+  /**
+   * Cached collection of values.
+   */
+  protected ObjectCollection<V> values;
+
+  /**
+   * Creates a new hash map.
+   *
+   * <p>
+   * The actual table size will be the least power of two greater than {@code expected}/{@code f}.
+   *
+   * @param expected the expected number of elements in the hash map.
+   * @param f the load factor.
+   */
+  public Long2ObjectOpenHashDataMap(
+      final long expected,
+      final float f,
+      final Supplier<DataMap<Long>> keySupplier,
+      final Supplier<DataMap<V>> valueSupplier) {
+    if (f <= 0 || f >= 1) {
+      throw new IllegalArgumentException("Load factor must be greater than 0 and smaller than 1");
+    }
+    if (expected < 0) {
+      throw new IllegalArgumentException("The expected number of elements must be nonnegative");
+    }
+    this.f = f;
+    this.minN = n = bigArraySize(expected, f);
+    this.mask = n - 1;
+    this.maxFill = maxFill(n, f);
+    this.keySupplier = keySupplier;
+    this.valueSupplier = valueSupplier;
+    this.key = keySupplier.get();
+    this.value = valueSupplier.get();
+  }
+
+  private long realSize() {
+    return containsNullKey ? size.get() - 1 : size.get();
+  }
+
+  private void ensureCapacity(final long capacity) {
+    final long needed = bigArraySize(capacity, f);
+    if (needed > n) {
+      rehash(needed);
+    }
+  }
+
+  private void tryCapacity(final long capacity) {
+    final long needed = Math.min(1 << 30,
+        Math.max(2, HashCommon.nextPowerOfTwo((long) Math.ceil(capacity / f))));
+    if (needed > n) {
+      rehash(needed);
+    }
+  }
+
+  private V removeEntry(final long pos) {
+    final V oldValue = value.get(pos);
+    value.put(pos, null);
+    size.decrementAndGet();
+    shiftKeys(pos);
+    if (n > minN && size.get() < maxFill / 4 && n > DEFAULT_INITIAL_SIZE) {
+      rehash(n / 2);
+    }
+    return oldValue;
+  }
+
+  private V removeNullEntry() {
+    containsNullKey = false;
+    final V oldValue = value.get(n);
+    value.put(n, null);
+    size.decrementAndGet();
+    if (n > minN && size.get() < maxFill / 4 && n > DEFAULT_INITIAL_SIZE) {
+      rehash(n / 2);
+    }
+    return oldValue;
+  }
+
+  @Override
+  public void putAll(Map<? extends Long, ? extends V> m) {
+    if (f <= .5) {
+      ensureCapacity(m.size()); // The resulting map will be sized for m.size() elements
+    } else {
+      tryCapacity(size() + m.size()); // The resulting map will be tentatively sized for size() +
+    }
+    // m.size()
+    // elements
+    super.putAll(m);
+  }
+
+  private long find(final long k) {
+    if (((k) == 0)) {
+      return containsNullKey ? n : -(n + 1);
+    }
+    long curr;
+    long pos;
+    // The starting point.
+    if (((curr = key.get(pos = HashCommon.mix((k)) & mask)) == 0)) {
+      return -(pos + 1);
+    }
+    if (((k) == (curr))) {
+      return pos;
+    }
+    // There's always an unused entry.
+    while (true) {
+      if (((curr = key.get(pos = (pos + 1) & mask)) == 0)) {
+        return -(pos + 1);
+      }
+      if (((k) == (curr))) {
+        return pos;
+      }
+    }
+  }
+
+  private void insert(final long pos, final long k, final V v) {
+    if (pos == n) {
+      containsNullKey = true;
+    }
+    key.put(pos, k);
+    value.put(pos, v);
+    if (size.getAndIncrement() >= maxFill) {
+      rehash(bigArraySize(size.get() + 1, f));
+    }
+  }
+
+  @Override
+  public V put(final long k, final V v) {
+    final long pos = find(k);
+    if (pos < 0) {
+      insert(-pos - 1, k, v);
+      return defRetValue;
+    }
+    final V oldValue = value.get(pos);
+    value.put(pos, v);
+    return oldValue;
+  }
+
+  /**
+   * Shifts left entries with the specified hash code, starting at the specified position, and
+   * empties the resulting free entry.
+   *
+   * @param pos a starting position.
+   */
+  protected final void shiftKeys(long pos) {
+    // Shift entries with the same hash.
+    long last, slot;
+    long curr;
+    for (;;) {
+      pos = ((last = pos) + 1) & mask;
+      for (;;) {
+        if (((curr = key.get(pos)) == 0)) {
+          key.put(last, 0L);
+          value.put(last, null);
+          return;
+        }
+        slot = HashCommon.mix((curr)) & mask;
+        if (last <= pos ? last >= slot || slot > pos : last >= slot && slot > pos) {
+          break;
+        }
+        pos = (pos + 1) & mask;
+      }
+      key.put(last, curr);
+      value.put(last, value.get(pos));
+    }
+  }
+
+  @Override
+
+  public V remove(final long k) {
+    if (((k) == 0)) {
+      if (containsNullKey) {
+        return removeNullEntry();
+      }
+      return defRetValue;
+    }
+    long curr;
+    long pos;
+    // The starting point.
+    if (((curr = key.get(pos = HashCommon.mix((k)) & mask)) == 0)) {
+      return defRetValue;
+    }
+    if (((k) == (curr))) {
+      return removeEntry(pos);
+    }
+    while (true) {
+      if (((curr = key.get(pos = (pos + 1) & mask)) == 0)) {
+        return defRetValue;
+      }
+      if (((k) == (curr))) {
+        return removeEntry(pos);
+      }
+    }
+  }
+
+  @Override
+
+  public V get(final long k) {
+    if (((k) == 0)) {
+      return containsNullKey ? value.get(n) : defRetValue;
+    }
+    long curr;
+    long pos;
+    // The starting point.
+    if (((curr = key.get(pos = HashCommon.mix((k)) & mask)) == 0)) {
+      return defRetValue;
+    }
+    if (((k) == (curr))) {
+      return value.get(pos);
+    }
+    // There's always an unused entry.
+    while (true) {
+      if (((curr = key.get(pos = (pos + 1) & mask)) == 0)) {
+        return defRetValue;
+      }
+      if (((k) == (curr))) {
+        return value.get(pos);
+      }
+    }
+  }
+
+  @Override
+
+  public boolean containsKey(final long k) {
+    if (((k) == 0)) {
+      return containsNullKey;
+    }
+    long curr;
+    long pos;
+    // The starting point.
+    if (((curr = key.get(pos = HashCommon.mix((k)) & mask)) == 0)) {
+      return false;
+    }
+    if (((k) == (curr))) {
+      return true;
+    }
+    // There's always an unused entry.
+    while (true) {
+      if (((curr = key.get(pos = (pos + 1) & mask)) == 0)) {
+        return false;
+      }
+      if (((k) == (curr))) {
+        return true;
+      }
+    }
+  }
+
+  @Override
+  public boolean containsValue(final Object v) {
+    if (containsNullKey && java.util.Objects.equals(value.get(n), v)) {
+      return true;
+    }
+    for (long i = n; i-- != 0;) {
+      if (!((key.get(i)) == 0) && java.util.Objects.equals(value.get(i), v)) {
+        return true;
+      }
+    }
+    return false;
+  }
+
+  /**
+   * {@inheritDoc}
+   */
+  @Override
+
+  public V getOrDefault(final long k, final V defaultValue) {
+    if (((k) == 0)) {
+      return containsNullKey ? value.get(n) : defaultValue;
+    }
+    long curr;
+    long pos;
+    // The starting point.
+    if (((curr = key.get(pos = HashCommon.mix((k)) & mask)) == 0)) {
+      return defaultValue;
+    }
+    if (((k) == (curr))) {
+      return value.get(pos);
+    }
+    // There's always an unused entry.
+    while (true) {
+      if (((curr = key.get(pos = (pos + 1) & mask)) == 0)) {
+        return defaultValue;
+      }
+      if (((k) == (curr))) {
+        return value.get(pos);
+      }
+    }
+  }
+
+  /**
+   * {@inheritDoc}
+   */
+  @Override
+  public V putIfAbsent(final long k, final V v) {
+    final long pos = find(k);
+    if (pos >= 0) {
+      return value.get(pos);
+    }
+    insert(-pos - 1, k, v);
+    return defRetValue;
+  }
+
+  /**
+   * {@inheritDoc}
+   */
+  @Override
+
+  public boolean remove(final long k, final Object v) {
+    if (((k) == 0)) {
+      if (containsNullKey && java.util.Objects.equals(v, value.get(n))) {
+        removeNullEntry();
+        return true;
+      }
+      return false;
+    }
+    long curr;
+    long pos;
+    // The starting point.
+    if (((curr = key.get(pos = HashCommon.mix((k)) & mask)) == 0)) {
+      return false;
+    }
+    if (((k) == (curr)) && java.util.Objects.equals(v, value.get(pos))) {
+      removeEntry(pos);
+      return true;
+    }
+    while (true) {
+      if (((curr = key.get(pos = (pos + 1) & mask)) == 0)) {
+        return false;
+      }
+      if (((k) == (curr)) && java.util.Objects.equals(v, value.get(pos))) {
+        removeEntry(pos);
+        return true;
+      }
+    }
+  }
+
+  /**
+   * {@inheritDoc}
+   */
+  @Override
+  public boolean replace(final long k, final V oldValue, final V v) {
+    final long pos = find(k);
+    if (pos < 0 || !java.util.Objects.equals(oldValue, value.get(pos))) {
+      return false;
+    }
+    value.put(pos, v);
+    return true;
+  }
+
+  /**
+   * {@inheritDoc}
+   */
+  @Override
+  public V replace(final long k, final V v) {
+    final long pos = find(k);
+    if (pos < 0) {
+      return defRetValue;
+    }
+    final V oldValue = value.get(pos);
+    value.put(pos, v);
+    return oldValue;
+  }
+
+  /**
+   * {@inheritDoc}
+   */
+  @Override
+  public V computeIfAbsent(final long k,
+      final java.util.function.LongFunction<? extends V> mappingFunction) {
+    java.util.Objects.requireNonNull(mappingFunction);
+    final long pos = find(k);
+    if (pos >= 0) {
+      return value.get(pos);
+    }
+    final V newValue = mappingFunction.apply(k);
+    insert(-pos - 1, k, newValue);
+    return newValue;
+  }
+
+  /**
+   * {@inheritDoc}
+   */
+  @Override
+  public V computeIfAbsent(final long key, final Long2ObjectFunction<? extends V> mappingFunction) {
+    java.util.Objects.requireNonNull(mappingFunction);
+    final long pos = find(key);
+    if (pos >= 0) {
+      return value.get(pos);
+    }
+    if (!mappingFunction.containsKey(key)) {
+      return defRetValue;
+    }
+    final V newValue = mappingFunction.get(key);
+    insert(-pos - 1, key, newValue);
+    return newValue;
+  }
+
+  /**
+   * {@inheritDoc}
+   */
+  @Override
+  public V computeIfPresent(final long k,
+      final java.util.function.BiFunction<? super Long, ? super V, ? extends V> remappingFunction) {
+    java.util.Objects.requireNonNull(remappingFunction);
+    final long pos = find(k);
+    if (pos < 0) {
+      return defRetValue;
+    }
+    if (value.get(pos) == null) {
+      return defRetValue;
+    }
+    final V newValue = remappingFunction.apply(k, (value.get(pos)));
+    if (newValue == null) {
+      if (((k) == 0)) {
+        removeNullEntry();
+      } else {
+        removeEntry(pos);
+      }
+      return defRetValue;
+    }
+    value.put(pos, newValue);
+    return newValue;
+  }
+
+  /**
+   * {@inheritDoc}
+   */
+  @Override
+  public V compute(final long k,
+      final java.util.function.BiFunction<? super Long, ? super V, ? extends V> remappingFunction) {
+    java.util.Objects.requireNonNull(remappingFunction);
+    final long pos = find(k);
+    final V newValue = remappingFunction.apply(k, pos >= 0 ? (value.get(pos)) : null);
+    if (newValue == null) {
+      if (pos >= 0) {
+        if (((k) == 0)) {
+          removeNullEntry();
+        } else {
+          removeEntry(pos);
+        }
+      }
+      return defRetValue;
+    }
+    if (pos < 0) {
+      insert(-pos - 1, k, (newValue));
+      return (newValue);
+    }
+    value.put(pos, (newValue));
+    return (newValue);
+  }
+
+  /**
+   * {@inheritDoc}
+   */
+  @Override
+  public V merge(final long k, final V v,
+      final java.util.function.BiFunction<? super V, ? super V, ? extends V> remappingFunction) {
+    java.util.Objects.requireNonNull(remappingFunction);
+    java.util.Objects.requireNonNull(v);
+    final long pos = find(k);
+    if (pos < 0 || value.get(pos) == null) {
+      if (pos < 0) {
+        insert(-pos - 1, k, v);
+      } else {
+        value.put(pos, v);
+      }
+      return v;
+    }
+    final V newValue = remappingFunction.apply((value.get(pos)), (v));
+    if (newValue == null) {
+      if (((k) == 0)) {
+        removeNullEntry();
+      } else {
+        removeEntry(pos);
+      }
+      return defRetValue;
+    }
+    value.put(pos, newValue);
+    return newValue;
+  }
+
+  /*
+   * Removes all elements from this map.
+   *
+   * <p>To increase object reuse, this method does not change the table size. If you want to reduce
+   * the table size, you must use {@link #trim()}.
+   *
+   */
+  @Override
+  public void clear() {
+    if (size.get() == 0) {
+      return;
+    }
+    size.set(0);
+    containsNullKey = false;
+    // TODO: Arrays.fill(key, 0);
+    // TODO: Arrays.fill(value, null);
+  }
+
+  @Override
+  public long sizeAsLong() {
+    return size.get();
+  }
+
+  @Override
+  public int size() {
+    return (int) Math.min(sizeAsLong(), Integer.MAX_VALUE);
+  }
+
+  @Override
+  public boolean isEmpty() {
+    return sizeAsLong() == 0;
+  }
+
+  /**
+   * The entry class for a hash map does not record key and value, but rather the position in the
+   * hash table of the corresponding entry. This is necessary so that calls to
+   * {@link java.util.Map.Entry#setValue(Object)} are reflected in the map
+   */
+  final class MapEntry implements Long2ObjectMap.Entry<V>, Map.Entry<Long, V>, LongObjectPair<V> {
+    // The table index this entry refers to, or -1 if this entry has been deleted.
+    long index;
+
+    MapEntry(final long index) {
+      this.index = index;
+    }
+
+    MapEntry() {}
+
+    @Override
+    public long getLongKey() {
+      return key.get(index);
+    }
+
+    @Override
+    public long leftLong() {
+      return key.get(index);
+    }
+
+    @Override
+    public V getValue() {
+      return value.get(index);
+    }
+
+    @Override
+    public V right() {
+      return value.get(index);
+    }
+
+    @Override
+    public V setValue(final V v) {
+      final V oldValue = value.get(index);
+      value.put(index, v);
+      return oldValue;
+    }
+
+    @Override
+    public LongObjectPair<V> right(final V v) {
+      value.put(index, v);
+      return this;
+    }
+
+    /**
+     * {@inheritDoc}
+     *
+     * @deprecated Please use the corresponding type-specific method instead.
+     */
+    @Deprecated
+    @Override
+    public Long getKey() {
+      return Long.valueOf(key.get(index));
+    }
+
+    @Override
+    public boolean equals(final Object o) {
+      if (!(o instanceof Map.Entry)) {
+        return false;
+      }
+      Map.Entry<Long, V> e = (Map.Entry<Long, V>) o;
+      return ((key.get(index)) == ((e.getKey()).longValue()))
+          && java.util.Objects.equals(value.get(index), (e.getValue()));
+    }
+
+    @Override
+    public int hashCode() {
+      return HashCommon.long2int(key.get(index))
+          ^ ((value.get(index)) == null ? 0 : (value.get(index)).hashCode());
+    }
+
+    @Override
+    public String toString() {
+      return key.get(index) + "=>" + value.get(index);
+    }
+  }
+
+  /**
+   * An iterator over a hash map.
+   */
+  private abstract class MapIterator<ConsumerType> {
+    /**
+     * The index of the last entry returned, if positive or zero; initially, {@link #n}. If
+     * negative, the last entry returned was that of the key of index {@code - pos - 1} from the
+     * {@link #wrapped} list.
+     */
+    long pos = n;
+    /**
+     * The index of the last entry that has been returned (more precisely, the value of {@link #pos}
+     * if {@link #pos} is positive, or {@link Integer#MIN_VALUE} if {@link #pos} is negative). It is
+     * -1 if either we did not return an entry yet, or the last returned entry has been removed.
+     */
+    long last = -1;
+    /**
+     * A downward counter measuring how many entries must still be returned.
+     */
+    long c = size.get();
+    /**
+     * A boolean telling us whether we should return the entry with the null key.
+     */
+    boolean mustReturnNullKey = Long2ObjectOpenHashDataMap.this.containsNullKey;
+    /**
+     * A lazily allocated list containing keys of entries that have wrapped around the table because
+     * of removals.
+     */
+    LongArrayList wrapped;
+
+    abstract void acceptOnIndex(final ConsumerType action, final long index);
+
+    public boolean hasNext() {
+      return c != 0;
+    }
+
+    public long nextEntry() {
+      if (!hasNext()) {
+        throw new NoSuchElementException();
+      }
+      c--;
+      if (mustReturnNullKey) {
+        mustReturnNullKey = false;
+        return last = n;
+      }
+      for (;;) {
+        if (--pos < 0) {
+          // We are just enumerating elements from the wrapped list.
+          last = Integer.MIN_VALUE;
+          final long k = wrapped.getLong((int) -pos - 1); // TODO: check if -pos - 1 is correct
+          long p = HashCommon.mix((k)) & mask;
+          while (!((k) == (key.get(p)))) {
+            p = (p + 1) & mask;
+          }
+          return p;
+        }
+        if (!((key.get(pos)) == 0)) {
+          return last = pos;
+        }
+      }
+    }
+
+    public void forEachRemaining(final ConsumerType action) {
+      if (mustReturnNullKey) {
+        mustReturnNullKey = false;
+        acceptOnIndex(action, last = n);
+        c--;
+      }
+      while (c != 0) {
+        if (--pos < 0) {
+          // We are just enumerating elements from the wrapped list.
+          last = Integer.MIN_VALUE;
+          final long k = wrapped.getLong((int) -pos - 1); // TODO: check if -pos - 1 is correct
+          long p = HashCommon.mix((k)) & mask;
+          while (!((k) == (key.get(p)))) {
+            p = (p + 1) & mask;
+          }
+          acceptOnIndex(action, p);
+          c--;
+        } else if (!((key.get(pos)) == 0)) {
+          acceptOnIndex(action, last = pos);
+          c--;
+        }
+      }
+    }
+
+    /**
+     * Shifts left entries with the specified hash code, starting at the specified position, and
+     * empties the resulting free entry.
+     *
+     * @param pos a starting position.
+     */
+    private void shiftKeys(long pos) {
+      // Shift entries with the same hash.
+      long last, slot;
+      long curr;
+      for (;;) {
+        pos = ((last = pos) + 1) & mask;
+        for (;;) {
+          if (((curr = key.get(pos)) == 0)) {
+            key.put(last, 0L);
+            value.put(last, null);
+            return;
+          }
+          slot = HashCommon.mix((curr)) & mask;
+          if (last <= pos ? last >= slot || slot > pos : last >= slot && slot > pos) {
+            break;
+          }
+          pos = (pos + 1) & mask;
+        }
+        if (pos < last) { // Wrapped entry.
+          if (wrapped == null) {
+            wrapped = new LongArrayList(2);
+          }
+          wrapped.add(key.get(pos));

Review Comment:
   ## Deprecated method or constructor invocation
   
   Invoking [AbstractLongCollection.add](1) should be avoided because it has been deprecated.
   
   [Show more details](https://github.com/apache/incubator-baremaps/security/code-scanning/772)



##########
baremaps-core/src/main/java/org/apache/baremaps/database/collection/Long2LongPackedOpenHashDataMap.java:
##########
@@ -0,0 +1,1555 @@
+/*
+ * Licensed 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.baremaps.database.collection;
+
+import static it.unimi.dsi.fastutil.HashCommon.bigArraySize;
+import static it.unimi.dsi.fastutil.HashCommon.maxFill;
+
+import it.unimi.dsi.fastutil.Hash;
+import it.unimi.dsi.fastutil.HashCommon;
+import it.unimi.dsi.fastutil.longs.*;
+import it.unimi.dsi.fastutil.objects.*;
+import java.util.Map;
+import java.util.NoSuchElementException;
+import java.util.concurrent.atomic.AtomicLong;
+import java.util.function.Consumer;
+import java.util.function.LongConsumer;
+import java.util.function.Supplier;
+import org.apache.baremaps.database.type.PairDataType.Pair;
+
+/**
+ * An open addressed hash map of long keys and long values derived from fastutil's
+ * {@link Long2LongOpenHashMap}. This implementation allows for the use of on-heap, off-heap, or
+ * memory mapped memory. The keys and values are packed in the same memory.
+ */
+public class Long2LongPackedOpenHashDataMap extends AbstractLong2LongMap
+    implements DataMap<Long>, Hash {
+
+  private final Supplier<DataMap<Pair<Long, Long>>> indexSupplier;
+
+  /**
+   * The hash index.
+   */
+  protected DataMap<Pair<Long, Long>> index;
+  /**
+   * The mask for wrapping a position counter.
+   */
+  protected long mask;
+  /**
+   * Whether this map contains the key zero.
+   */
+  protected boolean containsNullKey;
+  /**
+   * The current table size.
+   */
+  protected long n;
+  /**
+   * Threshold after which we rehash. It must be the table size times {@link #f}.
+   */
+  protected long maxFill;
+  /**
+   * We never resize below this threshold, which is the construction-time {#n}.
+   */
+  protected final long minN;
+  /**
+   * Number of entries in the set (including the key zero, if present).
+   */
+  protected AtomicLong size = new AtomicLong();
+  /**
+   * The acceptable load factor.
+   */
+  protected final float f;
+  /**
+   * Cached set of entries.
+   */
+  protected FastEntrySet entries;
+  /**
+   * Cached set of keys.
+   */
+  protected LongSet keys;
+  /**
+   * Cached collection of values.
+   */
+  protected LongCollection values;
+
+  /**
+   * Creates a new hash map.
+   *
+   * <p>
+   * The actual table size will be the least power of two greater than {@code expected}/{@code f}.
+   *
+   * @param expected the expected number of elements in the hash map.
+   * @param f the load factor.
+   */
+  public Long2LongPackedOpenHashDataMap(final long expected, final float f,
+      final Supplier<DataMap<Pair<Long, Long>>> indexSupplier) {
+    if (f <= 0 || f >= 1) {
+      throw new IllegalArgumentException("Load factor must be greater than 0 and smaller than 1");
+    }
+    if (expected < 0) {
+      throw new IllegalArgumentException("The expected number of elements must be nonnegative");
+    }
+    this.f = f;
+    this.minN = n = bigArraySize(expected, f);
+    this.mask = n - 1;
+    this.maxFill = maxFill(n, f);
+    this.indexSupplier = indexSupplier;
+    this.index = indexSupplier.get();
+  }
+
+  private long realSize() {
+    return containsNullKey ? size.get() - 1 : size.get();
+  }
+
+  private void ensureCapacity(final long capacity) {
+    final long needed = bigArraySize(capacity, f);
+    if (needed > n) {
+      rehash(needed);
+    }
+  }
+
+  private void tryCapacity(final long capacity) {
+    final long needed =
+        Math.min(1 << 30, Math.max(2, HashCommon.nextPowerOfTwo((long) Math.ceil(capacity / f))));
+    if (needed > n) {
+      rehash(needed);
+    }
+  }
+
+  private long removeEntry(final long pos) {
+    final long oldValue = index.get(pos).right();
+    size.decrementAndGet();
+    shiftKeys(pos);
+    if (n > minN && size.get() < maxFill / 4 && n > DEFAULT_INITIAL_SIZE) {
+      rehash(n / 2);
+    }
+    return oldValue;
+  }
+
+  private long removeNullEntry() {
+    containsNullKey = false;
+    final long oldValue = index.get(n).right();
+    size.decrementAndGet();
+    if (n > minN && size.get() < maxFill / 4 && n > DEFAULT_INITIAL_SIZE) {
+      rehash(n / 2);
+    }
+    return oldValue;
+  }
+
+  @Override
+  public void putAll(Map<? extends Long, ? extends Long> m) {
+    if (f <= .5) {
+      ensureCapacity(m.size());
+    } else {
+      tryCapacity(size() + m.size());
+    }
+    super.putAll(m);
+  }
+
+  private long find(final long k) {
+    if (((k) == 0)) {
+      return containsNullKey ? n : -(n + 1);
+    }
+    long curr;
+    long pos;
+    if (((curr = index.get(pos = HashCommon.mix((k)) & mask).left()) == 0)) {
+      return -(pos + 1);
+    }
+    if (((k) == (curr))) {
+      return pos;
+    }
+    while (true) {
+      if (((curr = index.get(pos = (pos + 1) & mask).left()) == 0)) {
+        return -(pos + 1);
+      }
+      if (((k) == (curr))) {
+        return pos;
+      }
+    }
+  }
+
+  private void insert(final long pos, final long k, final long v) {
+    if (pos == n) {
+      containsNullKey = true;
+    }
+    index.put(pos, new Pair<>(k, v));
+    if (size.getAndIncrement() >= maxFill) {
+      rehash(bigArraySize(size.get() + 1, f));
+    }
+  }
+
+  @Override
+  public long put(final long k, final long v) {
+    final long pos = find(k);
+    if (pos < 0) {
+      insert(-pos - 1, k, v);
+      return defRetValue;
+    }
+    final long oldValue = index.get(pos).right();
+    index.put(pos, new Pair<>(pos, v));
+    return oldValue;
+  }
+
+  private long addToValue(final long pos, final long incr) {
+    final long oldValue = index.get(pos).right();
+    index.put(pos, new Pair<>(pos, oldValue + incr));
+    return oldValue;
+  }
+
+  /**
+   * Adds an increment to value currently associated with a key.
+   *
+   * <p>
+   * Note that this method respects the {@linkplain #defaultReturnValue() default return value}
+   * semantics: when called with a key that does not currently appears in the map, the key will be
+   * associated with the default return value plus the given increment.
+   *
+   * @param k the key.
+   * @param incr the increment.
+   * @return the old value, or the {@linkplain #defaultReturnValue() default return value} if no
+   *         value was present for the given key.
+   */
+  public long addTo(final long k, final long incr) {
+    long pos;
+    if (((k) == 0)) {
+      if (containsNullKey) {
+        return addToValue(n, incr);
+      }
+      pos = n;
+      containsNullKey = true;
+    } else {
+      long curr;
+      // The starting point.
+      if (!((curr = index.get(pos = HashCommon.mix((k)) & mask).left()) == 0)) {
+        if ((curr == k)) {
+          return addToValue(pos, incr);
+        }
+        while (!((curr = index.get(pos = (pos + 1) & mask).left()) == 0)) {
+          if ((curr == k)) {
+            return addToValue(pos, incr);
+          }
+        }
+      }
+    }
+    index.put(pos, new Pair<>(k, defRetValue + incr));
+    if (size.incrementAndGet() >= maxFill) {
+      rehash(bigArraySize(size.get() + 1, f));
+    }
+    return defRetValue;
+  }
+
+  /**
+   * Shifts left entries with the specified hash code, starting at the specified position, and
+   * empties the resulting free entry.
+   *
+   * @param pos a starting position.
+   */
+  protected final void shiftKeys(long pos) {
+    // Shift entries with the same hash.
+    long last, slot;
+    long curr;
+    for (;;) {
+      pos = ((last = pos) + 1) & mask;
+      for (;;) {
+        if (((curr = index.get(pos).left()) == 0)) {
+          index.put(last, new Pair<>(curr, 0L));
+          return;
+        }
+        slot = HashCommon.mix((curr)) & mask;
+        if (last <= pos ? last >= slot || slot > pos : last >= slot && slot > pos) {
+          break;
+        }
+        pos = (pos + 1) & mask;
+      }
+      index.put(last, new Pair<>(curr, index.get(pos).right()));
+    }
+  }
+
+  @Override
+
+  public long remove(final long k) {
+    if (((k) == 0)) {
+      if (containsNullKey) {
+        return removeNullEntry();
+      }
+      return defRetValue;
+    }
+    long curr;
+    long pos;
+    // The starting point.
+    if (((curr = index.get(pos = HashCommon.mix((k)) & mask).left()) == 0)) {
+      return defRetValue;
+    }
+    if (((k) == (curr))) {
+      return removeEntry(pos);
+    }
+    while (true) {
+      if (((curr = index.get(pos = (pos + 1) & mask).left()) == 0)) {
+        return defRetValue;
+      }
+      if (((k) == (curr))) {
+        return removeEntry(pos);
+      }
+    }
+  }
+
+  @Override
+
+  public long get(final long k) {
+    if (((k) == 0)) {
+      return containsNullKey ? index.get(n).right() : defRetValue;
+    }
+    long curr;
+    long pos;
+    // The starting point.
+    if (((curr = index.get(pos = HashCommon.mix((k)) & mask).left()) == 0)) {
+      return defRetValue;
+    }
+    if (((k) == (curr))) {
+      return index.get(pos).right();
+    }
+    // There's always an unused entry.
+    while (true) {
+      if (((curr = index.get(pos = (pos + 1) & mask).left()) == 0)) {
+        return defRetValue;
+      }
+      if (((k) == (curr))) {
+        return index.get(pos).right();
+      }
+    }
+  }
+
+  @Override
+
+  public boolean containsKey(final long k) {
+    if (((k) == 0)) {
+      return containsNullKey;
+    }
+    long curr;
+    long pos;
+    // The starting point.
+    if (((curr = index.get(pos = HashCommon.mix((k)) & mask).left()) == 0)) {
+      return false;
+    }
+    if (((k) == (curr))) {
+      return true;
+    }
+    // There's always an unused entry.
+    while (true) {
+      if (((curr = index.get(pos = (pos + 1) & mask).left()) == 0)) {
+        return false;
+      }
+      if (((k) == (curr))) {
+        return true;
+      }
+    }
+  }
+
+  @Override
+  public boolean containsValue(final long v) {
+    if (containsNullKey && ((index.get(n).right()) == (v))) {
+      return true;
+    }
+    for (long i = n; i-- != 0;) {
+      if (!((index.get(i).left()) == 0) && ((index.get(i).right()) == (v))) {
+        return true;
+      }
+    }
+    return false;
+  }
+
+  /**
+   * {@inheritDoc}
+   */
+  @Override
+
+  public long getOrDefault(final long k, final long defaultValue) {
+    if (((k) == 0)) {
+      return containsNullKey ? index.get(n).right() : defaultValue;
+    }
+    long curr;
+    long pos;
+    // The starting point.
+    if (((curr = index.get(pos = HashCommon.mix((k)) & mask).left()) == 0)) {
+      return defaultValue;
+    }
+    if (((k) == (curr))) {
+      return index.get(pos).right();
+    }
+    // There's always an unused entry.
+    while (true) {
+      if (((curr = index.get(pos = (pos + 1) & mask).left()) == 0)) {
+        return defaultValue;
+      }
+      if (((k) == (curr))) {
+        return index.get(pos).right();
+      }
+    }
+  }
+
+  /**
+   * {@inheritDoc}
+   */
+  @Override
+  public long putIfAbsent(final long k, final long v) {
+    final long pos = find(k);
+    if (pos >= 0) {
+      return index.get(pos).right();
+    }
+    insert(-pos - 1, k, v);
+    return defRetValue;
+  }
+
+  /**
+   * {@inheritDoc}
+   */
+  @Override
+
+  public boolean remove(final long k, final long v) {
+    if (((k) == 0)) {
+      if (containsNullKey && ((v) == (index.get(n).right()))) {
+        removeNullEntry();
+        return true;
+      }
+      return false;
+    }
+    long curr;
+    long pos;
+    // The starting point.
+    if (((curr = index.get(pos = HashCommon.mix((k)) & mask).left()) == 0)) {
+      return false;
+    }
+    if (((k) == (curr)) && ((v) == (index.get(pos).right()))) {
+      removeEntry(pos);
+      return true;
+    }
+    while (true) {
+      if (((curr = index.get(pos = (pos + 1) & mask).left()) == 0)) {
+        return false;
+      }
+      if (((k) == (curr)) && ((v) == (index.get(pos).right()))) {
+        removeEntry(pos);
+        return true;
+      }
+    }
+  }
+
+  /**
+   * {@inheritDoc}
+   */
+  @Override
+  public boolean replace(final long k, final long oldValue, final long v) {
+    final long pos = find(k);
+    if (pos < 0 || !((oldValue) == (index.get(pos).right()))) {
+      return false;
+    }
+    index.put(pos, new Pair<>(k, v));
+    return true;
+  }
+
+  /**
+   * {@inheritDoc}
+   */
+  @Override
+  public long replace(final long k, final long v) {
+    final long pos = find(k);
+    if (pos < 0) {
+      return defRetValue;
+    }
+    final long oldValue = index.get(pos).right();
+    index.put(pos, new Pair<>(k, v));
+    return oldValue;
+  }
+
+  /**
+   * {@inheritDoc}
+   */
+  @Override
+  public long computeIfAbsent(final long k,
+      final java.util.function.LongUnaryOperator mappingFunction) {
+    java.util.Objects.requireNonNull(mappingFunction);
+    final long pos = find(k);
+    if (pos >= 0) {
+      return index.get(pos).right();
+    }
+    final long newValue = mappingFunction.applyAsLong(k);
+    insert(-pos - 1, k, newValue);
+    return newValue;
+  }
+
+  /**
+   * {@inheritDoc}
+   */
+  @Override
+  public long computeIfAbsent(final long key, final Long2LongFunction mappingFunction) {
+    java.util.Objects.requireNonNull(mappingFunction);
+    final long pos = find(key);
+    if (pos >= 0) {
+      return index.get(pos).right();
+    }
+    if (!mappingFunction.containsKey(key)) {
+      return defRetValue;
+    }
+    final long newValue = mappingFunction.get(key);
+    insert(-pos - 1, key, newValue);
+    return newValue;
+  }
+
+  /**
+   * {@inheritDoc}
+   */
+  @Override
+  public long computeIfAbsentNullable(final long k,
+      final java.util.function.LongFunction<? extends Long> mappingFunction) {
+    java.util.Objects.requireNonNull(mappingFunction);
+    final long pos = find(k);
+    if (pos >= 0) {
+      return index.get(pos).right();
+    }
+    final Long newValue = mappingFunction.apply(k);
+    if (newValue == null) {
+      return defRetValue;
+    }
+    final long v = newValue;
+    insert(-pos - 1, k, v);
+    return v;
+  }
+
+  /**
+   * {@inheritDoc}
+   */
+  @Override
+  public long computeIfPresent(final long k,
+      final java.util.function.BiFunction<? super Long, ? super Long, ? extends Long> remappingFunction) {
+    java.util.Objects.requireNonNull(remappingFunction);
+    final long pos = find(k);
+    if (pos < 0) {
+      return defRetValue;
+    }
+    final Long newValue =
+        remappingFunction.apply(k, index.get(pos).right());
+    if (newValue == null) {
+      if (((k) == 0)) {
+        removeNullEntry();
+      } else {
+        removeEntry(pos);
+      }
+      return defRetValue;
+    }
+    long newVal = newValue;
+    index.put(pos, new Pair<>(k, newVal));
+    return newVal;
+  }
+
+  /**
+   * {@inheritDoc}
+   */
+  @Override
+  public long compute(final long k,
+      final java.util.function.BiFunction<? super Long, ? super Long, ? extends Long> remappingFunction) {
+    java.util.Objects.requireNonNull(remappingFunction);
+    final long pos = find(k);
+    final Long newValue =
+        remappingFunction.apply(k, pos >= 0 ? index.get(pos).right() : null);
+    if (newValue == null) {
+      if (pos >= 0) {
+        if (((k) == 0)) {
+          removeNullEntry();
+        } else {
+          removeEntry(pos);
+        }
+      }
+      return defRetValue;
+    }
+    if (pos < 0) {
+      insert(-pos - 1, k, newValue);
+      return newValue;
+    }
+    index.put(pos, new Pair<>(k, newValue));
+    return newValue;
+  }
+
+  /**
+   * {@inheritDoc}
+   */
+  @Override
+  public long merge(final long k, final long v,
+      final java.util.function.BiFunction<? super Long, ? super Long, ? extends Long> remappingFunction) {
+    java.util.Objects.requireNonNull(remappingFunction);
+
+    final long pos = find(k);
+    if (pos < 0) {
+      insert(-pos - 1, k, v);
+      return v;
+    }
+    final Long newValue =
+        remappingFunction.apply(index.get(pos).right(), v);
+    if (newValue == null) {
+      if (((k) == 0)) {
+        removeNullEntry();
+      } else {
+        removeEntry(pos);
+      }
+      return defRetValue;
+    }
+    long newVal = newValue;
+    index.put(pos, new Pair<>(k, newVal));
+    return newVal;
+  }
+
+  /*
+   * Removes all elements from this map.
+   *
+   * <p>To increase object reuse, this method does not change the table size. If you want to reduce
+   * the table size, you must use {@link #trim()}.
+   *
+   */
+  @Override
+  public void clear() {
+    if (size.get() == 0) {
+      return;
+    }
+    size.set(0);
+    containsNullKey = false;
+    // TODO: Arrays.fill(key, 0);
+  }
+
+  @Override
+  public boolean isEmpty() {
+    return size.get() == 0;
+  }
+
+  @Override
+  public long sizeAsLong() {
+    return size.get();
+  }
+
+  @Override
+  public int size() {
+    return (int) Math.min(sizeAsLong(), Integer.MAX_VALUE);
+  }
+
+  /**
+   * The entry class for a hash map does not record key and value, but rather the position in the
+   * hash table of the corresponding entry. This is necessary so that calls to
+   * {@link Map.Entry#setValue(Object)} are reflected in the map
+   */
+  final class MapEntry implements Long2LongMap.Entry, Map.Entry<Long, Long>, LongLongPair {
+    // The table index this entry refers to, or -1 if this entry has been deleted.
+    long index;
+
+    MapEntry(final long index) {
+      this.index = index;
+    }
+
+    MapEntry() {}
+
+    @Override
+    public long getLongKey() {
+      return Long2LongPackedOpenHashDataMap.this.index.get(index).left();
+    }
+
+    @Override
+    public long leftLong() {
+      return Long2LongPackedOpenHashDataMap.this.index.get(index).left();
+    }
+
+    @Override
+    public long getLongValue() {
+      return Long2LongPackedOpenHashDataMap.this.index.get(index).right();
+    }
+
+    @Override
+    public long rightLong() {
+      return Long2LongPackedOpenHashDataMap.this.index.get(index).right();
+    }
+
+    @Override
+    public long setValue(final long v) {
+      final long oldValue = Long2LongPackedOpenHashDataMap.this.index.get(index).right();
+      Long2LongPackedOpenHashDataMap.this.index.put(index,
+          new Pair<>(Long2LongPackedOpenHashDataMap.this.index.get(index).left(), v));
+      return oldValue;
+    }
+
+    @Override
+    public LongLongPair right(final long v) {
+      Long2LongPackedOpenHashDataMap.this.index.put(index,
+          new Pair<>(Long2LongPackedOpenHashDataMap.this.index.get(index).left(), v));
+      return this;
+    }
+
+    /**
+     * {@inheritDoc}
+     *
+     * @deprecated Please use the corresponding type-specific method instead.
+     */
+    @Deprecated
+    @Override
+    public Long getKey() {
+      return Long2LongPackedOpenHashDataMap.this.index.get(index).left();
+    }
+
+    /**
+     * {@inheritDoc}
+     *
+     * @deprecated Please use the corresponding type-specific method instead.
+     */
+    @Deprecated
+    @Override
+    public Long getValue() {
+      return Long2LongPackedOpenHashDataMap.this.index.get(index).right();
+    }
+
+    /**
+     * {@inheritDoc}
+     *
+     * @deprecated Please use the corresponding type-specific method instead.
+     */
+    @Deprecated
+    @Override
+    public Long setValue(final Long v) {
+      return setValue(v);
+    }
+
+    @Override
+    public boolean equals(final Object o) {
+      if (!(o instanceof Map.Entry)) {
+        return false;
+      }
+      Map.Entry<Long, Long> e = (Map.Entry<Long, Long>) o;
+      var pair = Long2LongPackedOpenHashDataMap.this.index.get(index);
+      return ((pair.left()) == ((e.getKey()).longValue()))
+          && ((pair.right()) == ((e.getValue()).longValue()));
+    }
+
+    @Override
+    public int hashCode() {
+      var pair = Long2LongPackedOpenHashDataMap.this.index.get(index);
+      return HashCommon.long2int(pair.left()) ^ HashCommon.long2int(pair.right());
+    }
+
+    @Override
+    public String toString() {
+      var pair = Long2LongPackedOpenHashDataMap.this.index.get(index);
+      return pair.left() + "=>" + pair.right();
+    }
+  }
+
+  /**
+   * An iterator over a hash map.
+   */
+  private abstract class MapIterator<ConsumerType> {
+    /**
+     * The index of the last entry returned, if positive or zero; initially, {@link #n}. If
+     * negative, the last entry returned was that of the key of index {@code - pos - 1} from the
+     * {@link #wrapped} list.
+     */
+    long pos = n;
+    /**
+     * The index of the last entry that has been returned (more precisely, the value of {@link #pos}
+     * if {@link #pos} is positive, or {@link Integer#MIN_VALUE} if {@link #pos} is negative). It is
+     * -1 if either we did not return an entry yet, or the last returned entry has been removed.
+     */
+    long last = -1;
+    /**
+     * A downward counter measuring how many entries must still be returned.
+     */
+    long c = size.get();
+    /**
+     * A boolean telling us whether we should return the entry with the null key.
+     */
+    boolean mustReturnNullKey = Long2LongPackedOpenHashDataMap.this.containsNullKey;
+    /**
+     * A lazily allocated list containing keys of entries that have wrapped around the table because
+     * of removals.
+     */
+    LongArrayList wrapped;
+
+    abstract void acceptOnIndex(final ConsumerType action, final long index);
+
+    public boolean hasNext() {
+      return c != 0;
+    }
+
+    public long nextEntry() {
+      if (!hasNext()) {
+        throw new NoSuchElementException();
+      }
+      c--;
+      if (mustReturnNullKey) {
+        mustReturnNullKey = false;
+        return last = n;
+      }
+      for (;;) {
+        if (--pos < 0) {
+          // We are just enumerating elements from the wrapped list.
+          last = Integer.MIN_VALUE;
+          final long k = wrapped.getLong((int) -pos - 1); // TODO: check if -pos - 1 is correct
+          long p = (int) HashCommon.mix((k)) & mask;
+          while (!((k) == (index.get(p).left()))) {
+            p = (p + 1) & mask;
+          }
+          return p;
+        }
+        if (!((index.get(pos).left()) == 0)) {
+          return last = pos;
+        }
+      }
+    }
+
+    public void forEachRemaining(final ConsumerType action) {
+      if (mustReturnNullKey) {
+        mustReturnNullKey = false;
+        acceptOnIndex(action, last = n);
+        c--;
+      }
+      while (c != 0) {
+        if (--pos < 0) {
+          // We are just enumerating elements from the wrapped list.
+          last = Integer.MIN_VALUE;
+          final long k = wrapped.getLong((int) -pos - 1); // TODO: check if -pos - 1 is correct
+          long p = (int) HashCommon.mix((k)) & mask;
+          while (!((k) == (index.get(p).left()))) {
+            p = (p + 1) & mask;
+          }
+          acceptOnIndex(action, p);
+          c--;
+        } else if (!((index.get(pos).left()) == 0)) {
+          acceptOnIndex(action, last = pos);
+          c--;
+        }
+      }
+    }
+
+    /**
+     * Shifts left entries with the specified hash code, starting at the specified position, and
+     * empties the resulting free entry.
+     *
+     * @param pos a starting position.
+     */
+    private void shiftKeys(long pos) {
+      // Shift entries with the same hash.
+      long last, slot;
+      long curr;
+      for (;;) {
+        pos = ((last = pos) + 1) & mask;
+        for (;;) {
+          if (((curr = index.get(pos).left()) == 0)) {
+            index.put(last, new Pair<>(curr, 0L));
+            return;
+          }
+          slot = (int) HashCommon.mix((curr)) & mask;
+          if (last <= pos ? last >= slot || slot > pos : last >= slot && slot > pos) {
+            break;
+          }
+          pos = (pos + 1) & mask;
+        }
+        var pair = index.get(pos);
+        if (pos < last) { // Wrapped entry.
+          if (wrapped == null) {
+            wrapped = new LongArrayList(2);
+          }
+          wrapped.add(pair.left());

Review Comment:
   ## Deprecated method or constructor invocation
   
   Invoking [AbstractLongCollection.add](1) should be avoided because it has been deprecated.
   
   [Show more details](https://github.com/apache/incubator-baremaps/security/code-scanning/770)



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@baremaps.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org