You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@arrow.apache.org by em...@apache.org on 2019/08/10 05:55:40 UTC

[arrow] branch master updated: ARROW-6024: [Java] Provide more hash algorithms

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

emkornfield pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/arrow.git


The following commit(s) were added to refs/heads/master by this push:
     new bc27f17  ARROW-6024: [Java] Provide more hash algorithms
bc27f17 is described below

commit bc27f177e362e6aaee1930c3710cda33b22886da
Author: Liya Fan <fa...@foxmail.com>
AuthorDate: Fri Aug 9 20:52:13 2019 -0700

    ARROW-6024: [Java] Provide more hash algorithms
    
    Provide more hash algorithms to choose for different scenarios. In particular, we provide the following hash algorithms:
    
    * Simple hasher: A hasher that calculates the hash code of integers as is, and do not perform any finalization. So the computation is extremely efficient, but the quality of the produced hash code may not be good.
    
    * Murmur finalizing hasher: Finalize the hash code by the Murmur hashing algorithm. Details of the algorithm can be found in https://en.wikipedia.org/wiki/MurmurHash. Murmur hashing is computational expensive, as it involves several integer multiplications. However, the produced hash codes have good quality in the sense that they are uniformly distributed in the universe.
    
    * Jenkins finalizing hasher: Finalize the hash code by Bob Jenkins' algorithm. Details of this algorithm can be found in http://www.burtleburtle.net/bob/hash/integer.html.  Jenkins hashing is less computational expensive than Murmur hashing, as it involves no integer multiplication. However, the produced hash codes also have good quality in the sense that they are uniformly distributed in the universe.
    
    * Non-negative hasher: Wrapper for another hasher, to make the generated hash code non-negative. This can be useful for scenarios like hash table.
    
    Closes #4934 from liyafan82/fly_0724_hash and squashes the following commits:
    
    e4d466399 <Pindikura Ravindra> Update MurmurHasher.java
    8bcd5a4aa <liyafan82>  Provide more hash algorithms
    
    Lead-authored-by: Liya Fan <fa...@foxmail.com>
    Co-authored-by: Pindikura Ravindra <ra...@dremio.com>
    Co-authored-by: liyafan82 <fa...@foxmail.com>
    Signed-off-by: Micah Kornfield <em...@gmail.com>
---
 .../apache/arrow/memory/util/ArrowBufPointer.java  |  10 +-
 .../arrow/memory/util/ByteFunctionHelpers.java     |   4 +-
 .../arrow/memory/util/hash/ArrowBufHasher.java     | 116 +------------
 .../arrow/memory/util/hash/DirectHasher.java       |  87 ----------
 .../arrow/memory/util/hash/MurmurHasher.java       | 182 +++++++++++++++++++++
 .../arrow/memory/util/hash/SimpleHasher.java       | 126 ++++++++++++++
 .../arrow/memory/util/TestArrowBufPointer.java     |  31 +---
 .../memory/util/{ => hash}/TestArrowBufHasher.java |  62 +++++--
 8 files changed, 371 insertions(+), 247 deletions(-)

diff --git a/java/memory/src/main/java/org/apache/arrow/memory/util/ArrowBufPointer.java b/java/memory/src/main/java/org/apache/arrow/memory/util/ArrowBufPointer.java
index 3d0bf7b..5c6352c 100644
--- a/java/memory/src/main/java/org/apache/arrow/memory/util/ArrowBufPointer.java
+++ b/java/memory/src/main/java/org/apache/arrow/memory/util/ArrowBufPointer.java
@@ -17,8 +17,10 @@
 
 package org.apache.arrow.memory.util;
 
+import java.nio.ByteOrder;
+
 import org.apache.arrow.memory.util.hash.ArrowBufHasher;
-import org.apache.arrow.memory.util.hash.DirectHasher;
+import org.apache.arrow.memory.util.hash.SimpleHasher;
 import org.apache.arrow.util.Preconditions;
 
 import io.netty.buffer.ArrowBuf;
@@ -29,6 +31,8 @@ import io.netty.buffer.ArrowBuf;
  */
 public final class ArrowBufPointer {
 
+  public static final boolean LITTLE_ENDIAN = ByteOrder.nativeOrder() == ByteOrder.LITTLE_ENDIAN;
+
   /**
    * The hash code when the arrow buffer is null.
    */
@@ -53,7 +57,7 @@ public final class ArrowBufPointer {
    * The default constructor.
    */
   public ArrowBufPointer() {
-    this(DirectHasher.INSTANCE);
+    this(SimpleHasher.INSTANCE);
   }
 
   /**
@@ -72,7 +76,7 @@ public final class ArrowBufPointer {
    * @param length the length off set of the memory region pointed to.
    */
   public ArrowBufPointer(ArrowBuf buf, int offset, int length) {
-    this(buf, offset, length, DirectHasher.INSTANCE);
+    this(buf, offset, length, SimpleHasher.INSTANCE);
   }
 
   /**
diff --git a/java/memory/src/main/java/org/apache/arrow/memory/util/ByteFunctionHelpers.java b/java/memory/src/main/java/org/apache/arrow/memory/util/ByteFunctionHelpers.java
index da81b41..98a2158 100644
--- a/java/memory/src/main/java/org/apache/arrow/memory/util/ByteFunctionHelpers.java
+++ b/java/memory/src/main/java/org/apache/arrow/memory/util/ByteFunctionHelpers.java
@@ -19,7 +19,7 @@ package org.apache.arrow.memory.util;
 
 import org.apache.arrow.memory.BoundsChecking;
 import org.apache.arrow.memory.util.hash.ArrowBufHasher;
-import org.apache.arrow.memory.util.hash.DirectHasher;
+import org.apache.arrow.memory.util.hash.SimpleHasher;
 
 import io.netty.buffer.ArrowBuf;
 import io.netty.util.internal.PlatformDependent;
@@ -253,7 +253,7 @@ public class ByteFunctionHelpers {
    */
   public static final int hash(final ArrowBuf buf, int start, int end) {
 
-    ArrowBufHasher hasher = DirectHasher.INSTANCE;
+    ArrowBufHasher hasher = SimpleHasher.INSTANCE;
 
     return hasher.hashCode(buf, start, end - start);
   }
diff --git a/java/memory/src/main/java/org/apache/arrow/memory/util/hash/ArrowBufHasher.java b/java/memory/src/main/java/org/apache/arrow/memory/util/hash/ArrowBufHasher.java
index 51d29ba..25e2c79 100644
--- a/java/memory/src/main/java/org/apache/arrow/memory/util/hash/ArrowBufHasher.java
+++ b/java/memory/src/main/java/org/apache/arrow/memory/util/hash/ArrowBufHasher.java
@@ -17,53 +17,16 @@
 
 package org.apache.arrow.memory.util.hash;
 
-import static io.netty.util.internal.PlatformDependent.getByte;
-import static io.netty.util.internal.PlatformDependent.getInt;
-import static io.netty.util.internal.PlatformDependent.getLong;
-
-import java.nio.ByteOrder;
-
 import io.netty.buffer.ArrowBuf;
 
 /**
  * Utility for calculating the hash code for a consecutive memory region.
  * This class provides the basic framework for efficiently calculating the hash code.
- * It first splits the memory region into small segments with 8 bytes, 4 bytes and 1 byte,
- * and calculates hash codes for them separately. It produces the final hash code by combining
- * the hash codes and finalizing the resulting hash code.
- *
  * <p>
- *   To compute the hash code, the user simply calls the hashCode methods with the starting
- *   address and length of the memory region.
- * </p>
- * <p>
- *   A default light-weight implementation of this class is given in {@link DirectHasher}. However, the users can
- *   devise their own customized hasher by sub-classing this method and overriding the abstract methods.
- *   In particular
- *   <li>
- *     {@link ArrowBufHasher#combineHashCode(int, int)} provides the method for combining hash
- *     codes for individual small segments.
- *   </li>
- *   <li>
- *     {@link ArrowBufHasher#finalizeHashCode(int)} provides the method for finalizing the hash code.
- *   </li>
- *   <li>
- *     {@link ArrowBufHasher#getByteHashCode(byte)} provides the method for calculating the hash code
- *     for 1-byte memory segment.
- *   </li>
- *   <li>
- *     {@link ArrowBufHasher#getIntHashCode(int)} provides the method for calculating the hash code
- *     for 4-byte memory segment.
- *   </li>
- *   <li>
- *     {@link ArrowBufHasher#getLongHashCode(long)} provides the method for calculating the hash code
- *     for 8-byte memory segment.
- *   </li>
+ *   A default light-weight implementation is given in {@link SimpleHasher}.
  * </p>
  */
-public abstract class ArrowBufHasher {
-
-  public static final boolean LITTLE_ENDIAN = ByteOrder.nativeOrder() == ByteOrder.LITTLE_ENDIAN;
+public interface ArrowBufHasher {
 
   /**
    * Calculates the hash code for a memory region.
@@ -71,39 +34,7 @@ public abstract class ArrowBufHasher {
    * @param length length of the memory region.
    * @return the hash code.
    */
-  public int hashCode(long address, int length) {
-    int hashValue = 0;
-    int index = 0;
-    while (index + 8 <= length) {
-      long longValue = getLong(address + index);
-      if (!LITTLE_ENDIAN) {
-        // assume the buffer is in little endian
-        longValue = Long.reverseBytes(longValue);
-      }
-      int longHash = getLongHashCode(longValue);
-      hashValue = combineHashCode(hashValue, longHash);
-      index += 8;
-    }
-
-    while (index + 4 <= length) {
-      int intValue = getInt(address + index);
-      if (!LITTLE_ENDIAN) {
-        intValue = Integer.reverseBytes(intValue);
-      }
-      int intHash = getIntHashCode(intValue);
-      hashValue = combineHashCode(hashValue, intHash);
-      index += 4;
-    }
-
-    while (index < length) {
-      byte byteValue = getByte(address + index);
-      int byteHash = getByteHashCode(byteValue);
-      hashValue = combineHashCode(hashValue, byteHash);
-      index += 1;
-    }
-
-    return finalizeHashCode(hashValue);
-  }
+  int hashCode(long address, int length);
 
   /**
    * Calculates the hash code for a memory region.
@@ -112,44 +43,5 @@ public abstract class ArrowBufHasher {
    * @param length length of the memory region.
    * @return the hash code.
    */
-  public int hashCode(ArrowBuf buf, int offset, int length) {
-    buf.checkBytes(offset, offset + length);
-    return hashCode(buf.memoryAddress() + offset, length);
-  }
-
-  /**
-   * Calculates the hash code by combining the existing hash code and a new hash code.
-   * @param currentHashCode the existing hash code.
-   * @param newHashCode the new hash code.
-   * @return the combined hash code.
-   */
-  protected abstract int combineHashCode(int currentHashCode, int newHashCode);
-
-  /**
-   * Gets the hash code for a byte value.
-   * @param byteValue the byte value.
-   * @return the hash code.
-   */
-  protected abstract int getByteHashCode(byte byteValue);
-
-  /**
-   * Gets the hash code for a integer value.
-   * @param intValue the integer value.
-   * @return the hash code.
-   */
-  protected abstract int getIntHashCode(int intValue);
-
-  /**
-   * Gets the hash code for a long value.
-   * @param longValue the long value.
-   * @return the hash code.
-   */
-  protected abstract int getLongHashCode(long longValue);
-
-  /**
-   * Finalize the hash code.
-   * @param hashCode the current hash code.
-   * @return the finalized hash code.
-   */
-  protected abstract int finalizeHashCode(int hashCode);
+  int hashCode(ArrowBuf buf, int offset, int length);
 }
diff --git a/java/memory/src/main/java/org/apache/arrow/memory/util/hash/DirectHasher.java b/java/memory/src/main/java/org/apache/arrow/memory/util/hash/DirectHasher.java
deleted file mode 100644
index aa889e6..0000000
--- a/java/memory/src/main/java/org/apache/arrow/memory/util/hash/DirectHasher.java
+++ /dev/null
@@ -1,87 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.arrow.memory.util.hash;
-
-/**
- * Calculate hash code by directly returning the integers.
- * This is the default and the fastest way to get the hash code.
- * <p>
- *   Objects of class are stateless, so it can be shared between threads.
- * </p>
- */
-public class DirectHasher extends ArrowBufHasher {
-
-  public static DirectHasher INSTANCE = new DirectHasher();
-
-  private static final int DEFAULT_SEED = 0;
-
-  private DirectHasher() {
-
-  }
-
-  @Override
-  protected int combineHashCode(int currentHashCode, int newHashCode) {
-    return currentHashCode * 37 + newHashCode;
-  }
-
-  @Override
-  protected int getByteHashCode(byte byteValue) {
-    return (int) byteValue;
-  }
-
-  @Override
-  protected int getIntHashCode(int intValue) {
-    return intValue;
-  }
-
-  @Override
-  protected int getLongHashCode(long longValue) {
-    return Long.hashCode(longValue);
-  }
-
-  @Override
-  protected int finalizeHashCode(int hashCode) {
-    // finalize by the Murmur hashing algorithm
-    // details can be found in
-    // https://en.wikipedia.org/wiki/MurmurHash
-
-    int c1 = 0xcc9e2d51;
-    int c2 = 0x1b873593;
-    int r1 = 15;
-    int r2 = 13;
-    int m = 5;
-    int n = 0xe6546b64;
-
-    int k = hashCode;
-    k = k * c1;
-    k = k << r1;
-    k = k * c2;
-
-    int hash = DEFAULT_SEED;
-    hash = hash ^ k;
-    hash = hash << r2;
-    hash = hash * m + n;
-
-    return hash;
-  }
-
-  @Override
-  public boolean equals(Object obj) {
-    return obj != null && this.getClass() == obj.getClass();
-  }
-}
diff --git a/java/memory/src/main/java/org/apache/arrow/memory/util/hash/MurmurHasher.java b/java/memory/src/main/java/org/apache/arrow/memory/util/hash/MurmurHasher.java
new file mode 100644
index 0000000..9574fe6
--- /dev/null
+++ b/java/memory/src/main/java/org/apache/arrow/memory/util/hash/MurmurHasher.java
@@ -0,0 +1,182 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.arrow.memory.util.hash;
+
+import static io.netty.util.internal.PlatformDependent.getByte;
+import static io.netty.util.internal.PlatformDependent.getInt;
+
+import org.apache.arrow.memory.util.ArrowBufPointer;
+
+import io.netty.buffer.ArrowBuf;
+
+/**
+ * Implementation of the Murmur hashing algorithm.
+ * Details of the algorithm can be found in
+ * https://en.wikipedia.org/wiki/MurmurHash
+ * <p>
+ *   Murmur hashing is computationally expensive, as it involves several
+ *   integer multiplications. However, the produced hash codes have
+ *   good quality in the sense that they are uniformly distributed in the universe.
+ * </p>
+ * <p>
+ *   Therefore, this algorithm is suitable for scenarios where uniform hashing
+ *   is desired (e.g. in an open addressing hash table/hash set).
+ * </p>
+ */
+public class MurmurHasher implements ArrowBufHasher {
+
+  private final int seed;
+
+  /**
+   * Creates a default Murmur hasher, with seed 0.
+   */
+  public MurmurHasher() {
+    this(0);
+  }
+
+  /**
+   * Creates a Murmur hasher.
+   * @param seed the seed for the hasher.
+   */
+  public MurmurHasher(int seed) {
+    this.seed = seed;
+  }
+
+  @Override
+  public int hashCode(long address, int length) {
+    return hashCode(address, length, seed);
+  }
+
+  @Override
+  public int hashCode(ArrowBuf buf, int offset, int length) {
+    buf.checkBytes(offset, offset + length);
+    return hashCode(buf.memoryAddress() + offset, length);
+  }
+
+  /**
+   * Calculates the hash code for a memory region.
+   * @param buf the buffer for the memory region.
+   * @param offset offset within the buffer for the memory region.
+   * @param length length of the memory region.
+   * @param seed the seed.
+   * @return the hash code.
+   */
+  public static int hashCode(ArrowBuf buf, int offset, int length, int seed) {
+    buf.checkBytes(offset, offset + length);
+    return hashCode(buf.memoryAddress() + offset, length, seed);
+  }
+
+  /**
+   * Calculates the hash code for a memory region.
+   * @param address start address of the memory region.
+   * @param length length of the memory region.
+   * @param seed the seed.
+   * @return the hash code.
+   */
+  public static int hashCode(long address, int length, int seed) {
+    int index = 0;
+    int hash = seed;
+    while (index + 4 <= length) {
+      int intValue = getInt(address + index);
+      if (!ArrowBufPointer.LITTLE_ENDIAN) {
+        intValue = Integer.reverseBytes(intValue);
+      }
+      hash = combineHashCode(hash, intValue);
+      index += 4;
+    }
+
+    if (index < length) {
+      // process remaining data as a integer in little endian
+      int intValue = 0;
+      for (int i = index - 1; i >= index; i--) {
+        intValue <<= 8;
+        intValue |= (getByte(address + i) & 0x000000ff);
+        index += 1;
+      }
+      hash = combineHashCode(hash, intValue);
+    }
+    return finalizeHashCode(hash, length);
+  }
+
+  /**
+   * Combine the current hash code and a new int value to calculate
+   * a new hash code.
+   * @param currentHashCode the current hash code.
+   * @param intValue the new int value.
+   * @return the new hah code.
+   */
+  public static int combineHashCode(int currentHashCode, int intValue) {
+    int c1 = 0xcc9e2d51;
+    int c2 = 0x1b873593;
+    int r1 = 15;
+    int r2 = 13;
+    int m = 5;
+    int n = 0xe6546b64;
+
+    int k = intValue;
+    k = k * c1;
+    k = rotateLeft(k, r1);
+    k = k * c2;
+
+    int hash = currentHashCode;
+    hash = hash ^ k;
+    hash = rotateLeft(hash, r2);
+    hash = hash * m + n;
+
+    return hash;
+  }
+
+  /**
+   * Finalizing the hash code.
+   * @param hashCode the current hash code.
+   * @param length the length of the memory region.
+   * @return the finalized hash code.
+   */
+  public static int finalizeHashCode(int hashCode, int length) {
+    hashCode = hashCode ^ length;
+
+    hashCode = hashCode ^ (hashCode >>> 16);
+    hashCode = hashCode * 0x85ebca6b;
+    hashCode = hashCode ^ (hashCode >>> 13);
+    hashCode = hashCode * 0xc2b2ae35;
+    hashCode = hashCode ^ (hashCode >>> 16);
+
+    return hashCode;
+  }
+
+  private static int rotateLeft(int value, int count) {
+    return (value << count) | (value >>> (32 - count));
+  }
+
+  @Override
+  public boolean equals(Object o) {
+    if (this == o) {
+      return true;
+    }
+    if (o == null || getClass() != o.getClass()) {
+      return false;
+    }
+    MurmurHasher that = (MurmurHasher) o;
+    return seed == that.seed;
+  }
+
+  @Override
+  public int hashCode() {
+    return seed;
+  }
+}
diff --git a/java/memory/src/main/java/org/apache/arrow/memory/util/hash/SimpleHasher.java b/java/memory/src/main/java/org/apache/arrow/memory/util/hash/SimpleHasher.java
new file mode 100644
index 0000000..a75566b
--- /dev/null
+++ b/java/memory/src/main/java/org/apache/arrow/memory/util/hash/SimpleHasher.java
@@ -0,0 +1,126 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.arrow.memory.util.hash;
+
+import static io.netty.util.internal.PlatformDependent.getByte;
+import static io.netty.util.internal.PlatformDependent.getInt;
+import static io.netty.util.internal.PlatformDependent.getLong;
+
+import org.apache.arrow.memory.util.ArrowBufPointer;
+
+import io.netty.buffer.ArrowBuf;
+
+/**
+ * A simple hasher that calculates the hash code of integers as is,
+ * and does not perform any finalization. So the computation is extremely
+ * efficient.
+ * <p>
+ *   This algorithm only provides the most basic semantics for the hash code. That is,
+ *   if two objects are equal, they must have equal hash code. However, the quality of the
+ *   produced hash code may not be good. In other words, the generated hash codes are
+ *   far from being uniformly distributed in the universe.
+ * </p>
+ * <p>
+ *   Therefore, this algorithm is suitable only for scenarios where the most basic semantics
+ *   of the hash code is required (e.g. in scenarios that require fast and proactive data pruning)
+ * </p>
+ * <p>
+ *   An object of this class is stateless, so it can be shared between threads.
+ * </p>
+ */
+public class SimpleHasher implements ArrowBufHasher {
+
+  public static SimpleHasher INSTANCE = new SimpleHasher();
+
+  protected SimpleHasher() {
+  }
+
+  /**
+   * Calculates the hash code for a memory region.
+   * @param address start address of the memory region.
+   * @param length length of the memory region.
+   * @return the hash code.
+   */
+  public int hashCode(long address, int length) {
+    int hashValue = 0;
+    int index = 0;
+    while (index + 8 <= length) {
+      long longValue = getLong(address + index);
+      if (!ArrowBufPointer.LITTLE_ENDIAN) {
+        // assume the buffer is in little endian
+        longValue = Long.reverseBytes(longValue);
+      }
+      int longHash = getLongHashCode(longValue);
+      hashValue = combineHashCode(hashValue, longHash);
+      index += 8;
+    }
+
+    while (index + 4 <= length) {
+      int intValue = getInt(address + index);
+      if (!ArrowBufPointer.LITTLE_ENDIAN) {
+        intValue = Integer.reverseBytes(intValue);
+      }
+      int intHash = intValue;
+      hashValue = combineHashCode(hashValue, intHash);
+      index += 4;
+    }
+
+    while (index < length) {
+      byte byteValue = getByte(address + index);
+      int byteHash = byteValue;
+      hashValue = combineHashCode(hashValue, byteHash);
+      index += 1;
+    }
+
+    return finalizeHashCode(hashValue);
+  }
+
+  /**
+   * Calculates the hash code for a memory region.
+   * @param buf the buffer for the memory region.
+   * @param offset offset within the buffer for the memory region.
+   * @param length length of the memory region.
+   * @return the hash code.
+   */
+  public int hashCode(ArrowBuf buf, int offset, int length) {
+    buf.checkBytes(offset, offset + length);
+    return hashCode(buf.memoryAddress() + offset, length);
+  }
+
+  protected int combineHashCode(int currentHashCode, int newHashCode) {
+    return currentHashCode * 37 + newHashCode;
+  }
+
+  protected int getLongHashCode(long longValue) {
+    return Long.hashCode(longValue);
+  }
+
+  protected int finalizeHashCode(int hashCode) {
+    return hashCode;
+  }
+
+  @Override
+  public int hashCode() {
+    return 123;
+  }
+
+  @Override
+  public boolean equals(Object obj) {
+    return obj != null && (obj instanceof SimpleHasher);
+  }
+}
diff --git a/java/memory/src/test/java/org/apache/arrow/memory/util/TestArrowBufPointer.java b/java/memory/src/test/java/org/apache/arrow/memory/util/TestArrowBufPointer.java
index 36b78af..2296f52 100644
--- a/java/memory/src/test/java/org/apache/arrow/memory/util/TestArrowBufPointer.java
+++ b/java/memory/src/test/java/org/apache/arrow/memory/util/TestArrowBufPointer.java
@@ -25,7 +25,7 @@ import org.apache.arrow.memory.BufferAllocator;
 import org.apache.arrow.memory.RootAllocator;
 
 import org.apache.arrow.memory.util.hash.ArrowBufHasher;
-import org.apache.arrow.memory.util.hash.DirectHasher;
+import org.apache.arrow.memory.util.hash.SimpleHasher;
 import org.junit.After;
 import org.junit.Before;
 import org.junit.Test;
@@ -156,39 +156,20 @@ public class TestArrowBufPointer {
    * Hasher with a counter that increments each time a hash code is calculated.
    * This is to validate that the hash code in {@link ArrowBufPointer} is reused.
    */
-  class CounterHasher extends ArrowBufHasher {
+  class CounterHasher implements ArrowBufHasher {
 
     protected int counter = 0;
 
     @Override
-    protected int combineHashCode(int currentHashCode, int newHashCode) {
-      return 0;
-    }
-
-    @Override
-    protected int getByteHashCode(byte byteValue) {
-      return 0;
-    }
-
-    @Override
-    protected int getIntHashCode(int intValue) {
-      return 0;
-    }
-
-    @Override
-    protected int getLongHashCode(long longValue) {
-      return 0;
-    }
-
-    @Override
-    protected int finalizeHashCode(int hashCode) {
-      return 0;
+    public int hashCode(long address, int length) {
+      counter += 1;
+      return SimpleHasher.INSTANCE.hashCode(address, length);
     }
 
     @Override
     public int hashCode(ArrowBuf buf, int offset, int length) {
       counter += 1;
-      return DirectHasher.INSTANCE.hashCode(buf, offset, length);
+      return SimpleHasher.INSTANCE.hashCode(buf, offset, length);
     }
 
     @Override
diff --git a/java/memory/src/test/java/org/apache/arrow/memory/util/TestArrowBufHasher.java b/java/memory/src/test/java/org/apache/arrow/memory/util/hash/TestArrowBufHasher.java
similarity index 58%
rename from java/memory/src/test/java/org/apache/arrow/memory/util/TestArrowBufHasher.java
rename to java/memory/src/test/java/org/apache/arrow/memory/util/hash/TestArrowBufHasher.java
index 9fc9c50..c7c0398 100644
--- a/java/memory/src/test/java/org/apache/arrow/memory/util/TestArrowBufHasher.java
+++ b/java/memory/src/test/java/org/apache/arrow/memory/util/hash/TestArrowBufHasher.java
@@ -15,31 +15,42 @@
  * limitations under the License.
  */
 
-package org.apache.arrow.memory.util;
+package org.apache.arrow.memory.util.hash;
 
 import static org.junit.Assert.assertEquals;
 import static org.junit.jupiter.api.Assertions.assertThrows;
 
+import java.util.Arrays;
+import java.util.Collection;
+
 import org.apache.arrow.memory.BufferAllocator;
 import org.apache.arrow.memory.RootAllocator;
 
-import org.apache.arrow.memory.util.hash.ArrowBufHasher;
-import org.apache.arrow.memory.util.hash.DirectHasher;
 import org.junit.After;
 import org.junit.Before;
 import org.junit.Test;
 
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
+
 import io.netty.buffer.ArrowBuf;
 
 /**
  * Test cases for {@link ArrowBufHasher} and its subclasses.
  */
+@RunWith(Parameterized.class)
 public class TestArrowBufHasher {
 
   private final int BUFFER_LENGTH = 1024;
 
   private BufferAllocator allocator;
 
+  private ArrowBufHasher hasher;
+
+  public TestArrowBufHasher(String name, ArrowBufHasher hasher) {
+    this.hasher = hasher;
+  }
+
   @Before
   public void prepare() {
     allocator = new RootAllocator(1024 * 1024);
@@ -51,7 +62,7 @@ public class TestArrowBufHasher {
   }
 
   @Test
-  public void testDirectHasher() {
+  public void testHasher() {
     try (ArrowBuf buf1 = allocator.buffer(BUFFER_LENGTH);
          ArrowBuf buf2 = allocator.buffer(BUFFER_LENGTH)) {
       // prepare data
@@ -60,30 +71,34 @@ public class TestArrowBufHasher {
         buf2.setFloat(i * 4, i / 10.0f);
       }
 
-      ArrowBufHasher hasher = DirectHasher.INSTANCE;
-
-      assertEquals(hasher.hashCode(buf1, 0, 100), hasher.hashCode(buf2, 0, 100));
-      assertEquals(hasher.hashCode(buf1, 1, 5), hasher.hashCode(buf2, 1, 5));
-      assertEquals(hasher.hashCode(buf1, 10, 17), hasher.hashCode(buf2, 10, 17));
-      assertEquals(hasher.hashCode(buf1, 33, 25), hasher.hashCode(buf2, 33, 25));
-      assertEquals(hasher.hashCode(buf1, 22, 22), hasher.hashCode(buf2, 22, 22));
-      assertEquals(hasher.hashCode(buf1, 123, 333), hasher.hashCode(buf2, 123, 333));
-      assertEquals(hasher.hashCode(buf1, 374, 1), hasher.hashCode(buf2, 374, 1));
-      assertEquals(hasher.hashCode(buf1, 11, 0), hasher.hashCode(buf2, 11, 0));
-      assertEquals(hasher.hashCode(buf1, 75, 25), hasher.hashCode(buf2, 75, 25));
-      assertEquals(hasher.hashCode(buf1, 0, 1024), hasher.hashCode(buf2, 0, 1024));
+      verifyHashCodesEqual(buf1, 0, 100, buf2, 0, 100);
+      verifyHashCodesEqual(buf1, 1, 5, buf2, 1, 5);
+      verifyHashCodesEqual(buf1, 10, 17, buf2, 10, 17);
+      verifyHashCodesEqual(buf1, 33, 25, buf2, 33, 25);
+      verifyHashCodesEqual(buf1, 22, 22, buf2, 22, 22);
+      verifyHashCodesEqual(buf1, 123, 333, buf2, 123, 333);
+      verifyHashCodesEqual(buf1, 374, 1, buf2, 374, 1);
+      verifyHashCodesEqual(buf1, 11, 0, buf2, 11, 0);
+      verifyHashCodesEqual(buf1, 75, 25, buf2, 75, 25);
+      verifyHashCodesEqual(buf1, 0, 1024, buf2, 0, 1024);
     }
   }
 
+  private void verifyHashCodesEqual(ArrowBuf buf1, int offset1, int length1,
+                                    ArrowBuf buf2, int offset2, int length2) {
+    int hashCode1 = hasher.hashCode(buf1, offset1, length1);
+    int hashCode2 = hasher.hashCode(buf2, offset2, length2);
+    assertEquals(hashCode1, hashCode2);
+  }
+
   @Test
-  public void testDirectHasherNegative() {
+  public void testHasherNegative() {
     try (ArrowBuf buf = allocator.buffer(BUFFER_LENGTH)) {
       // prepare data
       for (int i = 0; i < BUFFER_LENGTH / 4; i++) {
         buf.setFloat(i * 4, i / 10.0f);
       }
 
-      ArrowBufHasher hasher = DirectHasher.INSTANCE;
       assertThrows(IllegalArgumentException.class, () -> {
         hasher.hashCode(buf, 0, -1);
       });
@@ -97,4 +112,15 @@ public class TestArrowBufHasher {
       });
     }
   }
+
+  @Parameterized.Parameters(name = "hasher = {0}")
+  public static Collection<Object[]> getHasher() {
+    return Arrays.asList(
+      new Object[] {SimpleHasher.class.getSimpleName(),
+        SimpleHasher.INSTANCE},
+      new Object[] {MurmurHasher.class.getSimpleName(),
+        new MurmurHasher()
+      }
+    );
+  }
 }