You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@kudu.apache.org by aw...@apache.org on 2020/01/13 19:30:34 UTC

[kudu] branch master updated: [util] Import FastHash hash function to util

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

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


The following commit(s) were added to refs/heads/master by this push:
     new 5067abb  [util] Import FastHash hash function to util
5067abb is described below

commit 5067abbdd2dc95eea74072dec1b07e3c0eb98423
Author: Bankim Bhavsar <ba...@cloudera.com>
AuthorDate: Thu Dec 19 15:38:45 2019 -0800

    [util] Import FastHash hash function to util
    
    FastHash is a simple, robust, and efficient general purpose
    hash function from Google.
    
    It's fast, small in size and with least/no quality problems
    as per smhasher and hence a good candidate for BlockBloomFilter.
    
    C++ implementation has been ported straight from
    https://code.google.com/archive/p/fast-hash/ with minor style
    modifications.
    
    Java implementation has been adapted from the C implementation.
    
    Change-Id: Id0a21d6af10d9ba0dbd9ab46d73552d42976e8d7
    Reviewed-on: http://gerrit.cloudera.org:8080/14934
    Tested-by: Kudu Jenkins
    Reviewed-by: Alexey Serbin <as...@cloudera.com>
---
 .../main/java/org/apache/kudu/util/HashUtil.java   | 108 +++++++++++++++++++++
 .../java/org/apache/kudu/util/TestFashHash.java    |  64 ++++++++++++
 src/kudu/util/hash_util-test.cc                    |  29 ++++++
 src/kudu/util/hash_util.h                          |  62 +++++++++++-
 4 files changed, 260 insertions(+), 3 deletions(-)

diff --git a/java/kudu-client/src/main/java/org/apache/kudu/util/HashUtil.java b/java/kudu-client/src/main/java/org/apache/kudu/util/HashUtil.java
new file mode 100644
index 0000000..0ca8896
--- /dev/null
+++ b/java/kudu-client/src/main/java/org/apache/kudu/util/HashUtil.java
@@ -0,0 +1,108 @@
+// 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.kudu.util;
+
+/**
+ * Hash utility functions.
+ */
+public class HashUtil {
+  /**
+   * Compute 64-bit FastHash of the supplied data backed by byte array.
+   *
+   * FastHash is simple, robust, and efficient general-purpose hash function from Google.
+   * Implementation is adapted from https://code.google.com/archive/p/fast-hash/
+   *
+   * @param buf the data to hash
+   * @param len length of the supplied data
+   * @param seed seed to compute the hash
+   * @return computed 64-bit hash value
+   */
+  public static long FastHash64(final byte[] buf, int len, long seed) {
+    final long m = 0x880355f21e6d1965L;
+    long h = seed ^ (len * m);
+    long v;
+
+    int len8 = len / 8;
+    for (int i = 0; i < len8; ++i) {
+      int pos = i * 8;
+      v = buf[pos] +
+          ((long)buf[pos + 1] << 8) +  ((long)buf[pos + 2] << 16) +
+          ((long)buf[pos + 3] << 24) + ((long)buf[pos + 4] << 32) +
+          ((long)buf[pos + 5] << 40) + ((long)buf[pos + 6] << 48) +
+          ((long)buf[pos + 7] << 56);
+      h ^= FastHashMix(v);
+      h *= m;
+    }
+
+    v = 0;
+    int pos2 = len8 * 8;
+    switch (len & 7) {
+      case 7:
+        v ^= (long)buf[pos2 + 6] << 48;
+      // fall through
+      case 6:
+        v ^= (long)buf[pos2 + 5] << 40;
+      // fall through
+      case 5:
+        v ^= (long)buf[pos2 + 4] << 32;
+      // fall through
+      case 4:
+        v ^= (long)buf[pos2 + 3] << 24;
+      // fall through
+      case 3:
+        v ^= (long)buf[pos2 + 2] << 16;
+      // fall through
+      case 2:
+        v ^= (long)buf[pos2 + 1] << 8;
+      // fall through
+      case 1:
+        v ^= buf[pos2];
+        h ^= FastHashMix(v);
+        h *= m;
+    }
+
+    return FastHashMix(h);
+  }
+
+  /**
+   * Compute 32-bit FastHash of the supplied data backed by byte array.
+   *
+   * FastHash is simple, robust, and efficient general-purpose hash function from Google.
+   * Implementation is adapted from https://code.google.com/archive/p/fast-hash/
+   *
+   * @param buf the data to compute the hash
+   * @param len length of the supplied data
+   * @param seed seed to compute the hash
+   * @return computed 32-bit hash value
+   */
+  public static int FastHash32(final byte[] buf, int len, int seed) {
+    // the following trick converts the 64-bit hashcode to Fermat
+    // residue, which shall retain information from both the higher
+    // and lower parts of hashcode.
+    long h = FastHash64(buf, len, seed);
+    return (int)(h - (h >>> 32));
+  }
+
+  // Compression function for Merkle-Damgard construction.
+  private static long FastHashMix(long h) {
+    h ^= h >>> 23;
+    h *= 0x2127599bf4325c37L;
+    h ^= h >>> 47;
+    return h;
+  }
+};
diff --git a/java/kudu-client/src/test/java/org/apache/kudu/util/TestFashHash.java b/java/kudu-client/src/test/java/org/apache/kudu/util/TestFashHash.java
new file mode 100644
index 0000000..6fa28fa
--- /dev/null
+++ b/java/kudu-client/src/test/java/org/apache/kudu/util/TestFashHash.java
@@ -0,0 +1,64 @@
+// 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.kudu.util;
+
+import static java.nio.charset.StandardCharsets.UTF_8;
+import static org.junit.Assert.assertEquals;
+
+import org.apache.kudu.test.junit.RetryRule;
+import org.junit.Rule;
+import org.junit.Test;
+
+/**
+ * Test FastHash64/32 returns the expected values for inputs.
+ *
+ * These tests are duplicated on the C++ side to ensure that hash computations
+ * are stable across both platforms.
+ */
+public class TestFashHash {
+  @Rule
+  public RetryRule retryRule = new RetryRule();
+
+  @Test
+  public void testFastHash64() {
+    long hash;
+
+    hash = HashUtil.FastHash64("ab".getBytes(UTF_8), 2, 0);
+    assertEquals(Long.parseUnsignedLong("17293172613997361769"), hash);
+
+    hash = HashUtil.FastHash64("abcdefg".getBytes(UTF_8), 7, 0);
+    assertEquals(Long.parseUnsignedLong("10206404559164245992"), hash);
+
+    hash = HashUtil.FastHash64("quick brown fox".getBytes(UTF_8), 15, 42);
+    assertEquals(Long.parseUnsignedLong("3757424404558187042"), hash);
+  }
+
+  @Test
+  public void testFastHash32() {
+    int hash;
+
+    hash = HashUtil.FastHash32("ab".getBytes(UTF_8), 2, 0);
+    assertEquals(Integer.parseUnsignedInt("2564147595"), hash);
+
+    hash = HashUtil.FastHash32("abcdefg".getBytes(UTF_8), 7, 0);
+    assertEquals(Integer.parseUnsignedInt("1497700618"), hash);
+
+    hash = HashUtil.FastHash32("quick brown fox".getBytes(UTF_8), 15, 42);
+    assertEquals(Integer.parseUnsignedInt("1676541068"), hash);
+  }
+}
diff --git a/src/kudu/util/hash_util-test.cc b/src/kudu/util/hash_util-test.cc
index 4e40dd5..72c975a 100644
--- a/src/kudu/util/hash_util-test.cc
+++ b/src/kudu/util/hash_util-test.cc
@@ -39,4 +39,33 @@ TEST(HashUtilTest, TestMurmur2Hash64) {
   ASSERT_EQ(3575930248840144026, hash);
 }
 
+// Test FastHash64/32 returns the expected values for inputs. These tests are
+// duplicated on the Java side to ensure that hash computations are stable
+// across both platforms.
+TEST(HashUtilTest, TestFastHash64) {
+  uint64_t hash;
+
+  hash = HashUtil::FastHash64("ab", 2, 0);
+  ASSERT_EQ(17293172613997361769UL, hash);
+
+  hash = HashUtil::FastHash64("abcdefg", 7, 0);
+  ASSERT_EQ(10206404559164245992UL, hash);
+
+  hash = HashUtil::FastHash64("quick brown fox", 15, 42);
+  ASSERT_EQ(3757424404558187042UL, hash);
+}
+
+TEST(HashUtilTest, TestFastHash32) {
+  uint64_t hash;
+
+  hash = HashUtil::FastHash32("ab", 2, 0);
+  ASSERT_EQ(2564147595U, hash);
+
+  hash = HashUtil::FastHash32("abcdefg", 7, 0);
+  ASSERT_EQ(1497700618U, hash);
+
+  hash = HashUtil::FastHash32("quick brown fox", 15, 42);
+  ASSERT_EQ(1676541068U, hash);
+}
+
 } // namespace kudu
diff --git a/src/kudu/util/hash_util.h b/src/kudu/util/hash_util.h
index d3a513b..2513af0 100644
--- a/src/kudu/util/hash_util.h
+++ b/src/kudu/util/hash_util.h
@@ -28,12 +28,12 @@ namespace kudu {
 class HashUtil {
  public:
 
-  static const uint64_t MURMUR_PRIME = 0xc6a4a7935bd1e995;
-  static const int MURMUR_R = 47;
-
   /// Murmur2 hash implementation returning 64-bit hashes.
   ATTRIBUTE_NO_SANITIZE_INTEGER
   static uint64_t MurmurHash2_64(const void* input, int len, uint64_t seed) {
+    static constexpr uint64_t MURMUR_PRIME = 0xc6a4a7935bd1e995UL;
+    static constexpr int MURMUR_R = 47;
+
     uint64_t h = seed ^ (len * MURMUR_PRIME);
 
     const uint64_t* data = reinterpret_cast<const uint64_t*>(input);
@@ -65,6 +65,62 @@ class HashUtil {
     h ^= h >> MURMUR_R;
     return h;
   }
+
+
+  // FastHash is simple, robust, and efficient general-purpose hash function from Google.
+  // Implementation is adapted from https://code.google.com/archive/p/fast-hash/
+  //
+  // Compute 64-bit FastHash.
+  ATTRIBUTE_NO_SANITIZE_INTEGER
+  static uint64_t FastHash64(const void* buf, size_t len, uint64_t seed) {
+    static constexpr uint64_t kMultiplier = 0x880355f21e6d1965UL;
+    const uint64_t* pos = static_cast<const uint64_t*>(buf);
+    const uint64_t* end = pos + (len / 8);
+    uint64_t h = seed ^ (len * kMultiplier);
+    uint64_t v;
+
+    while (pos != end) {
+      v  = *pos++;
+      h ^= FastHashMix(v);
+      h *= kMultiplier;
+    }
+
+    const uint8_t* pos2 = reinterpret_cast<const uint8_t*>(pos);
+    v = 0;
+
+    switch (len & 7) {
+      case 7: v ^= static_cast<uint64_t>(pos2[6]) << 48;
+      case 6: v ^= static_cast<uint64_t>(pos2[5]) << 40;
+      case 5: v ^= static_cast<uint64_t>(pos2[4]) << 32;
+      case 4: v ^= static_cast<uint64_t>(pos2[3]) << 24;
+      case 3: v ^= static_cast<uint64_t>(pos2[2]) << 16;
+      case 2: v ^= static_cast<uint64_t>(pos2[1]) << 8;
+      case 1: v ^= static_cast<uint64_t>(pos2[0]);
+        h ^= FastHashMix(v);
+        h *= kMultiplier;
+    }
+
+    return FastHashMix(h);
+  }
+
+  // Compute 32-bit FastHash.
+  static uint32_t FastHash32(const void* buf, size_t len, uint32_t seed) {
+    // the following trick converts the 64-bit hashcode to Fermat
+    // residue, which shall retain information from both the higher
+    // and lower parts of hashcode.
+    uint64_t h = FastHash64(buf, len, seed);
+    return h - (h >> 32);
+  }
+
+ private:
+  // Compression function for Merkle-Damgard construction.
+  ATTRIBUTE_NO_SANITIZE_INTEGER
+  static uint64_t FastHashMix(uint64_t h) {
+    h ^= h >> 23;
+    h *= 0x2127599bf4325c37UL;
+    h ^= h >> 47;
+    return h;
+  }
 };
 
 } // namespace kudu