You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@drill.apache.org by su...@apache.org on 2016/04/20 07:32:33 UTC

[1/3] drill git commit: DRILL-4237, DRILL-4478: Implement hash to use murmur3 and add correspondent unit tests

Repository: drill
Updated Branches:
  refs/heads/master 852b01aa6 -> c6a03eb17


http://git-wip-us.apache.org/repos/asf/drill/blob/c6a03eb1/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/impl/XXHash.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/impl/XXHash.java b/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/impl/XXHash.java
index 0124b2f..72ab492 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/impl/XXHash.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/impl/XXHash.java
@@ -24,14 +24,15 @@ import org.apache.drill.exec.memory.BoundsChecking;
 
 import com.google.common.primitives.UnsignedLongs;
 
-public final class XXHash {
+public final class XXHash extends DrillHash{
   static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(XXHash.class);
 
-  static final long PRIME64_1 = UnsignedLongs.decode("11400714785074694791");
-  static final long PRIME64_2 = UnsignedLongs.decode("14029467366897019727");
-  static final long PRIME64_3 = UnsignedLongs.decode("1609587929392839161");
-  static final long PRIME64_4 = UnsignedLongs.decode("9650029242287828579");
-  static final long PRIME64_5 = UnsignedLongs.decode("2870177450012600261");
+  //UnsignedLongs.decode won't give right output(keep the value in 8 bytes unchanged).
+  static final long PRIME64_1 = 0x9e3779b185ebca87L;//UnsignedLongs.decode("11400714785074694791");
+  static final long PRIME64_2 = 0xc2b2ae3d27d4eb4fL;//UnsignedLongs.decode("14029467366897019727");
+  static final long PRIME64_3 = 0x165667b19e3779f9L;//UnsignedLongs.decode("1609587929392839161");
+  static final long PRIME64_4 = 0x85ebca77c2b2ae63L;//UnsignedLongs.decode("9650029242287828579");
+  static final long PRIME64_5 = 0x27d4eb2f165667c5L;//UnsignedLongs.decode("2870177450012600261");
 
   private static long hash64bytes(long start, long bEnd, long seed) {
     long len = bEnd - start;
@@ -114,12 +115,14 @@ public final class XXHash {
     }
 
     if (p + 4 <= bEnd) {
-      h64 ^= PlatformDependent.getInt(p) * PRIME64_1;
+      //IMPORTANT: we are expecting a long from these 4 bytes. Which means it is always positive
+      long finalInt = getIntLittleEndian(p);
+      h64 ^= finalInt * PRIME64_1;
       h64 = Long.rotateLeft(h64, 23) * PRIME64_2 + PRIME64_3;
       p += 4;
     }
     while (p < bEnd) {
-      h64 ^= PlatformDependent.getByte(p) * PRIME64_5;
+      h64 ^= ((long)(PlatformDependent.getByte(p) & 0x00ff)) * PRIME64_5;
       h64 = Long.rotateLeft(h64, 11) * PRIME64_1;
       p++;
     }
@@ -128,25 +131,17 @@ public final class XXHash {
   }
 
   private static long applyFinalHashComputation(long h64) {
-    h64 ^= h64 >> 33;
+    //IMPORTANT: using logical right shift instead of arithmetic right shift
+    h64 ^= h64 >>> 33;
     h64 *= PRIME64_2;
-    h64 ^= h64 >> 29;
+    h64 ^= h64 >>> 29;
     h64 *= PRIME64_3;
-    h64 ^= h64 >> 32;
+    h64 ^= h64 >>> 32;
     return h64;
   }
 
 
-  /* 64 bit variations */
-  public static long hash64(int val, long seed){
-    long h64 = seed + PRIME64_5;
-    h64 += 4; // add length (4 bytes) to hash value
-    h64 ^= val * PRIME64_1;
-    h64 = Long.rotateLeft(h64, 23) * PRIME64_2 + PRIME64_3;
-    return applyFinalHashComputation(h64);
-  }
-
-  public static long hash64(long val, long seed){
+  public static long hash64Internal(long val, long seed){
     long h64 = seed + PRIME64_5;
     h64 += 8; // add length (8 bytes) to hash value
     long k1 = val* PRIME64_2;
@@ -157,17 +152,22 @@ public final class XXHash {
     return applyFinalHashComputation(h64);
   }
 
-  public static long hash64(float val, long seed){
-    return hash64(Float.floatToIntBits(val), seed);
+  /**
+   * @param val the input 64 bit hash value
+   * @return converted 32 bit hash value
+   */
+  private static int convert64To32(long val) {
+    return (int) (val & 0x00FFFFFFFF);
   }
 
+
   public static long hash64(double val, long seed){
-    return hash64(Double.doubleToLongBits(val), seed);
+    return hash64Internal(Double.doubleToLongBits(val), seed);
   }
 
-  public static long hash64(int start, int end, DrillBuf buffer, long seed){
+  public static long hash64(long start, long end, DrillBuf buffer, long seed){
     if (BoundsChecking.BOUNDS_CHECKING_ENABLED) {
-      buffer.checkBytes(start, end);
+      buffer.checkBytes((int)start, (int)end);
     }
 
     long s = buffer.memoryAddress() + start;
@@ -176,38 +176,12 @@ public final class XXHash {
     return hash64bytes(s, e, seed);
   }
 
-  /* 32 bit variations */
-  public static int hash32(int val, long seed){
-    return convert64To32(hash64(val, seed));
-  }
-
-  public static int hash32(long val, long seed){
-    return convert64To32(hash64(val, seed));
-  }
-
-  public static int hash32(float val, long seed){
-    return convert64To32(hash64(val, seed));
-  }
-
   public static int hash32(double val, long seed){
     return convert64To32(hash64(val, seed));
   }
 
-  public static int hash32(int start, int end, DrillBuf buffer, long seed){
+  public static int hash32(int start, int end, DrillBuf buffer, int seed){
     return convert64To32(hash64(start, end, buffer, seed));
   }
 
-  /**
-   * Convert a 64 bit hash value to a 32 bit by taking the XOR of the
-   * most significant 4 bytes with the least significant 4 bytes.
-   * @param val the input 64 bit hash value
-   * @return converted 32 bit hash value
-   */
-  private static int convert64To32(long val) {
-
-    int msb = (int) ((val >>> 32) & 0xFFFFFFFF);
-    int lsb = (int) (val);
-    return (msb ^ lsb);
-  }
-
 }

http://git-wip-us.apache.org/repos/asf/drill/blob/c6a03eb1/exec/java-exec/src/test/java/org/apache/drill/TestFunctionsQuery.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/test/java/org/apache/drill/TestFunctionsQuery.java b/exec/java-exec/src/test/java/org/apache/drill/TestFunctionsQuery.java
index 53a86bb..475d08a 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/TestFunctionsQuery.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/TestFunctionsQuery.java
@@ -584,22 +584,6 @@ public class TestFunctionsQuery extends BaseTestQuery {
   }
 
   @Test
-  public void testHashFunctions() throws Exception {
-    String query = "select " +
-        "hash(cast(hire_date as date)) hash_date, " +
-        "hash(cast(employee_id as decimal(9, 2))) as hash_dec9, " +
-        "hash(cast(employee_id as decimal(38, 11))) as hash_dec38 " +
-        "from cp.`employee.json` where employee_id = 1 limit 1";
-
-    testBuilder()
-        .sqlQuery(query)
-        .unOrdered()
-        .baselineColumns("hash_date", "hash_dec9", "hash_dec38")
-        .baselineValues(312993367, 292570647, 337328302)
-        .go();
-  }
-
-  @Test
   public void testDecimalAddConstant() throws Exception {
     String query = "select (cast('-1' as decimal(37, 3)) + cast (employee_id as decimal(37, 3))) as CNT " +
         "from cp.`employee.json` where employee_id <= 4";
@@ -795,6 +779,7 @@ public class TestFunctionsQuery extends BaseTestQuery {
         .go();
   }
 
+
   /*
    * We may apply implicit casts in Hash Join while dealing with different numeric data types
    * For this to work we need to distribute the data based on a common key, below method
@@ -810,7 +795,7 @@ public class TestFunctionsQuery extends BaseTestQuery {
         "hash64AsDouble(cast(employee_id as decimal(9, 0))) = hash64AsDouble(cast(employee_id as decimal(18, 0))) col5, " +
         "hash64AsDouble(cast(employee_id as decimal(18, 0))) = hash64AsDouble(cast(employee_id as decimal(28, 0))) col6, " +
         "hash64AsDouble(cast(employee_id as decimal(28, 0))) = hash64AsDouble(cast(employee_id as decimal(38, 0))) col7 " +
-        "from cp.`employee.json` where employee_id = 1";
+        "from cp.`employee.json`  where employee_id = 1";
 
     testBuilder()
         .sqlQuery(query)
@@ -818,11 +803,32 @@ public class TestFunctionsQuery extends BaseTestQuery {
         .baselineColumns("col1", "col2", "col3", "col4", "col5", "col6", "col7")
         .baselineValues(true, true, true, true, true, true, true)
         .go();
+
+    java.util.Random seedGen = new java.util.Random();
+    seedGen.setSeed(System.currentTimeMillis());
+    int seed = seedGen.nextInt();
+
+    String querytemplate = "select " +
+            "hash64AsDouble(cast(employee_id as int), #RAND_SEED#) = hash64AsDouble(cast(employee_id as bigint), #RAND_SEED#) col1, " +
+            "hash64AsDouble(cast(employee_id as bigint), #RAND_SEED#) = hash64AsDouble(cast(employee_id as float), #RAND_SEED#) col2, " +
+            "hash64AsDouble(cast(employee_id as float), #RAND_SEED#) = hash64AsDouble(cast(employee_id as double), #RAND_SEED#) col3, " +
+            "hash64AsDouble(cast(employee_id as double), #RAND_SEED#) = hash64AsDouble(cast(employee_id as decimal(9, 0)), #RAND_SEED#) col4, " +
+            "hash64AsDouble(cast(employee_id as decimal(9, 0)), #RAND_SEED#) = hash64AsDouble(cast(employee_id as decimal(18, 0)), #RAND_SEED#) col5, " +
+            "hash64AsDouble(cast(employee_id as decimal(18, 0)), #RAND_SEED#) = hash64AsDouble(cast(employee_id as decimal(28, 0)), #RAND_SEED#) col6, " +
+            "hash64AsDouble(cast(employee_id as decimal(28, 0)), #RAND_SEED#) = hash64AsDouble(cast(employee_id as decimal(38, 0)), #RAND_SEED#) col7 " +
+            "from cp.`employee.json` where employee_id = 1";
+
+    String queryWithSeed = querytemplate.replaceAll("#RAND_SEED#", String.format("%d",seed));
+    testBuilder()
+            .sqlQuery(queryWithSeed)
+            .unOrdered()
+            .baselineColumns("col1", "col2", "col3", "col4", "col5", "col6", "col7")
+            .baselineValues(true, true, true, true, true, true, true)
+            .go();
+
   }
 
-  /*
-   * hash32 version of the above test
-   */
+
   @Test
   public void testHash32() throws Exception {
     String query = "select " +
@@ -841,6 +847,29 @@ public class TestFunctionsQuery extends BaseTestQuery {
         .baselineColumns("col1", "col2", "col3", "col4", "col5", "col6", "col7")
         .baselineValues(true, true, true, true, true, true, true)
         .go();
+
+    java.util.Random seedGen = new java.util.Random();
+    seedGen.setSeed(System.currentTimeMillis());
+    int seed = seedGen.nextInt();
+
+    String querytemplate = "select " +
+            "hash32AsDouble(cast(employee_id as int), #RAND_SEED#) = hash32AsDouble(cast(employee_id as bigint), #RAND_SEED#) col1, " +
+            "hash32AsDouble(cast(employee_id as bigint), #RAND_SEED#) = hash32AsDouble(cast(employee_id as float), #RAND_SEED#) col2, " +
+            "hash32AsDouble(cast(employee_id as float),  #RAND_SEED#) = hash32AsDouble(cast(employee_id as double), #RAND_SEED#) col3, " +
+            "hash32AsDouble(cast(employee_id as double), #RAND_SEED#) = hash32AsDouble(cast(employee_id as decimal(9, 0)), #RAND_SEED#) col4, " +
+            "hash32AsDouble(cast(employee_id as decimal(9, 0)), #RAND_SEED#) = hash32AsDouble(cast(employee_id as decimal(18, 0)), #RAND_SEED#) col5, " +
+            "hash32AsDouble(cast(employee_id as decimal(18, 0)), #RAND_SEED#) = hash32AsDouble(cast(employee_id as decimal(28, 0)), #RAND_SEED#) col6, " +
+            "hash32AsDouble(cast(employee_id as decimal(28, 0)), #RAND_SEED#) = hash32AsDouble(cast(employee_id as decimal(38, 0)), #RAND_SEED#) col7 " +
+            "from cp.`employee.json` where employee_id = 1";
+
+    String queryWithSeed = querytemplate.replaceAll("#RAND_SEED#", String.format("%d",seed));
+    testBuilder()
+            .sqlQuery(queryWithSeed)
+            .unOrdered()
+            .baselineColumns("col1", "col2", "col3", "col4", "col5", "col6", "col7")
+            .baselineValues(true, true, true, true, true, true, true)
+            .go();
+
   }
 
   @Test


[3/3] drill git commit: DRILL-4459: Resolve SchemaChangeException while querying hive json table

Posted by su...@apache.org.
DRILL-4459: Resolve SchemaChangeException while querying hive json table

- Replace drill var16char to varchar datatype for hive string datatype
- Change testGenericUDF() and testUDF() to use VarChar instead of Var16Char
- Add unit test for hive GET_JSON_OBJECT UDF

closes #431


Project: http://git-wip-us.apache.org/repos/asf/drill/repo
Commit: http://git-wip-us.apache.org/repos/asf/drill/commit/3b056db0
Tree: http://git-wip-us.apache.org/repos/asf/drill/tree/3b056db0
Diff: http://git-wip-us.apache.org/repos/asf/drill/diff/3b056db0

Branch: refs/heads/master
Commit: 3b056db0f504d50fe11a6028b1a633ec74d478d2
Parents: 852b01a
Author: Vitalii Diravka <vi...@gmail.com>
Authored: Thu Mar 10 14:52:28 2016 +0000
Committer: Sudheesh Katkam <sk...@maprtech.com>
Committed: Tue Apr 19 21:58:16 2016 -0700

----------------------------------------------------------------------
 .../core/src/main/codegen/data/HiveTypes.tdd    |  4 +-
 .../templates/ObjectInspectorHelper.java        | 26 +++++++---
 .../codegen/templates/ObjectInspectors.java     | 20 ++++----
 .../drill/exec/expr/fn/HiveFuncHolder.java      |  5 +-
 .../exec/expr/fn/HiveFunctionRegistry.java      | 12 ++---
 .../apache/drill/exec/fn/hive/TestHiveUDFs.java | 51 ++++++++++----------
 .../drill/exec/fn/hive/TestInbuiltHiveUDFs.java | 14 ++++++
 .../exec/hive/TestInfoSchemaOnHiveStorage.java  |  1 +
 .../exec/store/hive/HiveTestDataGenerator.java  |  6 +++
 .../resources/functions/hive/GenericUDF.json    |  8 +--
 .../src/test/resources/functions/hive/UDF.json  |  2 +-
 .../core/src/test/resources/simple.json         | 19 ++++++++
 12 files changed, 110 insertions(+), 58 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/drill/blob/3b056db0/contrib/storage-hive/core/src/main/codegen/data/HiveTypes.tdd
----------------------------------------------------------------------
diff --git a/contrib/storage-hive/core/src/main/codegen/data/HiveTypes.tdd b/contrib/storage-hive/core/src/main/codegen/data/HiveTypes.tdd
index 73752a3..e13dc36 100644
--- a/contrib/storage-hive/core/src/main/codegen/data/HiveTypes.tdd
+++ b/contrib/storage-hive/core/src/main/codegen/data/HiveTypes.tdd
@@ -80,10 +80,10 @@
       needOIForDrillType: false
     },
     {
-      hiveType: "STRING",l
+      hiveType: "STRING",
       hiveOI: "StringObjectInspector",
       javaType: "",
-      drillType: "Var16Char",
+      drillType: "VarChar",
       needOIForDrillType: true
     },
     {

http://git-wip-us.apache.org/repos/asf/drill/blob/3b056db0/contrib/storage-hive/core/src/main/codegen/templates/ObjectInspectorHelper.java
----------------------------------------------------------------------
diff --git a/contrib/storage-hive/core/src/main/codegen/templates/ObjectInspectorHelper.java b/contrib/storage-hive/core/src/main/codegen/templates/ObjectInspectorHelper.java
index da83c40..d068868 100644
--- a/contrib/storage-hive/core/src/main/codegen/templates/ObjectInspectorHelper.java
+++ b/contrib/storage-hive/core/src/main/codegen/templates/ObjectInspectorHelper.java
@@ -40,30 +40,40 @@ import java.lang.UnsupportedOperationException;
 import java.sql.Timestamp;
 import java.util.HashMap;
 import java.util.Map;
+import com.google.common.collect.Multimap;
+import com.google.common.collect.ArrayListMultimap;
 
 public class ObjectInspectorHelper {
   static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(ObjectInspectorHelper.class);
 
-  private static Map<MinorType, Class> OIMAP_REQUIRED = new HashMap<>();
-  private static Map<MinorType, Class> OIMAP_OPTIONAL = new HashMap<>();
+  private static Multimap<MinorType, Class> OIMAP_REQUIRED = ArrayListMultimap.create();
+  private static Multimap<MinorType, Class> OIMAP_OPTIONAL = ArrayListMultimap.create();
   static {
 <#list drillOI.map as entry>
     <#if entry.needOIForDrillType == true>
-    OIMAP_REQUIRED.put(MinorType.${entry.drillType?upper_case}, Drill${entry.drillType}ObjectInspector.Required.class);
-    OIMAP_OPTIONAL.put(MinorType.${entry.drillType?upper_case}, Drill${entry.drillType}ObjectInspector.Optional.class);
+    OIMAP_REQUIRED.put(MinorType.${entry.drillType?upper_case}, Drill${entry.drillType}${entry.hiveOI}.Required.class);
+    OIMAP_OPTIONAL.put(MinorType.${entry.drillType?upper_case}, Drill${entry.drillType}${entry.hiveOI}.Optional.class);
     </#if>
 </#list>
   }
 
-  public static ObjectInspector getDrillObjectInspector(DataMode mode, MinorType minorType) {
+  public static ObjectInspector getDrillObjectInspector(DataMode mode, MinorType minorType, boolean varCharToStringReplacement) {
     try {
       if (mode == DataMode.REQUIRED) {
         if (OIMAP_REQUIRED.containsKey(minorType)) {
-          return (ObjectInspector) OIMAP_REQUIRED.get(minorType).newInstance();
+          if (varCharToStringReplacement && minorType == MinorType.VARCHAR) {
+            return (ObjectInspector) ((Class) OIMAP_REQUIRED.get(minorType).toArray()[1]).newInstance();
+          } else {
+            return (ObjectInspector) ((Class) OIMAP_REQUIRED.get(minorType).toArray()[0]).newInstance();
+          }
         }
       } else if (mode == DataMode.OPTIONAL) {
         if (OIMAP_OPTIONAL.containsKey(minorType)) {
-          return (ObjectInspector) OIMAP_OPTIONAL.get(minorType).newInstance();
+          if (varCharToStringReplacement && minorType == MinorType.VARCHAR) {
+            return (ObjectInspector) ((Class) OIMAP_OPTIONAL.get(minorType).toArray()[1]).newInstance();
+          } else {
+            return (ObjectInspector) ((Class) OIMAP_OPTIONAL.get(minorType).toArray()[0]).newInstance();
+          }
         }
       } else {
         throw new UnsupportedOperationException("Repeated types are not supported as arguement to Hive UDFs");
@@ -191,7 +201,7 @@ public class ObjectInspectorHelper {
           <#elseif entry.hiveType == "STRING">
             JVar data = jc._else().decl(m.directClass(byte[].class.getCanonicalName()), "data",
               castedOI.invoke("getPrimitiveJavaObject").arg(returnValue)
-                      .invoke("getBytes").arg(DirectExpression.direct("com.google.common.base.Charsets.UTF_16")));
+                      .invoke("getBytes"));
             jc._else().add(returnValueHolder.ref("buffer")
               .invoke("setBytes").arg(JExpr.lit(0)).arg(data));
             jc._else().assign(returnValueHolder.ref("start"), JExpr.lit(0));

http://git-wip-us.apache.org/repos/asf/drill/blob/3b056db0/contrib/storage-hive/core/src/main/codegen/templates/ObjectInspectors.java
----------------------------------------------------------------------
diff --git a/contrib/storage-hive/core/src/main/codegen/templates/ObjectInspectors.java b/contrib/storage-hive/core/src/main/codegen/templates/ObjectInspectors.java
index 5c696f9..ffd3a56 100644
--- a/contrib/storage-hive/core/src/main/codegen/templates/ObjectInspectors.java
+++ b/contrib/storage-hive/core/src/main/codegen/templates/ObjectInspectors.java
@@ -19,7 +19,7 @@
 
 <#list drillOI.map as entry>
 <#if entry.needOIForDrillType == true>
-<@pp.changeOutputFile name="/org/apache/drill/exec/expr/fn/impl/hive/Drill${entry.drillType}ObjectInspector.java" />
+<@pp.changeOutputFile name="/org/apache/drill/exec/expr/fn/impl/hive/Drill${entry.drillType}${entry.hiveOI}.java" />
 
 <#include "/@includes/license.ftl" />
 
@@ -47,7 +47,7 @@ import org.apache.hadoop.io.IntWritable;
 import org.apache.hadoop.io.LongWritable;
 import org.apache.hadoop.io.Text;
 
-public class Drill${entry.drillType}ObjectInspector {
+public class Drill${entry.drillType}${entry.hiveOI} {
 <#assign seq = ["Required", "Optional"]>
 <#list seq as mode>
 
@@ -56,7 +56,7 @@ public class Drill${entry.drillType}ObjectInspector {
       super(TypeInfoFactory.${entry.hiveType?lower_case}TypeInfo);
     }
 
-<#if entry.drillType == "VarChar">
+<#if entry.drillType == "VarChar" && entry.hiveType == "VARCHAR">
     @Override
     public HiveVarcharWritable getPrimitiveWritableObject(Object o) {
     <#if mode == "Optional">
@@ -85,18 +85,18 @@ public class Drill${entry.drillType}ObjectInspector {
       final String s = StringFunctionHelpers.toStringFromUTF8(h.start, h.end, h.buffer);
       return new HiveVarchar(s, HiveVarchar.MAX_VARCHAR_LENGTH);
     }
-<#elseif entry.drillType == "Var16Char">
+<#elseif entry.drillType == "VarChar" && entry.hiveType == "STRING">
     @Override
     public Text getPrimitiveWritableObject(Object o) {
     <#if mode == "Optional">
       if (o == null) {
         return null;
       }
-      final NullableVar16CharHolder h = (NullableVar16CharHolder)o;
+      final NullableVarCharHolder h = (NullableVarCharHolder)o;
     <#else>
-      final Var16CharHolder h = (Var16CharHolder)o;
+      final VarCharHolder h = (VarCharHolder)o;
     </#if>
-      return new Text(StringFunctionHelpers.toStringFromUTF16(h.start, h.end, h.buffer));
+      return new Text(StringFunctionHelpers.toStringFromUTF8(h.start, h.end, h.buffer));
     }
 
     @Override
@@ -105,11 +105,11 @@ public class Drill${entry.drillType}ObjectInspector {
       if (o == null) {
         return null;
       }
-      final NullableVar16CharHolder h = (NullableVar16CharHolder)o;
+      final NullableVarCharHolder h = (NullableVarCharHolder)o;
     <#else>
-      final Var16CharHolder h = (Var16CharHolder)o;
+      final VarCharHolder h = (VarCharHolder)o;
     </#if>
-      return StringFunctionHelpers.toStringFromUTF16(h.start, h.end, h.buffer);
+      return StringFunctionHelpers.toStringFromUTF8(h.start, h.end, h.buffer);
     }
 <#elseif entry.drillType == "VarBinary">
     @Override

http://git-wip-us.apache.org/repos/asf/drill/blob/3b056db0/contrib/storage-hive/core/src/main/java/org/apache/drill/exec/expr/fn/HiveFuncHolder.java
----------------------------------------------------------------------
diff --git a/contrib/storage-hive/core/src/main/java/org/apache/drill/exec/expr/fn/HiveFuncHolder.java b/contrib/storage-hive/core/src/main/java/org/apache/drill/exec/expr/fn/HiveFuncHolder.java
index a6e76b0..0a3cf18 100644
--- a/contrib/storage-hive/core/src/main/java/org/apache/drill/exec/expr/fn/HiveFuncHolder.java
+++ b/contrib/storage-hive/core/src/main/java/org/apache/drill/exec/expr/fn/HiveFuncHolder.java
@@ -45,6 +45,7 @@ import com.sun.codemodel.JExpr;
 import com.sun.codemodel.JInvocation;
 import com.sun.codemodel.JTryBlock;
 import com.sun.codemodel.JVar;
+import org.apache.hadoop.hive.serde2.objectinspector.PrimitiveObjectInspector;
 
 public class HiveFuncHolder extends AbstractFuncHolder {
   static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(FunctionImplementationRegistry.class);
@@ -188,7 +189,9 @@ public class HiveFuncHolder extends AbstractFuncHolder {
         oiArray.component(JExpr.lit(i)),
         oih.staticInvoke("getDrillObjectInspector")
           .arg(mode.staticInvoke("valueOf").arg(JExpr.lit(argTypes[i].getMode().getNumber())))
-          .arg(mt.staticInvoke("valueOf").arg(JExpr.lit(argTypes[i].getMinorType().getNumber()))));
+          .arg(mt.staticInvoke("valueOf").arg(JExpr.lit(argTypes[i].getMinorType().getNumber())))
+          .arg((((PrimitiveObjectInspector) returnOI).getPrimitiveCategory() ==
+              PrimitiveObjectInspector.PrimitiveCategory.STRING) ? JExpr.lit(true) : JExpr.lit(false)));
     }
 
     // declare and instantiate DeferredObject array

http://git-wip-us.apache.org/repos/asf/drill/blob/3b056db0/contrib/storage-hive/core/src/main/java/org/apache/drill/exec/expr/fn/HiveFunctionRegistry.java
----------------------------------------------------------------------
diff --git a/contrib/storage-hive/core/src/main/java/org/apache/drill/exec/expr/fn/HiveFunctionRegistry.java b/contrib/storage-hive/core/src/main/java/org/apache/drill/exec/expr/fn/HiveFunctionRegistry.java
index 5e74f6f..c716e9e 100644
--- a/contrib/storage-hive/core/src/main/java/org/apache/drill/exec/expr/fn/HiveFunctionRegistry.java
+++ b/contrib/storage-hive/core/src/main/java/org/apache/drill/exec/expr/fn/HiveFunctionRegistry.java
@@ -126,23 +126,21 @@ public class HiveFunctionRegistry implements PluggableFunctionRegistry{
 
   /**
    * Helper method which resolves the given function call to a Hive UDF. It takes an argument
-   * <i>convertVarCharToVar16Char</i> which tells to implicitly cast input arguments of type VARCHAR to VAR16CHAR
-   * and search Hive UDF registry using implicitly casted argument types.
+   * <i>varCharToStringReplacement</i> which tells to use hive STRING(true) or VARCHAR(false) type for drill VARCHAR type
+   * and search Hive UDF registry using this replacement.
    *
    * TODO: This is a rudimentary function resolver. Need to include more implicit casting such as DECIMAL28 to
    * DECIMAL38 as Hive UDFs can accept only DECIMAL38 type.
    */
-  private HiveFuncHolder resolveFunction(FunctionCall call, boolean convertVarCharToVar16Char) {
+  private HiveFuncHolder resolveFunction(FunctionCall call, boolean varCharToStringReplacement) {
     HiveFuncHolder holder;
     MajorType[] argTypes = new MajorType[call.args.size()];
     ObjectInspector[] argOIs = new ObjectInspector[call.args.size()];
     for (int i=0; i<call.args.size(); i++) {
       try {
         argTypes[i] = call.args.get(i).getMajorType();
-        if (convertVarCharToVar16Char && argTypes[i].getMinorType() == MinorType.VARCHAR) {
-          argTypes[i] = Types.withMode(MinorType.VAR16CHAR, argTypes[i].getMode());
-        }
-        argOIs[i] = ObjectInspectorHelper.getDrillObjectInspector(argTypes[i].getMode(), argTypes[i].getMinorType());
+        argOIs[i] = ObjectInspectorHelper.getDrillObjectInspector(argTypes[i].getMode(), argTypes[i].getMinorType(),
+            varCharToStringReplacement);
       } catch(Exception e) {
         // Hive throws errors if there are unsupported types. Consider there is no hive UDF supporting the
         // given argument types

http://git-wip-us.apache.org/repos/asf/drill/blob/3b056db0/contrib/storage-hive/core/src/test/java/org/apache/drill/exec/fn/hive/TestHiveUDFs.java
----------------------------------------------------------------------
diff --git a/contrib/storage-hive/core/src/test/java/org/apache/drill/exec/fn/hive/TestHiveUDFs.java b/contrib/storage-hive/core/src/test/java/org/apache/drill/exec/fn/hive/TestHiveUDFs.java
index 3ce9a6d..2ad8edc 100644
--- a/contrib/storage-hive/core/src/test/java/org/apache/drill/exec/fn/hive/TestHiveUDFs.java
+++ b/contrib/storage-hive/core/src/test/java/org/apache/drill/exec/fn/hive/TestHiveUDFs.java
@@ -25,11 +25,12 @@ import java.util.List;
 import org.apache.drill.BaseTestQuery;
 import org.apache.drill.exec.record.RecordBatchLoader;
 import org.apache.drill.exec.rpc.user.QueryDataBatch;
+import org.apache.drill.exec.vector.BigIntVector;
 import org.apache.drill.exec.vector.Float4Vector;
+import org.apache.drill.exec.vector.IntVector;
 import org.apache.drill.exec.vector.NullableFloat8Vector;
-import org.apache.drill.exec.vector.NullableIntVector;
-import org.apache.drill.exec.vector.NullableVar16CharVector;
-import org.apache.drill.exec.vector.Var16CharVector;
+import org.apache.drill.exec.vector.NullableVarCharVector;
+import org.apache.drill.exec.vector.VarCharVector;
 import org.junit.Test;
 
 import com.google.common.base.Charsets;
@@ -53,42 +54,42 @@ public class TestHiveUDFs extends BaseTestQuery {
         continue;
       }
       // Output columns and types
-      //  1. str1 : Var16Char
-      //  2. upperStr1 : NullableVar16Char
+      //  1. str1 : VarChar
+      //  2. upperStr1 : NullableVarChar
       //  3. concat : NullableVarChar
       //  4. flt1 : Float4
       //  5. format_number : NullableFloat8
-      //  6. nullableStr1 : NullableVar16Char
-      //  7. upperNullableStr1 : NullableVar16Char
-      Var16CharVector str1V = (Var16CharVector) batchLoader.getValueAccessorById(Var16CharVector.class, 0).getValueVector();
-      NullableVar16CharVector upperStr1V = (NullableVar16CharVector) batchLoader.getValueAccessorById(NullableVar16CharVector.class, 1).getValueVector();
-      NullableVar16CharVector concatV = (NullableVar16CharVector) batchLoader.getValueAccessorById(NullableVar16CharVector.class, 2).getValueVector();
+      //  6. nullableStr1 : NullableVarChar
+      //  7. upperNullableStr1 : NullableVarChar
+      VarCharVector str1V = (VarCharVector) batchLoader.getValueAccessorById(VarCharVector.class, 0).getValueVector();
+      NullableVarCharVector upperStr1V = (NullableVarCharVector) batchLoader.getValueAccessorById(NullableVarCharVector.class, 1).getValueVector();
+      NullableVarCharVector concatV = (NullableVarCharVector) batchLoader.getValueAccessorById(NullableVarCharVector.class, 2).getValueVector();
       Float4Vector flt1V = (Float4Vector) batchLoader.getValueAccessorById(Float4Vector.class, 3).getValueVector();
-      NullableVar16CharVector format_numberV = (NullableVar16CharVector) batchLoader.getValueAccessorById(NullableVar16CharVector.class, 4).getValueVector();
-      NullableVar16CharVector nullableStr1V = (NullableVar16CharVector) batchLoader.getValueAccessorById(NullableVar16CharVector.class, 5).getValueVector();
-      NullableVar16CharVector upperNullableStr1V = (NullableVar16CharVector) batchLoader.getValueAccessorById(NullableVar16CharVector.class, 6).getValueVector();
+      NullableVarCharVector format_numberV = (NullableVarCharVector) batchLoader.getValueAccessorById(NullableVarCharVector.class, 4).getValueVector();
+      NullableVarCharVector nullableStr1V = (NullableVarCharVector) batchLoader.getValueAccessorById(NullableVarCharVector.class, 5).getValueVector();
+      NullableVarCharVector upperNullableStr1V = (NullableVarCharVector) batchLoader.getValueAccessorById(NullableVarCharVector.class, 6).getValueVector();
 
       for (int i=0; i<batchLoader.getRecordCount(); i++) {
-        String in = new String(str1V.getAccessor().get(i), Charsets.UTF_16);
-        String upper = new String(upperStr1V.getAccessor().get(i), Charsets.UTF_16);
+        String in = new String(str1V.getAccessor().get(i), Charsets.UTF_8);
+        String upper = new String(upperStr1V.getAccessor().get(i), Charsets.UTF_8);
         assertTrue(in.toUpperCase().equals(upper));
 
 
-        String concat = new String(concatV.getAccessor().get(i), Charsets.UTF_16);
+        String concat = new String(concatV.getAccessor().get(i), Charsets.UTF_8);
         assertTrue(concat.equals(in+"-"+in));
 
         float flt1 = flt1V.getAccessor().get(i);
-        String format_number = new String(format_numberV.getAccessor().get(i), Charsets.UTF_16);
+        String format_number = new String(format_numberV.getAccessor().get(i), Charsets.UTF_8);
 
 
         String nullableStr1 = null;
         if (!nullableStr1V.getAccessor().isNull(i)) {
-          nullableStr1 = new String(nullableStr1V.getAccessor().get(i), Charsets.UTF_16);
+          nullableStr1 = new String(nullableStr1V.getAccessor().get(i), Charsets.UTF_8);
         }
 
         String upperNullableStr1 = null;
         if (!upperNullableStr1V.getAccessor().isNull(i)) {
-          upperNullableStr1 = new String(upperNullableStr1V.getAccessor().get(i), Charsets.UTF_16);
+          upperNullableStr1 = new String(upperNullableStr1V.getAccessor().get(i), Charsets.UTF_8);
         }
 
         assertEquals(nullableStr1 != null, upperNullableStr1 != null);
@@ -125,20 +126,20 @@ public class TestHiveUDFs extends BaseTestQuery {
       }
 
       // Output columns and types
-      // 1. str1 : Var16Char
+      // 1. str1 : VarChar
       // 2. str1Length : Int
       // 3. str1Ascii : Int
       // 4. flt1 : Float4
       // 5. pow : Float8
-      Var16CharVector str1V = (Var16CharVector) batchLoader.getValueAccessorById(Var16CharVector.class, 0).getValueVector();
-      NullableIntVector str1LengthV = (NullableIntVector) batchLoader.getValueAccessorById(NullableIntVector.class, 1).getValueVector();
-      NullableIntVector str1AsciiV = (NullableIntVector) batchLoader.getValueAccessorById(NullableIntVector.class, 2).getValueVector();
+      VarCharVector str1V = (VarCharVector) batchLoader.getValueAccessorById(VarCharVector.class, 0).getValueVector();
+      BigIntVector str1LengthV = (BigIntVector) batchLoader.getValueAccessorById(BigIntVector.class, 1).getValueVector();
+      IntVector str1AsciiV = (IntVector) batchLoader.getValueAccessorById(IntVector.class, 2).getValueVector();
       Float4Vector flt1V = (Float4Vector) batchLoader.getValueAccessorById(Float4Vector.class, 3).getValueVector();
       NullableFloat8Vector powV = (NullableFloat8Vector) batchLoader.getValueAccessorById(NullableFloat8Vector.class, 4).getValueVector();
 
       for (int i=0; i<batchLoader.getRecordCount(); i++) {
-        String str1 = new String(str1V.getAccessor().get(i), Charsets.UTF_16);
-        int str1Length = str1LengthV.getAccessor().get(i);
+        String str1 = new String(str1V.getAccessor().get(i), Charsets.UTF_8);
+        long str1Length = str1LengthV.getAccessor().get(i);
         assertTrue(str1.length() == str1Length);
 
         int str1Ascii = str1AsciiV.getAccessor().get(i);

http://git-wip-us.apache.org/repos/asf/drill/blob/3b056db0/contrib/storage-hive/core/src/test/java/org/apache/drill/exec/fn/hive/TestInbuiltHiveUDFs.java
----------------------------------------------------------------------
diff --git a/contrib/storage-hive/core/src/test/java/org/apache/drill/exec/fn/hive/TestInbuiltHiveUDFs.java b/contrib/storage-hive/core/src/test/java/org/apache/drill/exec/fn/hive/TestInbuiltHiveUDFs.java
index a126aaa..93c9374 100644
--- a/contrib/storage-hive/core/src/test/java/org/apache/drill/exec/fn/hive/TestInbuiltHiveUDFs.java
+++ b/contrib/storage-hive/core/src/test/java/org/apache/drill/exec/fn/hive/TestInbuiltHiveUDFs.java
@@ -19,9 +19,11 @@ package org.apache.drill.exec.fn.hive;
 
 import com.google.common.collect.Lists;
 import org.apache.commons.lang3.tuple.Pair;
+import org.apache.drill.TestBuilder;
 import org.apache.drill.common.expression.SchemaPath;
 import org.apache.drill.common.types.TypeProtos;
 import org.apache.drill.exec.hive.HiveTestBase;
+import org.apache.drill.exec.rpc.user.QueryDataBatch;
 import org.junit.Test;
 
 import java.util.List;
@@ -70,4 +72,16 @@ public class TestInbuiltHiveUDFs extends HiveTestBase {
         .build()
         .run();
   }
+
+  @Test // DRILL-4459
+  public void testGetJsonObject() throws Exception {
+    testBuilder()
+        .sqlQuery("select convert_from(json, 'json') as json from hive.simple_json " +
+            "where GET_JSON_OBJECT(simple_json.json, '$.employee_id') like 'Emp2'")
+        .ordered()
+        .baselineColumns("json")
+        .baselineValues(TestBuilder.mapOf("employee_id","Emp2","full_name","Kamesh",
+            "first_name","Bh","last_name","Venkata","position","Store"))
+        .go();
+  }
 }

http://git-wip-us.apache.org/repos/asf/drill/blob/3b056db0/contrib/storage-hive/core/src/test/java/org/apache/drill/exec/hive/TestInfoSchemaOnHiveStorage.java
----------------------------------------------------------------------
diff --git a/contrib/storage-hive/core/src/test/java/org/apache/drill/exec/hive/TestInfoSchemaOnHiveStorage.java b/contrib/storage-hive/core/src/test/java/org/apache/drill/exec/hive/TestInfoSchemaOnHiveStorage.java
index 03acd22..a8c6e68 100644
--- a/contrib/storage-hive/core/src/test/java/org/apache/drill/exec/hive/TestInfoSchemaOnHiveStorage.java
+++ b/contrib/storage-hive/core/src/test/java/org/apache/drill/exec/hive/TestInfoSchemaOnHiveStorage.java
@@ -42,6 +42,7 @@ public class TestInfoSchemaOnHiveStorage extends HiveTestBase {
         .baselineValues("hive.default", "kv_parquet")
         .baselineValues("hive.default", "kv_sh")
         .baselineValues("hive.default", "countstar_parquet")
+        .baselineValues("hive.default", "simple_json")
         .go();
 
     testBuilder()

http://git-wip-us.apache.org/repos/asf/drill/blob/3b056db0/contrib/storage-hive/core/src/test/java/org/apache/drill/exec/store/hive/HiveTestDataGenerator.java
----------------------------------------------------------------------
diff --git a/contrib/storage-hive/core/src/test/java/org/apache/drill/exec/store/hive/HiveTestDataGenerator.java b/contrib/storage-hive/core/src/test/java/org/apache/drill/exec/store/hive/HiveTestDataGenerator.java
index 9c2fe1c..7a5b72d 100644
--- a/contrib/storage-hive/core/src/test/java/org/apache/drill/exec/store/hive/HiveTestDataGenerator.java
+++ b/contrib/storage-hive/core/src/test/java/org/apache/drill/exec/store/hive/HiveTestDataGenerator.java
@@ -24,6 +24,7 @@ import java.sql.Date;
 import java.sql.Timestamp;
 import java.util.Map;
 
+import com.google.common.io.Resources;
 import org.apache.commons.io.FileUtils;
 import org.apache.commons.lang3.StringUtils;
 import org.apache.drill.BaseTestQuery;
@@ -504,6 +505,11 @@ public class HiveTestDataGenerator {
     executeQuery(hiveDriver, createTableWithHeaderFooterProperties("skipper.kv_sequencefile_large", "sequencefile", "1", "1"));
     executeQuery(hiveDriver, "insert into table skipper.kv_sequencefile_large select * from skipper.kv_text_large");
 
+      // Create a table based on json file
+      executeQuery(hiveDriver, "create table default.simple_json(json string)");
+      final String loadData = String.format("load data local inpath '" +
+          Resources.getResource("simple.json") + "' into table default.simple_json");
+      executeQuery(hiveDriver, loadData);
     ss.close();
   }
 

http://git-wip-us.apache.org/repos/asf/drill/blob/3b056db0/contrib/storage-hive/core/src/test/resources/functions/hive/GenericUDF.json
----------------------------------------------------------------------
diff --git a/contrib/storage-hive/core/src/test/resources/functions/hive/GenericUDF.json b/contrib/storage-hive/core/src/test/resources/functions/hive/GenericUDF.json
index f5ae299..5dc076e 100644
--- a/contrib/storage-hive/core/src/test/resources/functions/hive/GenericUDF.json
+++ b/contrib/storage-hive/core/src/test/resources/functions/hive/GenericUDF.json
@@ -13,10 +13,10 @@
             url: "http://apache.org",
             entries:[
                 {records: 100, types: [
-                   {name: "str1", type: "VAR16CHAR", mode: "REQUIRED"},
-                   {name: "str2", type: "VAR16CHAR", mode: "REQUIRED"},
-                   {name: "str3", type: "VAR16CHAR", mode: "REQUIRED"},
-                   {name: "nullableStr1", type: "VAR16CHAR", mode: "OPTIONAL"},
+                   {name: "str1", type: "VARCHAR", mode: "REQUIRED"},
+                   {name: "str2", type: "VARCHAR", mode: "REQUIRED"},
+                   {name: "str3", type: "VARCHAR", mode: "REQUIRED"},
+                   {name: "nullableStr1", type: "VARCHAR", mode: "OPTIONAL"},
                    {name: "flt1", type: "FLOAT4", mode: "REQUIRED"}
                 ]}
             ]

http://git-wip-us.apache.org/repos/asf/drill/blob/3b056db0/contrib/storage-hive/core/src/test/resources/functions/hive/UDF.json
----------------------------------------------------------------------
diff --git a/contrib/storage-hive/core/src/test/resources/functions/hive/UDF.json b/contrib/storage-hive/core/src/test/resources/functions/hive/UDF.json
index c2c7169..fecabec 100644
--- a/contrib/storage-hive/core/src/test/resources/functions/hive/UDF.json
+++ b/contrib/storage-hive/core/src/test/resources/functions/hive/UDF.json
@@ -13,7 +13,7 @@
             url: "http://apache.org",
             entries:[
                 {records: 100, types: [
-                   {name: "str1", type: "VAR16CHAR", mode: "REQUIRED"},
+                   {name: "str1", type: "VARCHAR", mode: "REQUIRED"},
                    {name: "flt1", type: "FLOAT4", mode: "REQUIRED"}
                 ]}
             ]

http://git-wip-us.apache.org/repos/asf/drill/blob/3b056db0/contrib/storage-hive/core/src/test/resources/simple.json
----------------------------------------------------------------------
diff --git a/contrib/storage-hive/core/src/test/resources/simple.json b/contrib/storage-hive/core/src/test/resources/simple.json
new file mode 100644
index 0000000..0bf69ce
--- /dev/null
+++ b/contrib/storage-hive/core/src/test/resources/simple.json
@@ -0,0 +1,19 @@
+[ {"employee_id":"Emp1101","full_name":"Steve Eurich","first_name":"Steve","last_name":"Eurich","position":"Store T"},
+  {"employee_id":"Emp1102","full_name":"Mary Pierson","first_name":"Mary","last_name":"Pierson","position":"Store T"},
+  {"employee_id":"Emp1103","full_name":"Leo Jones","first_name":"Leo","last_name":"Jones","position":"Store Tem"},
+  {"employee_id":"Emp1104","full_name":"Nancy Beatty","first_name":"Nancy","last_name":"Beatty","position":"Store T"},
+  {"employee_id":"Emp1105","full_name":"Clara McNight","first_name":"Clara","last_name":"McNight","position":"Store"},
+  {"employee_id":"Emp1106","first_name":"Marcella","last_name":"Isaacs","position_id":17,"position":"Stor"},
+  {"employee_id":"Emp1107","full_name":"Charlotte Yonce","first_name":"Charlotte","last_name":"Yonce","position":"Stor"},
+  {"employee_id":"Emp1108","full_name":"Benjamin Foster","first_name":"Benjamin","last_name":"Foster","position":"Stor"},
+  {"employee_id":"Emp1109","full_name":"John Reed","first_name":"John","last_name":"Reed","position":"Store Per"},
+  {"employee_id":"Emp1110","full_name":"Lynn Kwiatkowski","first_name":"Lynn","last_name":"Kwiatkowski","position":"St"},
+  {"employee_id":"Emp1111","full_name":"Donald Vann","first_name":"Donald","last_name":"Vann","position":"Store Per"},
+  {"employee_id":"Emp1112","first_name":"William","last_name":"Smith","position":"St"},
+  {"employee_id":"Emp1113","full_name":"Amy Hensley","first_name":"Amy","last_name":"Hensley","position":"Store Pe"},
+  {"employee_id":"Emp1114","full_name":"Judy Owens","first_name":"Judy","last_name":"Owens","position":"Store Per"},
+  {"employee_id":"Emp1115","full_name":"Frederick Castillo","first_name":"Frederick","last_name":"Castillo","position":"S"},
+  {"employee_id":"Emp1116","full_name":"Phil Munoz","first_name":"Phil","last_name":"Munoz","position":"Store Per"},
+  {"employee_id":"Emp1117","full_name":"Lori Lightfoot","first_name":"Lori","last_name":"Lightfoot","position":"Store"},
+  {"employee_id":"Emp1","full_name":"Kumar","first_name":"Anil","last_name":"B","position":"Store"},
+  {"employee_id":"Emp2","full_name":"Kamesh","first_name":"Bh","last_name":"Venkata","position":"Store"} ]


[2/3] drill git commit: DRILL-4237, DRILL-4478: Implement hash to use murmur3 and add correspondent unit tests

Posted by su...@apache.org.
DRILL-4237, DRILL-4478: Implement hash to use murmur3 and add correspondent unit tests

+ Avoid object or extra buffer creation
+ Clean up tests

closes #485


Project: http://git-wip-us.apache.org/repos/asf/drill/repo
Commit: http://git-wip-us.apache.org/repos/asf/drill/commit/c6a03eb1
Tree: http://git-wip-us.apache.org/repos/asf/drill/tree/c6a03eb1
Diff: http://git-wip-us.apache.org/repos/asf/drill/diff/c6a03eb1

Branch: refs/heads/master
Commit: c6a03eb1708f3cea6cea2f9530057ad707c374c0
Parents: 3b056db
Author: chunhui-shi <cs...@maprtech.com>
Authored: Tue Mar 15 10:36:54 2016 -0700
Committer: Sudheesh Katkam <sk...@maprtech.com>
Committed: Tue Apr 19 21:58:16 2016 -0700

----------------------------------------------------------------------
 .../drill/common/util/DrillStringUtils.java     |   4 +-
 .../drill/exec/expr/fn/impl/DrillHash.java      |  50 ++++
 .../drill/exec/expr/fn/impl/Hash32AsDouble.java |  33 +--
 .../exec/expr/fn/impl/Hash32Functions.java      |  60 ++--
 .../expr/fn/impl/Hash32FunctionsWithSeed.java   |  63 +++--
 .../expr/fn/impl/Hash32WithSeedAsDouble.java    |  32 +--
 .../drill/exec/expr/fn/impl/Hash64AsDouble.java |  32 +--
 .../exec/expr/fn/impl/Hash64Functions.java      |  60 ++--
 .../expr/fn/impl/Hash64FunctionsWithSeed.java   |  60 ++--
 .../expr/fn/impl/Hash64WithSeedAsDouble.java    |  32 +--
 .../drill/exec/expr/fn/impl/HashHelper.java     |  45 ++-
 .../drill/exec/expr/fn/impl/MurmurHash3.java    | 280 +++++++++++++++++++
 .../apache/drill/exec/expr/fn/impl/XXHash.java  |  80 ++----
 .../org/apache/drill/TestFunctionsQuery.java    |  69 +++--
 14 files changed, 638 insertions(+), 262 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/drill/blob/c6a03eb1/common/src/main/java/org/apache/drill/common/util/DrillStringUtils.java
----------------------------------------------------------------------
diff --git a/common/src/main/java/org/apache/drill/common/util/DrillStringUtils.java b/common/src/main/java/org/apache/drill/common/util/DrillStringUtils.java
index b016184..4dad397 100644
--- a/common/src/main/java/org/apache/drill/common/util/DrillStringUtils.java
+++ b/common/src/main/java/org/apache/drill/common/util/DrillStringUtils.java
@@ -163,10 +163,10 @@ public class DrillStringUtils {
   public static int parseBinaryString(ByteBuf str, int strStart, int strEnd) {
     int length = (strEnd - strStart);
     int dstEnd = strStart;
-    for (int i = strStart; i < length ; i++) {
+    for (int i = strStart; i < strStart+length ; i++) {
       byte b = str.getByte(i);
       if (b == '\\'
-          && length > i+3
+          && strEnd > i+3
           && (str.getByte(i+1) == 'x' || str.getByte(i+1) == 'X')) {
         // ok, take next 2 hex digits.
         byte hd1 = str.getByte(i+2);

http://git-wip-us.apache.org/repos/asf/drill/blob/c6a03eb1/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/impl/DrillHash.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/impl/DrillHash.java b/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/impl/DrillHash.java
new file mode 100644
index 0000000..8a92b81
--- /dev/null
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/impl/DrillHash.java
@@ -0,0 +1,50 @@
+/**
+ * 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.drill.exec.expr.fn.impl;
+
+import io.netty.buffer.DrillBuf;
+import io.netty.util.internal.PlatformDependent;
+
+/**
+ * The base class of hash classes used in Drill.
+ */
+public class DrillHash {
+
+    public static final long getLongLittleEndian(long offset) {
+        //return PlatformDependent.getLong(offset);
+        return     ((long) PlatformDependent.getByte(offset+7)    << 56)
+                | ((PlatformDependent.getByte(offset+6) & 0xffL) << 48)
+                | ((PlatformDependent.getByte(offset+5) & 0xffL) << 40)
+                | ((PlatformDependent.getByte(offset+4) & 0xffL) << 32)
+                | ((PlatformDependent.getByte(offset+3) & 0xffL) << 24)
+                | ((PlatformDependent.getByte(offset+2) & 0xffL) << 16)
+                | ((PlatformDependent.getByte(offset+1) & 0xffL) << 8)
+                | ((PlatformDependent.getByte(offset) & 0xffL));
+    }
+
+    public static final long getIntLittleEndian(long offset) {
+        long retl = 0;
+        retl = ((PlatformDependent.getByte(offset+3) &0xffL) << 24)
+                | ((PlatformDependent.getByte(offset+2) & 0xffL) << 16)
+                | ((PlatformDependent.getByte(offset+1) & 0xffL) << 8)
+                | ((PlatformDependent.getByte(offset) & 0xffL));
+        return retl;
+    }
+
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/drill/blob/c6a03eb1/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/impl/Hash32AsDouble.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/impl/Hash32AsDouble.java b/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/impl/Hash32AsDouble.java
index 5a8e10a..0d912e8 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/impl/Hash32AsDouble.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/impl/Hash32AsDouble.java
@@ -45,6 +45,7 @@ import org.apache.drill.exec.expr.holders.NullableIntHolder;
  */
 public class Hash32AsDouble {
   @FunctionTemplate(name = "hash32AsDouble", scope = FunctionScope.SIMPLE, nulls = FunctionTemplate.NullHandling.INTERNAL)
+
   public static class NullableFloatHash implements DrillSimpleFunc {
 
     @Param
@@ -59,7 +60,7 @@ public class Hash32AsDouble {
       if (in.isSet == 0) {
         out.value = 0;
       } else {
-        out.value = org.apache.drill.exec.expr.fn.impl.XXHash.hash32((double) in.value, 0);
+        out.value = org.apache.drill.exec.expr.fn.impl.HashHelper.hash32(in.value, 0);
       }
     }
   }
@@ -76,7 +77,7 @@ public class Hash32AsDouble {
     }
 
     public void eval() {
-      out.value = org.apache.drill.exec.expr.fn.impl.XXHash.hash32((double) in.value, 0);
+      out.value = org.apache.drill.exec.expr.fn.impl.HashHelper.hash32(in.value, 0);
     }
   }
 
@@ -95,7 +96,7 @@ public class Hash32AsDouble {
       if (in.isSet == 0) {
         out.value = 0;
       } else {
-        out.value = org.apache.drill.exec.expr.fn.impl.XXHash.hash32((double) in.value, 0);
+        out.value = org.apache.drill.exec.expr.fn.impl.HashHelper.hash32(in.value, 0);
       }
     }
   }
@@ -112,7 +113,7 @@ public class Hash32AsDouble {
     }
 
     public void eval() {
-      out.value = org.apache.drill.exec.expr.fn.impl.XXHash.hash32(in.value, 0);
+      out.value = org.apache.drill.exec.expr.fn.impl.HashHelper.hash32(in.value, 0);
     }
   }
 
@@ -131,7 +132,7 @@ public class Hash32AsDouble {
       if (in.isSet == 0) {
         out.value = 0;
       } else {
-        out.value = org.apache.drill.exec.expr.fn.impl.XXHash.hash32((double) in.value, 0);
+        out.value = org.apache.drill.exec.expr.fn.impl.HashHelper.hash32((long) in.value, 0);
       }
     }
   }
@@ -150,7 +151,7 @@ public class Hash32AsDouble {
       if (in.isSet == 0) {
         out.value = 0;
       } else {
-        out.value = org.apache.drill.exec.expr.fn.impl.XXHash.hash32((double) in.value, 0);
+        out.value = org.apache.drill.exec.expr.fn.impl.HashHelper.hash32(in.value, 0);
       }
     }
   }
@@ -167,7 +168,7 @@ public class Hash32AsDouble {
     }
 
     public void eval() {
-      out.value = org.apache.drill.exec.expr.fn.impl.XXHash.hash32((double) in.value, 0);
+      out.value = org.apache.drill.exec.expr.fn.impl.HashHelper.hash32((long)in.value, 0);
     }
   }
 
@@ -182,7 +183,7 @@ public class Hash32AsDouble {
     }
 
     public void eval() {
-      out.value = org.apache.drill.exec.expr.fn.impl.XXHash.hash32((double) in.value, 0);
+      out.value = org.apache.drill.exec.expr.fn.impl.HashHelper.hash32(in.value, 0);
     }
   }
 
@@ -198,7 +199,7 @@ public class Hash32AsDouble {
 
     public void eval() {
       java.math.BigDecimal input = new java.math.BigDecimal(java.math.BigInteger.valueOf(in.value), in.scale);
-      out.value = org.apache.drill.exec.expr.fn.impl.XXHash.hash32(input.doubleValue(), 0);
+      out.value = org.apache.drill.exec.expr.fn.impl.HashHelper.hash32(input.doubleValue(), 0);
     }
   }
 
@@ -217,7 +218,7 @@ public class Hash32AsDouble {
         out.value = 0;
       } else {
         java.math.BigDecimal input = new java.math.BigDecimal(java.math.BigInteger.valueOf(in.value), in.scale);
-        out.value = org.apache.drill.exec.expr.fn.impl.XXHash.hash32(input.doubleValue(), 0);
+        out.value = org.apache.drill.exec.expr.fn.impl.HashHelper.hash32(input.doubleValue(), 0);
       }
     }
   }
@@ -234,7 +235,7 @@ public class Hash32AsDouble {
 
     public void eval() {
       java.math.BigDecimal input = new java.math.BigDecimal(java.math.BigInteger.valueOf(in.value), in.scale);
-      out.value = org.apache.drill.exec.expr.fn.impl.XXHash.hash32(input.doubleValue(), 0);
+      out.value = org.apache.drill.exec.expr.fn.impl.HashHelper.hash32(input.doubleValue(), 0);
     }
   }
 
@@ -253,7 +254,7 @@ public class Hash32AsDouble {
         out.value = 0;
       } else {
         java.math.BigDecimal input = new java.math.BigDecimal(java.math.BigInteger.valueOf(in.value), in.scale);
-        out.value = org.apache.drill.exec.expr.fn.impl.XXHash.hash32(input.doubleValue(), 0);
+        out.value = org.apache.drill.exec.expr.fn.impl.HashHelper.hash32(input.doubleValue(), 0);
       }
     }
   }
@@ -271,7 +272,7 @@ public class Hash32AsDouble {
     public void eval() {
       java.math.BigDecimal input = org.apache.drill.exec.util.DecimalUtility.getBigDecimalFromSparse(in.buffer,
           in.start, in.nDecimalDigits, in.scale);
-      out.value = org.apache.drill.exec.expr.fn.impl.XXHash.hash32(input.doubleValue(), 0);
+      out.value = org.apache.drill.exec.expr.fn.impl.HashHelper.hash32(input.doubleValue(), 0);
     }
   }
 
@@ -291,7 +292,7 @@ public class Hash32AsDouble {
       } else {
         java.math.BigDecimal input = org.apache.drill.exec.util.DecimalUtility.getBigDecimalFromSparse(in.buffer,
             in.start, in.nDecimalDigits, in.scale);
-        out.value = org.apache.drill.exec.expr.fn.impl.XXHash.hash32(input.doubleValue(), 0);
+        out.value = org.apache.drill.exec.expr.fn.impl.HashHelper.hash32(input.doubleValue(), 0);
       }
     }
   }
@@ -309,7 +310,7 @@ public class Hash32AsDouble {
     public void eval() {
       java.math.BigDecimal input = org.apache.drill.exec.util.DecimalUtility.getBigDecimalFromSparse(in.buffer,
           in.start, in.nDecimalDigits, in.scale);
-      out.value = org.apache.drill.exec.expr.fn.impl.XXHash.hash32(input.doubleValue(), 0);
+      out.value = org.apache.drill.exec.expr.fn.impl.HashHelper.hash32(input.doubleValue(), 0);
     }
   }
 
@@ -329,7 +330,7 @@ public class Hash32AsDouble {
       } else {
         java.math.BigDecimal input = org.apache.drill.exec.util.DecimalUtility.getBigDecimalFromSparse(in.buffer,
             in.start, in.nDecimalDigits, in.scale);
-        out.value = org.apache.drill.exec.expr.fn.impl.XXHash.hash32(input.doubleValue(), 0);
+        out.value = org.apache.drill.exec.expr.fn.impl.HashHelper.hash32(input.doubleValue(), 0);
       }
     }
   }

http://git-wip-us.apache.org/repos/asf/drill/blob/c6a03eb1/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/impl/Hash32Functions.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/impl/Hash32Functions.java b/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/impl/Hash32Functions.java
index f4b0c56..bb08197 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/impl/Hash32Functions.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/impl/Hash32Functions.java
@@ -68,7 +68,7 @@ public class Hash32Functions {
       if (in.isSet == 0) {
         out.value = 0;
       } else {
-        out.value = org.apache.drill.exec.expr.fn.impl.XXHash.hash32(in.value, 0);
+        out.value = org.apache.drill.exec.expr.fn.impl.HashHelper.hash32(in.value, 0);
       }
     }
   }
@@ -83,7 +83,7 @@ public class Hash32Functions {
     }
 
     public void eval() {
-      out.value = org.apache.drill.exec.expr.fn.impl.XXHash.hash32(in.value, 0);
+      out.value = org.apache.drill.exec.expr.fn.impl.HashHelper.hash32(in.value, 0);
     }
   }
 
@@ -100,7 +100,7 @@ public class Hash32Functions {
       if (in.isSet == 0) {
         out.value = 0;
       } else {
-        out.value = org.apache.drill.exec.expr.fn.impl.XXHash.hash32(in.value, 0);
+        out.value = org.apache.drill.exec.expr.fn.impl.HashHelper.hash32(in.value, 0);
       }
     }
   }
@@ -115,7 +115,7 @@ public class Hash32Functions {
     }
 
     public void eval() {
-      out.value = org.apache.drill.exec.expr.fn.impl.XXHash.hash32(in.value, 0);
+      out.value = org.apache.drill.exec.expr.fn.impl.HashHelper.hash32(in.value, 0);
     }
   }
 
@@ -132,7 +132,7 @@ public class Hash32Functions {
       if (in.isSet == 0) {
         out.value = 0;
       } else {
-        out.value = org.apache.drill.exec.expr.fn.impl.XXHash.hash32(in.start, in.end, in.buffer, 0);
+        out.value = org.apache.drill.exec.expr.fn.impl.HashHelper.hash32(in.start, in.end, in.buffer, 0);
       }
     }
   }
@@ -150,7 +150,7 @@ public class Hash32Functions {
       if (in.isSet == 0) {
         out.value = 0;
       } else {
-        out.value = org.apache.drill.exec.expr.fn.impl.XXHash.hash32(in.start, in.end, in.buffer, 0);
+        out.value = org.apache.drill.exec.expr.fn.impl.HashHelper.hash32(in.start, in.end, in.buffer, 0);
       }
     }
   }
@@ -168,7 +168,7 @@ public class Hash32Functions {
       if (in.isSet == 0) {
         out.value = 0;
       } else {
-        out.value = org.apache.drill.exec.expr.fn.impl.XXHash.hash32(in.start, in.end, in.buffer, 0);
+        out.value = org.apache.drill.exec.expr.fn.impl.HashHelper.hash32(in.start, in.end, in.buffer, 0);
       }
     }
   }
@@ -187,7 +187,7 @@ public class Hash32Functions {
         out.value = 0;
       }
       else {
-        out.value = org.apache.drill.exec.expr.fn.impl.XXHash.hash32(in.value, 0);
+        out.value = org.apache.drill.exec.expr.fn.impl.HashHelper.hash32(in.value, 0);
       }
     }
   }
@@ -205,7 +205,7 @@ public class Hash32Functions {
         out.value = 0;
       }
       else {
-        out.value = org.apache.drill.exec.expr.fn.impl.XXHash.hash32(in.value, 0);
+        out.value = org.apache.drill.exec.expr.fn.impl.HashHelper.hash32(in.value, 0);
       }
     }
   }
@@ -220,7 +220,7 @@ public class Hash32Functions {
     }
 
     public void eval() {
-      out.value = org.apache.drill.exec.expr.fn.impl.XXHash.hash32(in.start, in.end, in.buffer, 0);
+      out.value = org.apache.drill.exec.expr.fn.impl.HashHelper.hash32(in.start, in.end, in.buffer, 0);
     }
   }
 
@@ -234,7 +234,7 @@ public class Hash32Functions {
     }
 
     public void eval() {
-      out.value = org.apache.drill.exec.expr.fn.impl.XXHash.hash32(in.start, in.end, in.buffer, 0);
+      out.value = org.apache.drill.exec.expr.fn.impl.HashHelper.hash32(in.start, in.end, in.buffer, 0);
     }
   }
 
@@ -248,7 +248,7 @@ public class Hash32Functions {
     }
 
     public void eval() {
-      out.value = org.apache.drill.exec.expr.fn.impl.XXHash.hash32(in.start, in.end, in.buffer, 0);
+      out.value = org.apache.drill.exec.expr.fn.impl.HashHelper.hash32(in.start, in.end, in.buffer, 0);
     }
   }
 
@@ -262,7 +262,7 @@ public class Hash32Functions {
     }
 
     public void eval() {
-      out.value = org.apache.drill.exec.expr.fn.impl.XXHash.hash32(in.value, 0);
+      out.value = org.apache.drill.exec.expr.fn.impl.HashHelper.hash32(in.value, 0);
     }
   }
 
@@ -275,7 +275,7 @@ public class Hash32Functions {
     }
 
     public void eval() {
-      out.value = org.apache.drill.exec.expr.fn.impl.XXHash.hash32(in.value, 0);
+      out.value = org.apache.drill.exec.expr.fn.impl.HashHelper.hash32(in.value, 0);
     }
   }
   @FunctionTemplate(names = {"hash", "hash32", "hash32AsDouble"}, scope = FunctionScope.SIMPLE, nulls = FunctionTemplate.NullHandling.INTERNAL)
@@ -287,7 +287,7 @@ public class Hash32Functions {
     }
 
     public void eval() {
-      out.value = org.apache.drill.exec.expr.fn.impl.XXHash.hash32(in.value, 0);
+      out.value = org.apache.drill.exec.expr.fn.impl.HashHelper.hash32(in.value, 0);
     }
   }
 
@@ -303,7 +303,7 @@ public class Hash32Functions {
       if (in.isSet == 0) {
         out.value = 0;
       } else {
-        out.value = org.apache.drill.exec.expr.fn.impl.XXHash.hash32(in.value, 0);
+        out.value = org.apache.drill.exec.expr.fn.impl.HashHelper.hash32(in.value, 0);
       }
     }
   }
@@ -317,7 +317,7 @@ public class Hash32Functions {
     }
 
     public void eval() {
-      out.value = org.apache.drill.exec.expr.fn.impl.XXHash.hash32(in.value, 0);
+      out.value = org.apache.drill.exec.expr.fn.impl.HashHelper.hash32(in.value, 0);
     }
   }
 
@@ -333,7 +333,7 @@ public class Hash32Functions {
       if (in.isSet == 0) {
         out.value = 0;
       } else {
-        out.value = org.apache.drill.exec.expr.fn.impl.XXHash.hash32(in.value, 0);
+        out.value = org.apache.drill.exec.expr.fn.impl.HashHelper.hash32(in.value, 0);
       }
     }
   }
@@ -347,7 +347,7 @@ public class Hash32Functions {
     }
 
     public void eval() {
-      out.value = org.apache.drill.exec.expr.fn.impl.XXHash.hash32(in.value, 0);
+      out.value = org.apache.drill.exec.expr.fn.impl.HashHelper.hash32(in.value, 0);
     }
   }
 
@@ -363,7 +363,7 @@ public class Hash32Functions {
       if (in.isSet == 0) {
         out.value = 0;
       } else {
-        out.value = org.apache.drill.exec.expr.fn.impl.XXHash.hash32(in.value, 0);
+        out.value = org.apache.drill.exec.expr.fn.impl.HashHelper.hash32(in.value, 0);
       }
     }
   }
@@ -377,7 +377,7 @@ public class Hash32Functions {
     }
 
     public void eval() {
-      out.value = org.apache.drill.exec.expr.fn.impl.XXHash.hash32(in.value, 0);
+      out.value = org.apache.drill.exec.expr.fn.impl.HashHelper.hash32(in.value, 0);
     }
   }
 
@@ -393,7 +393,7 @@ public class Hash32Functions {
       if (in.isSet == 0) {
         out.value = 0;
       } else {
-        out.value = org.apache.drill.exec.expr.fn.impl.XXHash.hash32(in.value, 0);
+        out.value = org.apache.drill.exec.expr.fn.impl.HashHelper.hash32(in.value, 0);
       }
     }
   }
@@ -407,7 +407,7 @@ public class Hash32Functions {
     }
 
     public void eval() {
-      out.value = org.apache.drill.exec.expr.fn.impl.XXHash.hash32(in.value, 0);
+      out.value = org.apache.drill.exec.expr.fn.impl.HashHelper.hash32(in.value, 0);
     }
   }
 
@@ -423,7 +423,7 @@ public class Hash32Functions {
       if (in.isSet == 0) {
         out.value = 0;
       } else {
-        out.value = org.apache.drill.exec.expr.fn.impl.XXHash.hash32(in.value, 0);
+        out.value = org.apache.drill.exec.expr.fn.impl.HashHelper.hash32(in.value, 0);
       }
     }
   }
@@ -442,7 +442,7 @@ public class Hash32Functions {
       for (int i = 0; i < in.nDecimalDigits; i++) {
         xor = xor ^ Decimal28SparseHolder.getInteger(i, in.start, in.buffer);
       }
-      out.value = org.apache.drill.exec.expr.fn.impl.XXHash.hash32(xor, 0);
+      out.value = org.apache.drill.exec.expr.fn.impl.HashHelper.hash32(xor, 0);
     }
   }
 
@@ -462,7 +462,7 @@ public class Hash32Functions {
         for (int i = 0; i < in.nDecimalDigits; i++) {
           xor = xor ^ NullableDecimal28SparseHolder.getInteger(i, in.start, in.buffer);
         }
-        out.value = org.apache.drill.exec.expr.fn.impl.XXHash.hash32(xor, 0);
+        out.value = org.apache.drill.exec.expr.fn.impl.HashHelper.hash32(xor, 0);
       }
     }
   }
@@ -481,7 +481,7 @@ public class Hash32Functions {
       for (int i = 0; i < in.nDecimalDigits; i++) {
         xor = xor ^ Decimal38SparseHolder.getInteger(i, in.start, in.buffer);
       }
-      out.value = org.apache.drill.exec.expr.fn.impl.XXHash.hash32(xor, 0);
+      out.value = org.apache.drill.exec.expr.fn.impl.HashHelper.hash32(xor, 0);
     }
   }
 
@@ -501,7 +501,7 @@ public class Hash32Functions {
         for (int i = 0; i < in.nDecimalDigits; i++) {
           xor = xor ^ NullableDecimal38SparseHolder.getInteger(i, in.start, in.buffer);
         }
-        out.value = org.apache.drill.exec.expr.fn.impl.XXHash.hash32(xor, 0);
+        out.value = org.apache.drill.exec.expr.fn.impl.HashHelper.hash32(xor, 0);
       }
     }
   }
@@ -519,7 +519,7 @@ public class Hash32Functions {
       if (in.isSet == 0) {
         out.value = 0;
       } else {
-        out.value = org.apache.drill.exec.expr.fn.impl.XXHash.hash32(in.value, 0);
+        out.value = org.apache.drill.exec.expr.fn.impl.HashHelper.hash32(in.value, 0);
       }
     }
   }
@@ -534,7 +534,7 @@ public class Hash32Functions {
     }
 
     public void eval() {
-      out.value = org.apache.drill.exec.expr.fn.impl.XXHash.hash32(in.value, 0);
+      out.value = org.apache.drill.exec.expr.fn.impl.HashHelper.hash32(in.value, 0);
     }
   }
 

http://git-wip-us.apache.org/repos/asf/drill/blob/c6a03eb1/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/impl/Hash32FunctionsWithSeed.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/impl/Hash32FunctionsWithSeed.java b/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/impl/Hash32FunctionsWithSeed.java
index 5851956..7d49249 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/impl/Hash32FunctionsWithSeed.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/impl/Hash32FunctionsWithSeed.java
@@ -72,7 +72,7 @@ public class Hash32FunctionsWithSeed {
       if (in.isSet == 0) {
         out.value = seed.value;
       } else {
-        out.value = org.apache.drill.exec.expr.fn.impl.XXHash.hash32(in.value, seed.value);
+        out.value = org.apache.drill.exec.expr.fn.impl.HashHelper.hash32(in.value, seed.value);
       }
     }
   }
@@ -89,7 +89,7 @@ public class Hash32FunctionsWithSeed {
     }
 
     public void eval() {
-      out.value = org.apache.drill.exec.expr.fn.impl.XXHash.hash32(in.value, seed.value);
+      out.value = org.apache.drill.exec.expr.fn.impl.HashHelper.hash32(in.value, seed.value);
     }
   }
 
@@ -108,7 +108,7 @@ public class Hash32FunctionsWithSeed {
       if (in.isSet == 0) {
         out.value = seed.value;
       } else {
-        out.value = org.apache.drill.exec.expr.fn.impl.XXHash.hash32(in.value, seed.value);
+        out.value = org.apache.drill.exec.expr.fn.impl.HashHelper.hash32(in.value, seed.value);
       }
     }
   }
@@ -125,7 +125,7 @@ public class Hash32FunctionsWithSeed {
     }
 
     public void eval() {
-      out.value = org.apache.drill.exec.expr.fn.impl.XXHash.hash32(in.value, seed.value);
+      out.value = org.apache.drill.exec.expr.fn.impl.HashHelper.hash32(in.value, seed.value);
     }
   }
 
@@ -144,7 +144,7 @@ public class Hash32FunctionsWithSeed {
       if (in.isSet == 0) {
         out.value = seed.value;
       } else {
-        out.value = org.apache.drill.exec.expr.fn.impl.XXHash.hash32(in.start, in.end, in.buffer, seed.value);
+        out.value = org.apache.drill.exec.expr.fn.impl.HashHelper.hash32(in.start, in.end, in.buffer, seed.value);
       }
     }
   }
@@ -164,7 +164,7 @@ public class Hash32FunctionsWithSeed {
       if (in.isSet == 0) {
         out.value = seed.value;
       } else {
-        out.value = org.apache.drill.exec.expr.fn.impl.XXHash.hash32(in.start, in.end, in.buffer, seed.value);
+        out.value = org.apache.drill.exec.expr.fn.impl.HashHelper.hash32(in.start, in.end, in.buffer, seed.value);
       }
     }
   }
@@ -184,7 +184,7 @@ public class Hash32FunctionsWithSeed {
       if (in.isSet == 0) {
         out.value = seed.value;
       } else {
-        out.value = org.apache.drill.exec.expr.fn.impl.XXHash.hash32(in.start, in.end, in.buffer, seed.value);
+        out.value = org.apache.drill.exec.expr.fn.impl.HashHelper.hash32(in.start, in.end, in.buffer, seed.value);
       }
     }
   }
@@ -205,7 +205,7 @@ public class Hash32FunctionsWithSeed {
         out.value = seed.value;
       }
       else {
-        out.value = org.apache.drill.exec.expr.fn.impl.XXHash.hash32(in.value, seed.value);
+        out.value = org.apache.drill.exec.expr.fn.impl.HashHelper.hash32(in.value, seed.value);
       }
     }
   }
@@ -225,7 +225,7 @@ public class Hash32FunctionsWithSeed {
         out.value = seed.value;
       }
       else {
-        out.value = org.apache.drill.exec.expr.fn.impl.XXHash.hash32(in.value, seed.value);
+        out.value = org.apache.drill.exec.expr.fn.impl.HashHelper.hash32(in.value, seed.value);
       }
     }
   }
@@ -242,7 +242,7 @@ public class Hash32FunctionsWithSeed {
     }
 
     public void eval() {
-      out.value = org.apache.drill.exec.expr.fn.impl.XXHash.hash32(in.start, in.end, in.buffer, seed.value);
+      out.value = org.apache.drill.exec.expr.fn.impl.HashHelper.hash32(in.start, in.end, in.buffer, seed.value);
     }
   }
 
@@ -258,7 +258,7 @@ public class Hash32FunctionsWithSeed {
     }
 
     public void eval() {
-      out.value = org.apache.drill.exec.expr.fn.impl.XXHash.hash32(in.start, in.end, in.buffer, seed.value);
+      out.value = org.apache.drill.exec.expr.fn.impl.HashHelper.hash32(in.start, in.end, in.buffer, seed.value);
     }
   }
 
@@ -274,7 +274,7 @@ public class Hash32FunctionsWithSeed {
     }
 
     public void eval() {
-      out.value = org.apache.drill.exec.expr.fn.impl.XXHash.hash32(in.start, in.end, in.buffer, seed.value);
+      out.value = org.apache.drill.exec.expr.fn.impl.HashHelper.hash32(in.start, in.end, in.buffer, seed.value);
     }
   }
 
@@ -290,7 +290,7 @@ public class Hash32FunctionsWithSeed {
     }
 
     public void eval() {
-      out.value = org.apache.drill.exec.expr.fn.impl.XXHash.hash32(in.value, seed.value);
+      out.value = org.apache.drill.exec.expr.fn.impl.HashHelper.hash32(in.value, seed.value);
     }
   }
 
@@ -306,7 +306,7 @@ public class Hash32FunctionsWithSeed {
 
     public void eval() {
       // TODO: implement hash function for other types
-      out.value = org.apache.drill.exec.expr.fn.impl.XXHash.hash32(in.value, seed.value);
+      out.value = org.apache.drill.exec.expr.fn.impl.HashHelper.hash32(in.value, seed.value);
     }
   }
   @FunctionTemplate(names = {"hash32", "hash32AsDouble"}, scope = FunctionScope.SIMPLE, nulls = FunctionTemplate.NullHandling.INTERNAL)
@@ -320,7 +320,7 @@ public class Hash32FunctionsWithSeed {
     }
 
     public void eval() {
-      out.value = org.apache.drill.exec.expr.fn.impl.XXHash.hash32(in.value, seed.value);
+      out.value = org.apache.drill.exec.expr.fn.impl.HashHelper.hash32(in.value, seed.value);
     }
   }
 
@@ -338,7 +338,7 @@ public class Hash32FunctionsWithSeed {
       if (in.isSet == 0) {
         out.value = seed.value;
       } else {
-        out.value = org.apache.drill.exec.expr.fn.impl.XXHash.hash32(in.value, seed.value);
+        out.value = org.apache.drill.exec.expr.fn.impl.HashHelper.hash32(in.value, seed.value);
       }
     }
   }
@@ -354,7 +354,7 @@ public class Hash32FunctionsWithSeed {
     }
 
     public void eval() {
-      out.value = org.apache.drill.exec.expr.fn.impl.XXHash.hash32(in.value, seed.value);
+      out.value = org.apache.drill.exec.expr.fn.impl.HashHelper.hash32(in.value, seed.value);
     }
   }
 
@@ -372,7 +372,7 @@ public class Hash32FunctionsWithSeed {
       if (in.isSet == 0) {
         out.value = seed.value;
       } else {
-        out.value = org.apache.drill.exec.expr.fn.impl.XXHash.hash32(in.value, seed.value);
+        out.value = org.apache.drill.exec.expr.fn.impl.HashHelper.hash32(in.value, seed.value);
       }
     }
   }
@@ -388,7 +388,7 @@ public class Hash32FunctionsWithSeed {
     }
 
     public void eval() {
-      out.value = org.apache.drill.exec.expr.fn.impl.XXHash.hash32(in.value, seed.value);
+      out.value = org.apache.drill.exec.expr.fn.impl.HashHelper.hash32(in.value, seed.value);
     }
   }
 
@@ -406,7 +406,7 @@ public class Hash32FunctionsWithSeed {
       if (in.isSet == 0) {
         out.value = seed.value;
       } else {
-        out.value = org.apache.drill.exec.expr.fn.impl.XXHash.hash32(in.value, seed.value);
+        out.value = org.apache.drill.exec.expr.fn.impl.HashHelper.hash32(in.value, seed.value);
       }
     }
   }
@@ -422,7 +422,7 @@ public class Hash32FunctionsWithSeed {
     }
 
     public void eval() {
-      out.value = org.apache.drill.exec.expr.fn.impl.XXHash.hash32(in.value, seed.value);
+      out.value = org.apache.drill.exec.expr.fn.impl.HashHelper.hash32(in.value, seed.value);
     }
   }
 
@@ -440,7 +440,7 @@ public class Hash32FunctionsWithSeed {
       if (in.isSet == 0) {
         out.value = seed.value;
       } else {
-        out.value = org.apache.drill.exec.expr.fn.impl.XXHash.hash32(in.value, seed.value);
+        out.value = org.apache.drill.exec.expr.fn.impl.HashHelper.hash32(in.value, seed.value);
       }
     }
   }
@@ -456,7 +456,7 @@ public class Hash32FunctionsWithSeed {
     }
 
     public void eval() {
-      out.value = org.apache.drill.exec.expr.fn.impl.XXHash.hash32(in.value, seed.value);
+      out.value = org.apache.drill.exec.expr.fn.impl.HashHelper.hash32(in.value, seed.value);
     }
   }
 
@@ -474,7 +474,7 @@ public class Hash32FunctionsWithSeed {
       if (in.isSet == 0) {
         out.value = seed.value;
       } else {
-        out.value = org.apache.drill.exec.expr.fn.impl.XXHash.hash32(in.value, seed.value);
+        out.value = org.apache.drill.exec.expr.fn.impl.HashHelper.hash32(in.value, seed.value);
       }
     }
   }
@@ -490,7 +490,7 @@ public class Hash32FunctionsWithSeed {
     }
 
     public void eval() {
-      out.value = org.apache.drill.exec.expr.fn.impl.XXHash.hash32(in.start, in.start + Decimal28SparseHolder.WIDTH, in.buffer, seed.value);
+      out.value = org.apache.drill.exec.expr.fn.impl.HashHelper.hash32(in.start, in.start + Decimal28SparseHolder.WIDTH, in.buffer, seed.value);
     }
   }
 
@@ -508,7 +508,7 @@ public class Hash32FunctionsWithSeed {
       if (in.isSet == 0) {
         out.value = seed.value;
       } else {
-        out.value = org.apache.drill.exec.expr.fn.impl.XXHash.hash32(in.start, in.start + NullableDecimal28SparseHolder.WIDTH, in.buffer, seed.value);
+        out.value = org.apache.drill.exec.expr.fn.impl.HashHelper.hash32(in.start, in.start + NullableDecimal28SparseHolder.WIDTH, in.buffer, seed.value);
       }
     }
   }
@@ -524,7 +524,7 @@ public class Hash32FunctionsWithSeed {
     }
 
     public void eval() {
-      out.value = org.apache.drill.exec.expr.fn.impl.XXHash.hash32(in.start, in.start + Decimal38SparseHolder.WIDTH, in.buffer, seed.value);
+      out.value = org.apache.drill.exec.expr.fn.impl.HashHelper.hash32(in.start, in.start + Decimal38SparseHolder.WIDTH, in.buffer, seed.value);
     }
   }
 
@@ -542,7 +542,7 @@ public class Hash32FunctionsWithSeed {
       if (in.isSet == 0) {
         out.value = seed.value;
       } else {
-        out.value = org.apache.drill.exec.expr.fn.impl.XXHash.hash32(in.start, in.start + NullableDecimal38SparseHolder.WIDTH, in.buffer, seed.value);
+        out.value = org.apache.drill.exec.expr.fn.impl.HashHelper.hash32(in.start, in.start + NullableDecimal38SparseHolder.WIDTH, in.buffer, seed.value);
       }
     }
   }
@@ -562,7 +562,7 @@ public class Hash32FunctionsWithSeed {
       if (in.isSet == 0) {
         out.value = seed.value;
       } else {
-        out.value = org.apache.drill.exec.expr.fn.impl.XXHash.hash32(in.value, seed.value);
+        out.value = org.apache.drill.exec.expr.fn.impl.HashHelper.hash32(in.value, seed.value);
       }
     }
   }
@@ -579,6 +579,7 @@ public class Hash32FunctionsWithSeed {
     }
 
     public void eval() {
-      out.value = org.apache.drill.exec.expr.fn.impl.XXHash.hash32(in.value, seed.value);
+      out.value = org.apache.drill.exec.expr.fn.impl.HashHelper.hash32(in.value, seed.value);
     }
-  }}
+  }
+}

http://git-wip-us.apache.org/repos/asf/drill/blob/c6a03eb1/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/impl/Hash32WithSeedAsDouble.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/impl/Hash32WithSeedAsDouble.java b/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/impl/Hash32WithSeedAsDouble.java
index cd4dfa2..f3d4d54 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/impl/Hash32WithSeedAsDouble.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/impl/Hash32WithSeedAsDouble.java
@@ -59,7 +59,7 @@ public class Hash32WithSeedAsDouble {
       if (in.isSet == 0) {
         out.value = seed.value;
       } else {
-        out.value = org.apache.drill.exec.expr.fn.impl.XXHash.hash32((double) in.value, seed.value);
+        out.value = org.apache.drill.exec.expr.fn.impl.HashHelper.hash32((double) in.value, seed.value);
       }
     }
   }
@@ -76,7 +76,7 @@ public class Hash32WithSeedAsDouble {
     }
 
     public void eval() {
-      out.value = org.apache.drill.exec.expr.fn.impl.XXHash.hash32((double) in.value, seed.value);
+      out.value = org.apache.drill.exec.expr.fn.impl.HashHelper.hash32((double) in.value, seed.value);
     }
   }
 
@@ -95,7 +95,7 @@ public class Hash32WithSeedAsDouble {
       if (in.isSet == 0) {
         out.value = seed.value;
       } else {
-        out.value = org.apache.drill.exec.expr.fn.impl.XXHash.hash32(in.value, seed.value);
+        out.value = org.apache.drill.exec.expr.fn.impl.HashHelper.hash32(in.value, seed.value);
       }
     }
   }
@@ -112,7 +112,7 @@ public class Hash32WithSeedAsDouble {
     }
 
     public void eval() {
-      out.value = org.apache.drill.exec.expr.fn.impl.XXHash.hash32(in.value, seed.value);
+      out.value = org.apache.drill.exec.expr.fn.impl.HashHelper.hash32(in.value, seed.value);
     }
   }
 
@@ -132,7 +132,7 @@ public class Hash32WithSeedAsDouble {
         out.value = seed.value;
       }
       else {
-        out.value = org.apache.drill.exec.expr.fn.impl.XXHash.hash32((double) in.value, seed.value);
+        out.value = org.apache.drill.exec.expr.fn.impl.HashHelper.hash32((double) in.value, seed.value);
       }
     }
   }
@@ -152,7 +152,7 @@ public class Hash32WithSeedAsDouble {
         out.value = seed.value;
       }
       else {
-        out.value = org.apache.drill.exec.expr.fn.impl.XXHash.hash32((double) in.value, seed.value);
+        out.value = org.apache.drill.exec.expr.fn.impl.HashHelper.hash32((double) in.value, seed.value);
       }
     }
   }
@@ -169,7 +169,7 @@ public class Hash32WithSeedAsDouble {
     }
 
     public void eval() {
-      out.value = org.apache.drill.exec.expr.fn.impl.XXHash.hash32((double) in.value, seed.value);
+      out.value = org.apache.drill.exec.expr.fn.impl.HashHelper.hash32((double) in.value, seed.value);
     }
   }
 
@@ -185,7 +185,7 @@ public class Hash32WithSeedAsDouble {
 
     public void eval() {
       // TODO: implement hash function for other types
-      out.value = org.apache.drill.exec.expr.fn.impl.XXHash.hash32((double) in.value, seed.value);
+      out.value = org.apache.drill.exec.expr.fn.impl.HashHelper.hash32((double) in.value, seed.value);
     }
   }
 
@@ -201,7 +201,7 @@ public class Hash32WithSeedAsDouble {
 
     public void eval() {
       java.math.BigDecimal input = new java.math.BigDecimal(java.math.BigInteger.valueOf(in.value), in.scale);
-      out.value = org.apache.drill.exec.expr.fn.impl.XXHash.hash32(input.doubleValue(), seed.value);
+      out.value = org.apache.drill.exec.expr.fn.impl.HashHelper.hash32(input.doubleValue(), seed.value);
     }
   }
 
@@ -220,7 +220,7 @@ public class Hash32WithSeedAsDouble {
         out.value = seed.value;
       } else {
         java.math.BigDecimal input = new java.math.BigDecimal(java.math.BigInteger.valueOf(in.value), in.scale);
-        out.value = org.apache.drill.exec.expr.fn.impl.XXHash.hash32(input.doubleValue(), seed.value);
+        out.value = org.apache.drill.exec.expr.fn.impl.HashHelper.hash32(input.doubleValue(), seed.value);
       }
     }
   }
@@ -237,7 +237,7 @@ public class Hash32WithSeedAsDouble {
 
     public void eval() {
       java.math.BigDecimal input = new java.math.BigDecimal(java.math.BigInteger.valueOf(in.value), in.scale);
-      out.value = org.apache.drill.exec.expr.fn.impl.XXHash.hash32(input.doubleValue(), seed.value);
+      out.value = org.apache.drill.exec.expr.fn.impl.HashHelper.hash32(input.doubleValue(), seed.value);
     }
   }
 
@@ -256,7 +256,7 @@ public class Hash32WithSeedAsDouble {
         out.value = seed.value;
       } else {
         java.math.BigDecimal input = new java.math.BigDecimal(java.math.BigInteger.valueOf(in.value), in.scale);
-        out.value = org.apache.drill.exec.expr.fn.impl.XXHash.hash32(input.doubleValue(), seed.value);
+        out.value = org.apache.drill.exec.expr.fn.impl.HashHelper.hash32(input.doubleValue(), seed.value);
       }
     }
   }
@@ -274,7 +274,7 @@ public class Hash32WithSeedAsDouble {
     public void eval() {
       java.math.BigDecimal input = org.apache.drill.exec.util.DecimalUtility.getBigDecimalFromSparse(in.buffer,
           in.start, in.nDecimalDigits, in.scale);
-      out.value = org.apache.drill.exec.expr.fn.impl.XXHash.hash32(input.doubleValue(), 0);
+      out.value = org.apache.drill.exec.expr.fn.impl.HashHelper.hash32(input.doubleValue(), seed.value);
     }
   }
 
@@ -294,7 +294,7 @@ public class Hash32WithSeedAsDouble {
       } else {
         java.math.BigDecimal input = org.apache.drill.exec.util.DecimalUtility.getBigDecimalFromSparse(in.buffer,
             in.start, in.nDecimalDigits, in.scale);
-        out.value = org.apache.drill.exec.expr.fn.impl.XXHash.hash32(input.doubleValue(), 0);
+        out.value = org.apache.drill.exec.expr.fn.impl.HashHelper.hash32(input.doubleValue(), seed.value);
       }
     }
   }
@@ -312,7 +312,7 @@ public class Hash32WithSeedAsDouble {
     public void eval() {
       java.math.BigDecimal input = org.apache.drill.exec.util.DecimalUtility.getBigDecimalFromSparse(in.buffer,
           in.start, in.nDecimalDigits, in.scale);
-      out.value = org.apache.drill.exec.expr.fn.impl.XXHash.hash32(input.doubleValue(), 0);
+      out.value = org.apache.drill.exec.expr.fn.impl.HashHelper.hash32(input.doubleValue(), seed.value);
     }
   }
 
@@ -332,7 +332,7 @@ public class Hash32WithSeedAsDouble {
       } else {
         java.math.BigDecimal input = org.apache.drill.exec.util.DecimalUtility.getBigDecimalFromSparse(in.buffer,
             in.start, in.nDecimalDigits, in.scale);
-        out.value = org.apache.drill.exec.expr.fn.impl.XXHash.hash32(input.doubleValue(), 0);
+        out.value = org.apache.drill.exec.expr.fn.impl.HashHelper.hash32(input.doubleValue(), seed.value);
       }
     }
   }

http://git-wip-us.apache.org/repos/asf/drill/blob/c6a03eb1/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/impl/Hash64AsDouble.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/impl/Hash64AsDouble.java b/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/impl/Hash64AsDouble.java
index 91caafc..f9080b7 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/impl/Hash64AsDouble.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/impl/Hash64AsDouble.java
@@ -63,7 +63,7 @@ public class Hash64AsDouble {
       if (in.isSet == 0) {
         out.value = 0;
       } else {
-        out.value = org.apache.drill.exec.expr.fn.impl.XXHash.hash64((double) in.value, 0);
+        out.value = org.apache.drill.exec.expr.fn.impl.HashHelper.hash64((double) in.value, 0);
       }
     }
   }
@@ -80,7 +80,7 @@ public class Hash64AsDouble {
     }
 
     public void eval() {
-      out.value = org.apache.drill.exec.expr.fn.impl.XXHash.hash64((double) in.value, 0);
+      out.value = org.apache.drill.exec.expr.fn.impl.HashHelper.hash64((double) in.value, 0);
     }
   }
 
@@ -99,7 +99,7 @@ public class Hash64AsDouble {
       if (in.isSet == 0) {
         out.value = 0;
       } else {
-        out.value = org.apache.drill.exec.expr.fn.impl.XXHash.hash64((double) in.value, 0);
+        out.value = org.apache.drill.exec.expr.fn.impl.HashHelper.hash64((double) in.value, 0);
       }
     }
   }
@@ -116,7 +116,7 @@ public class Hash64AsDouble {
     }
 
     public void eval() {
-      out.value = org.apache.drill.exec.expr.fn.impl.XXHash.hash64(in.value, 0);
+      out.value = org.apache.drill.exec.expr.fn.impl.HashHelper.hash64(in.value, 0);
     }
   }
 
@@ -135,7 +135,7 @@ public class Hash64AsDouble {
       if (in.isSet == 0) {
         out.value = 0;
       } else {
-        out.value = org.apache.drill.exec.expr.fn.impl.XXHash.hash64((double) in.value, 0);
+        out.value = org.apache.drill.exec.expr.fn.impl.HashHelper.hash64((double) in.value, 0);
       }
     }
   }
@@ -154,7 +154,7 @@ public class Hash64AsDouble {
       if (in.isSet == 0) {
         out.value = 0;
       } else {
-        out.value = org.apache.drill.exec.expr.fn.impl.XXHash.hash64((double) in.value, 0);
+        out.value = org.apache.drill.exec.expr.fn.impl.HashHelper.hash64((double) in.value, 0);
       }
     }
   }
@@ -171,7 +171,7 @@ public class Hash64AsDouble {
     }
 
     public void eval() {
-      out.value = org.apache.drill.exec.expr.fn.impl.XXHash.hash64((double) in.value, 0);
+      out.value = org.apache.drill.exec.expr.fn.impl.HashHelper.hash64((double) in.value, 0);
     }
   }
 
@@ -187,7 +187,7 @@ public class Hash64AsDouble {
 
     public void eval() {
       // TODO: implement hash function for other types
-      out.value = org.apache.drill.exec.expr.fn.impl.XXHash.hash64((double) in.value, 0);
+      out.value = org.apache.drill.exec.expr.fn.impl.HashHelper.hash64((double) in.value, 0);
     }
   }
 
@@ -203,7 +203,7 @@ public class Hash64AsDouble {
 
     public void eval() {
       java.math.BigDecimal input = new java.math.BigDecimal(java.math.BigInteger.valueOf(in.value), in.scale);
-      out.value = org.apache.drill.exec.expr.fn.impl.XXHash.hash64(input.doubleValue(), 0);
+      out.value = org.apache.drill.exec.expr.fn.impl.HashHelper.hash64(input.doubleValue(), 0);
     }
   }
 
@@ -222,7 +222,7 @@ public class Hash64AsDouble {
         out.value = 0;
       } else {
         java.math.BigDecimal input = new java.math.BigDecimal(java.math.BigInteger.valueOf(in.value), in.scale);
-        out.value = org.apache.drill.exec.expr.fn.impl.XXHash.hash64(input.doubleValue(), 0);
+        out.value = org.apache.drill.exec.expr.fn.impl.HashHelper.hash64(input.doubleValue(), 0);
       }
     }
   }
@@ -239,7 +239,7 @@ public class Hash64AsDouble {
 
     public void eval() {
       java.math.BigDecimal input = new java.math.BigDecimal(java.math.BigInteger.valueOf(in.value), in.scale);
-      out.value = org.apache.drill.exec.expr.fn.impl.XXHash.hash64(input.doubleValue(), 0);
+      out.value = org.apache.drill.exec.expr.fn.impl.HashHelper.hash64(input.doubleValue(), 0);
     }
   }
 
@@ -258,7 +258,7 @@ public class Hash64AsDouble {
         out.value = 0;
       } else {
         java.math.BigDecimal input = new java.math.BigDecimal(java.math.BigInteger.valueOf(in.value), in.scale);
-        out.value = org.apache.drill.exec.expr.fn.impl.XXHash.hash64(input.doubleValue(), 0);
+        out.value = org.apache.drill.exec.expr.fn.impl.HashHelper.hash64(input.doubleValue(), 0);
       }
     }
   }
@@ -276,7 +276,7 @@ public class Hash64AsDouble {
     public void eval() {
       java.math.BigDecimal input = org.apache.drill.exec.util.DecimalUtility.getBigDecimalFromSparse(in.buffer,
           in.start, in.nDecimalDigits, in.scale);
-      out.value = org.apache.drill.exec.expr.fn.impl.XXHash.hash64(input.doubleValue(), 0);
+      out.value = org.apache.drill.exec.expr.fn.impl.HashHelper.hash64(input.doubleValue(), 0);
     }
   }
 
@@ -296,7 +296,7 @@ public class Hash64AsDouble {
       } else {
         java.math.BigDecimal input = org.apache.drill.exec.util.DecimalUtility.getBigDecimalFromSparse(in.buffer,
             in.start, in.nDecimalDigits, in.scale);
-        out.value = org.apache.drill.exec.expr.fn.impl.XXHash.hash64(input.doubleValue(), 0);
+        out.value = org.apache.drill.exec.expr.fn.impl.HashHelper.hash64(input.doubleValue(), 0);
       }
     }
   }
@@ -314,7 +314,7 @@ public class Hash64AsDouble {
     public void eval() {
       java.math.BigDecimal input = org.apache.drill.exec.util.DecimalUtility.getBigDecimalFromSparse(in.buffer,
           in.start, in.nDecimalDigits, in.scale);
-      out.value = org.apache.drill.exec.expr.fn.impl.XXHash.hash64(input.doubleValue(), 0);
+      out.value = org.apache.drill.exec.expr.fn.impl.HashHelper.hash64(input.doubleValue(), 0);
     }
   }
 
@@ -334,7 +334,7 @@ public class Hash64AsDouble {
       } else {
         java.math.BigDecimal input = org.apache.drill.exec.util.DecimalUtility.getBigDecimalFromSparse(in.buffer,
             in.start, in.nDecimalDigits, in.scale);
-        out.value = org.apache.drill.exec.expr.fn.impl.XXHash.hash64(input.doubleValue(), 0);
+        out.value = org.apache.drill.exec.expr.fn.impl.HashHelper.hash64(input.doubleValue(), 0);
       }
     }
   }

http://git-wip-us.apache.org/repos/asf/drill/blob/c6a03eb1/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/impl/Hash64Functions.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/impl/Hash64Functions.java b/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/impl/Hash64Functions.java
index 5145af2..157cbcd 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/impl/Hash64Functions.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/impl/Hash64Functions.java
@@ -72,7 +72,7 @@ public class Hash64Functions {
       if (in.isSet == 0) {
         out.value = 0;
       } else {
-        out.value = org.apache.drill.exec.expr.fn.impl.XXHash.hash64(in.value, 0);
+        out.value = org.apache.drill.exec.expr.fn.impl.HashHelper.hash64(in.value, 0);
       }
     }
   }
@@ -89,7 +89,7 @@ public class Hash64Functions {
     }
 
     public void eval() {
-      out.value = org.apache.drill.exec.expr.fn.impl.XXHash.hash64(in.value, 0);
+      out.value = org.apache.drill.exec.expr.fn.impl.HashHelper.hash64(in.value, 0);
     }
   }
 
@@ -108,7 +108,7 @@ public class Hash64Functions {
       if (in.isSet == 0) {
         out.value = 0;
       } else {
-        out.value = org.apache.drill.exec.expr.fn.impl.XXHash.hash64(in.value, 0);
+        out.value = org.apache.drill.exec.expr.fn.impl.HashHelper.hash64(in.value, 0);
       }
     }
   }
@@ -125,7 +125,7 @@ public class Hash64Functions {
     }
 
     public void eval() {
-      out.value = org.apache.drill.exec.expr.fn.impl.XXHash.hash64(in.value, 0);
+      out.value = org.apache.drill.exec.expr.fn.impl.HashHelper.hash64(in.value, 0);
     }
   }
 
@@ -144,7 +144,7 @@ public class Hash64Functions {
       if (in.isSet == 0) {
         out.value = 0;
       } else {
-        out.value = org.apache.drill.exec.expr.fn.impl.XXHash.hash64(in.start, in.end, in.buffer, 0);
+        out.value = org.apache.drill.exec.expr.fn.impl.HashHelper.hash64(in.start, in.end, in.buffer, 0);
       }
     }
   }
@@ -164,7 +164,7 @@ public class Hash64Functions {
       if (in.isSet == 0) {
         out.value = 0;
       } else {
-        out.value = org.apache.drill.exec.expr.fn.impl.XXHash.hash64(in.start, in.end, in.buffer, 0);
+        out.value = org.apache.drill.exec.expr.fn.impl.HashHelper.hash64(in.start, in.end, in.buffer, 0);
       }
     }
   }
@@ -184,7 +184,7 @@ public class Hash64Functions {
       if (in.isSet == 0) {
         out.value = 0;
       } else {
-        out.value = org.apache.drill.exec.expr.fn.impl.XXHash.hash64(in.start, in.end, in.buffer, 0);
+        out.value = org.apache.drill.exec.expr.fn.impl.HashHelper.hash64(in.start, in.end, in.buffer, 0);
       }
     }
   }
@@ -204,7 +204,7 @@ public class Hash64Functions {
       if (in.isSet == 0) {
         out.value = 0;
       } else {
-        out.value = org.apache.drill.exec.expr.fn.impl.XXHash.hash64(in.value, 0);
+        out.value = org.apache.drill.exec.expr.fn.impl.HashHelper.hash64(in.value, 0);
       }
     }
   }
@@ -223,7 +223,7 @@ public class Hash64Functions {
       if (in.isSet == 0) {
         out.value = 0;
       } else {
-        out.value = org.apache.drill.exec.expr.fn.impl.XXHash.hash64(in.value, 0);
+        out.value = org.apache.drill.exec.expr.fn.impl.HashHelper.hash64(in.value, 0);
       }
     }
   }
@@ -240,7 +240,7 @@ public class Hash64Functions {
     }
 
     public void eval() {
-      out.value = org.apache.drill.exec.expr.fn.impl.XXHash.hash64(in.start, in.end, in.buffer, 0);
+      out.value = org.apache.drill.exec.expr.fn.impl.HashHelper.hash64(in.start, in.end, in.buffer, 0);
     }
   }
 
@@ -256,7 +256,7 @@ public class Hash64Functions {
     }
 
     public void eval() {
-      out.value = org.apache.drill.exec.expr.fn.impl.XXHash.hash64(in.start, in.end, in.buffer, 0);
+      out.value = org.apache.drill.exec.expr.fn.impl.HashHelper.hash64(in.start, in.end, in.buffer, 0);
     }
   }
 
@@ -272,7 +272,7 @@ public class Hash64Functions {
     }
 
     public void eval() {
-      out.value = org.apache.drill.exec.expr.fn.impl.XXHash.hash64(in.start, in.end, in.buffer, 0);
+      out.value = org.apache.drill.exec.expr.fn.impl.HashHelper.hash64(in.start, in.end, in.buffer, 0);
     }
   }
 
@@ -288,7 +288,7 @@ public class Hash64Functions {
     }
 
     public void eval() {
-      out.value = org.apache.drill.exec.expr.fn.impl.XXHash.hash64(in.value, 0);
+      out.value = org.apache.drill.exec.expr.fn.impl.HashHelper.hash64(in.value, 0);
     }
   }
 
@@ -304,7 +304,7 @@ public class Hash64Functions {
 
     public void eval() {
       // TODO: implement hash function for other types
-      out.value = org.apache.drill.exec.expr.fn.impl.XXHash.hash64(in.value, 0);
+      out.value = org.apache.drill.exec.expr.fn.impl.HashHelper.hash64(in.value, 0);
     }
   }
 
@@ -319,7 +319,7 @@ public class Hash64Functions {
     }
 
     public void eval() {
-      out.value = org.apache.drill.exec.expr.fn.impl.XXHash.hash64(in.value, 0);
+      out.value = org.apache.drill.exec.expr.fn.impl.HashHelper.hash64(in.value, 0);
     }
   }
 
@@ -337,7 +337,7 @@ public class Hash64Functions {
       if (in.isSet == 0) {
         out.value = 0;
       } else {
-        out.value = org.apache.drill.exec.expr.fn.impl.XXHash.hash64(in.value, 0);
+        out.value = org.apache.drill.exec.expr.fn.impl.HashHelper.hash64(in.value, 0);
       }
     }
   }
@@ -353,7 +353,7 @@ public class Hash64Functions {
     }
 
     public void eval() {
-      out.value = org.apache.drill.exec.expr.fn.impl.XXHash.hash64(in.value, 0);
+      out.value = org.apache.drill.exec.expr.fn.impl.HashHelper.hash64(in.value, 0);
     }
   }
 
@@ -371,7 +371,7 @@ public class Hash64Functions {
       if (in.isSet == 0) {
         out.value = 0;
       } else {
-        out.value = org.apache.drill.exec.expr.fn.impl.XXHash.hash64(in.value, 0);
+        out.value = org.apache.drill.exec.expr.fn.impl.HashHelper.hash64(in.value, 0);
       }
     }
   }
@@ -387,7 +387,7 @@ public class Hash64Functions {
     }
 
     public void eval() {
-      out.value = org.apache.drill.exec.expr.fn.impl.XXHash.hash64(in.value, 0);
+      out.value = org.apache.drill.exec.expr.fn.impl.HashHelper.hash64(in.value, 0);
     }
   }
 
@@ -405,7 +405,7 @@ public class Hash64Functions {
       if (in.isSet == 0) {
         out.value = 0;
       } else {
-        out.value = org.apache.drill.exec.expr.fn.impl.XXHash.hash64(in.value, 0);
+        out.value = org.apache.drill.exec.expr.fn.impl.HashHelper.hash64(in.value, 0);
       }
     }
   }
@@ -421,7 +421,7 @@ public class Hash64Functions {
     }
 
     public void eval() {
-      out.value = org.apache.drill.exec.expr.fn.impl.XXHash.hash64(in.value, 0);
+      out.value = org.apache.drill.exec.expr.fn.impl.HashHelper.hash64(in.value, 0);
     }
   }
 
@@ -439,7 +439,7 @@ public class Hash64Functions {
       if (in.isSet == 0) {
         out.value = 0;
       } else {
-        out.value = org.apache.drill.exec.expr.fn.impl.XXHash.hash64(in.value, 0);
+        out.value = org.apache.drill.exec.expr.fn.impl.HashHelper.hash64(in.value, 0);
       }
     }
   }
@@ -455,7 +455,7 @@ public class Hash64Functions {
     }
 
     public void eval() {
-      out.value = org.apache.drill.exec.expr.fn.impl.XXHash.hash64(in.value, 0);
+      out.value = org.apache.drill.exec.expr.fn.impl.HashHelper.hash64(in.value, 0);
     }
   }
 
@@ -473,7 +473,7 @@ public class Hash64Functions {
       if (in.isSet == 0) {
         out.value = 0;
       } else {
-        out.value = org.apache.drill.exec.expr.fn.impl.XXHash.hash64(in.value, 0);
+        out.value = org.apache.drill.exec.expr.fn.impl.HashHelper.hash64(in.value, 0);
       }
     }
   }
@@ -489,7 +489,7 @@ public class Hash64Functions {
     }
 
     public void eval() {
-      out.value = org.apache.drill.exec.expr.fn.impl.XXHash.hash64(in.start, in.start + Decimal28SparseHolder.WIDTH,
+      out.value = org.apache.drill.exec.expr.fn.impl.HashHelper.hash64(in.start, in.start + Decimal28SparseHolder.WIDTH,
           in.buffer, 0);
     }
   }
@@ -508,7 +508,7 @@ public class Hash64Functions {
       if (in.isSet == 0) {
         out.value = 0;
       } else {
-        out.value = org.apache.drill.exec.expr.fn.impl.XXHash.hash64(in.start, in.start
+        out.value = org.apache.drill.exec.expr.fn.impl.HashHelper.hash64(in.start, in.start
             + NullableDecimal28SparseHolder.WIDTH, in.buffer, 0);
       }
     }
@@ -525,7 +525,7 @@ public class Hash64Functions {
     }
 
     public void eval() {
-      out.value = org.apache.drill.exec.expr.fn.impl.XXHash.hash64(in.start, in.start + Decimal38SparseHolder.WIDTH,
+      out.value = org.apache.drill.exec.expr.fn.impl.HashHelper.hash64(in.start, in.start + Decimal38SparseHolder.WIDTH,
           in.buffer, 0);
     }
   }
@@ -544,7 +544,7 @@ public class Hash64Functions {
       if (in.isSet == 0) {
         out.value = 0;
       } else {
-        out.value = org.apache.drill.exec.expr.fn.impl.XXHash.hash64(in.start, in.start
+        out.value = org.apache.drill.exec.expr.fn.impl.HashHelper.hash64(in.start, in.start
             + NullableDecimal38SparseHolder.WIDTH, in.buffer, 0);
       }
     }
@@ -565,7 +565,7 @@ public class Hash64Functions {
       if (in.isSet == 0) {
         out.value = 0;
       } else {
-        out.value = org.apache.drill.exec.expr.fn.impl.XXHash.hash64(in.value, 0);
+        out.value = org.apache.drill.exec.expr.fn.impl.HashHelper.hash64(in.value, 0);
       }
     }
   }
@@ -582,7 +582,7 @@ public class Hash64Functions {
     }
 
     public void eval() {
-      out.value = org.apache.drill.exec.expr.fn.impl.XXHash.hash64(in.value, 0);
+      out.value = org.apache.drill.exec.expr.fn.impl.HashHelper.hash64(in.value, 0);
     }
   }
 }

http://git-wip-us.apache.org/repos/asf/drill/blob/c6a03eb1/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/impl/Hash64FunctionsWithSeed.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/impl/Hash64FunctionsWithSeed.java b/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/impl/Hash64FunctionsWithSeed.java
index 004a221..833a209 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/impl/Hash64FunctionsWithSeed.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/impl/Hash64FunctionsWithSeed.java
@@ -72,7 +72,7 @@ public class Hash64FunctionsWithSeed {
       if (in.isSet == 0) {
         out.value = seed.value;
       } else {
-        out.value = org.apache.drill.exec.expr.fn.impl.XXHash.hash64(in.value, seed.value);
+        out.value = org.apache.drill.exec.expr.fn.impl.HashHelper.hash64(in.value, seed.value);
       }
     }
   }
@@ -89,7 +89,7 @@ public class Hash64FunctionsWithSeed {
     }
 
     public void eval() {
-      out.value = org.apache.drill.exec.expr.fn.impl.XXHash.hash64(in.value, seed.value);
+      out.value = org.apache.drill.exec.expr.fn.impl.HashHelper.hash64(in.value, seed.value);
     }
   }
 
@@ -108,7 +108,7 @@ public class Hash64FunctionsWithSeed {
       if (in.isSet == 0) {
         out.value = seed.value;
       } else {
-        out.value = org.apache.drill.exec.expr.fn.impl.XXHash.hash64(in.value, seed.value);
+        out.value = org.apache.drill.exec.expr.fn.impl.HashHelper.hash64(in.value, seed.value);
       }
     }
   }
@@ -125,7 +125,7 @@ public class Hash64FunctionsWithSeed {
     }
 
     public void eval() {
-      out.value = org.apache.drill.exec.expr.fn.impl.XXHash.hash64(in.value, seed.value);
+      out.value = org.apache.drill.exec.expr.fn.impl.HashHelper.hash64(in.value, seed.value);
     }
   }
 
@@ -144,7 +144,7 @@ public class Hash64FunctionsWithSeed {
       if (in.isSet == 0) {
         out.value = seed.value;
       } else {
-        out.value = org.apache.drill.exec.expr.fn.impl.XXHash.hash64(in.start, in.end, in.buffer, seed.value);
+        out.value = org.apache.drill.exec.expr.fn.impl.HashHelper.hash64(in.start, in.end, in.buffer, seed.value);
       }
     }
   }
@@ -164,7 +164,7 @@ public class Hash64FunctionsWithSeed {
       if (in.isSet == 0) {
         out.value = seed.value;
       } else {
-        out.value = org.apache.drill.exec.expr.fn.impl.XXHash.hash64(in.start, in.end, in.buffer, seed.value);
+        out.value = org.apache.drill.exec.expr.fn.impl.HashHelper.hash64(in.start, in.end, in.buffer, seed.value);
       }
     }
   }
@@ -184,7 +184,7 @@ public class Hash64FunctionsWithSeed {
       if (in.isSet == 0) {
         out.value = seed.value;
       } else {
-        out.value = org.apache.drill.exec.expr.fn.impl.XXHash.hash64(in.start, in.end, in.buffer, seed.value);
+        out.value = org.apache.drill.exec.expr.fn.impl.HashHelper.hash64(in.start, in.end, in.buffer, seed.value);
       }
     }
   }
@@ -205,7 +205,7 @@ public class Hash64FunctionsWithSeed {
         out.value = seed.value;
       }
       else {
-        out.value = org.apache.drill.exec.expr.fn.impl.XXHash.hash64(in.value, seed.value);
+        out.value = org.apache.drill.exec.expr.fn.impl.HashHelper.hash64(in.value, seed.value);
       }
     }
   }
@@ -225,7 +225,7 @@ public class Hash64FunctionsWithSeed {
         out.value = seed.value;
       }
       else {
-        out.value = org.apache.drill.exec.expr.fn.impl.XXHash.hash64(in.value, seed.value);
+        out.value = org.apache.drill.exec.expr.fn.impl.HashHelper.hash64(in.value, seed.value);
       }
     }
   }
@@ -242,7 +242,7 @@ public class Hash64FunctionsWithSeed {
     }
 
     public void eval() {
-      out.value = org.apache.drill.exec.expr.fn.impl.XXHash.hash64(in.start, in.end, in.buffer, seed.value);
+      out.value = org.apache.drill.exec.expr.fn.impl.HashHelper.hash64(in.start, in.end, in.buffer, seed.value);
     }
   }
 
@@ -258,7 +258,7 @@ public class Hash64FunctionsWithSeed {
     }
 
     public void eval() {
-      out.value = org.apache.drill.exec.expr.fn.impl.XXHash.hash64(in.start, in.end, in.buffer, seed.value);
+      out.value = org.apache.drill.exec.expr.fn.impl.HashHelper.hash64(in.start, in.end, in.buffer, seed.value);
     }
   }
 
@@ -274,7 +274,7 @@ public class Hash64FunctionsWithSeed {
     }
 
     public void eval() {
-      out.value = org.apache.drill.exec.expr.fn.impl.XXHash.hash64(in.start, in.end, in.buffer, seed.value);
+      out.value = org.apache.drill.exec.expr.fn.impl.HashHelper.hash64(in.start, in.end, in.buffer, seed.value);
     }
   }
 
@@ -290,7 +290,7 @@ public class Hash64FunctionsWithSeed {
     }
 
     public void eval() {
-      out.value = org.apache.drill.exec.expr.fn.impl.XXHash.hash64(in.value, seed.value);
+      out.value = org.apache.drill.exec.expr.fn.impl.HashHelper.hash64(in.value, seed.value);
     }
   }
 
@@ -306,7 +306,7 @@ public class Hash64FunctionsWithSeed {
 
     public void eval() {
       // TODO: implement hash function for other types
-      out.value = org.apache.drill.exec.expr.fn.impl.XXHash.hash64(in.value, seed.value);
+      out.value = org.apache.drill.exec.expr.fn.impl.HashHelper.hash64(in.value, seed.value);
     }
   }
   @FunctionTemplate(names = {"hash64", "hash64AsDouble"}, scope = FunctionScope.SIMPLE, nulls = FunctionTemplate.NullHandling.INTERNAL)
@@ -320,7 +320,7 @@ public class Hash64FunctionsWithSeed {
     }
 
     public void eval() {
-      out.value = org.apache.drill.exec.expr.fn.impl.XXHash.hash64(in.value, seed.value);
+      out.value = org.apache.drill.exec.expr.fn.impl.HashHelper.hash64(in.value, seed.value);
     }
   }
 
@@ -338,7 +338,7 @@ public class Hash64FunctionsWithSeed {
       if (in.isSet == 0) {
         out.value = seed.value;
       } else {
-        out.value = org.apache.drill.exec.expr.fn.impl.XXHash.hash64(in.value, seed.value);
+        out.value = org.apache.drill.exec.expr.fn.impl.HashHelper.hash64(in.value, seed.value);
       }
     }
   }
@@ -354,7 +354,7 @@ public class Hash64FunctionsWithSeed {
     }
 
     public void eval() {
-      out.value = org.apache.drill.exec.expr.fn.impl.XXHash.hash64(in.value, seed.value);
+      out.value = org.apache.drill.exec.expr.fn.impl.HashHelper.hash64(in.value, seed.value);
     }
   }
 
@@ -372,7 +372,7 @@ public class Hash64FunctionsWithSeed {
       if (in.isSet == 0) {
         out.value = seed.value;
       } else {
-        out.value = org.apache.drill.exec.expr.fn.impl.XXHash.hash64(in.value, seed.value);
+        out.value = org.apache.drill.exec.expr.fn.impl.HashHelper.hash64(in.value, seed.value);
       }
     }
   }
@@ -388,7 +388,7 @@ public class Hash64FunctionsWithSeed {
     }
 
     public void eval() {
-      out.value = org.apache.drill.exec.expr.fn.impl.XXHash.hash64(in.value, seed.value);
+      out.value = org.apache.drill.exec.expr.fn.impl.HashHelper.hash64(in.value, seed.value);
     }
   }
 
@@ -406,7 +406,7 @@ public class Hash64FunctionsWithSeed {
       if (in.isSet == 0) {
         out.value = seed.value;
       } else {
-        out.value = org.apache.drill.exec.expr.fn.impl.XXHash.hash64(in.value, seed.value);
+        out.value = org.apache.drill.exec.expr.fn.impl.HashHelper.hash64(in.value, seed.value);
       }
     }
   }
@@ -422,7 +422,7 @@ public class Hash64FunctionsWithSeed {
     }
 
     public void eval() {
-      out.value = org.apache.drill.exec.expr.fn.impl.XXHash.hash64(in.value, seed.value);
+      out.value = org.apache.drill.exec.expr.fn.impl.HashHelper.hash64(in.value, seed.value);
     }
   }
 
@@ -440,7 +440,7 @@ public class Hash64FunctionsWithSeed {
       if (in.isSet == 0) {
         out.value = seed.value;
       } else {
-        out.value = org.apache.drill.exec.expr.fn.impl.XXHash.hash64(in.value, seed.value);
+        out.value = org.apache.drill.exec.expr.fn.impl.HashHelper.hash64(in.value, seed.value);
       }
     }
   }
@@ -456,7 +456,7 @@ public class Hash64FunctionsWithSeed {
     }
 
     public void eval() {
-      out.value = org.apache.drill.exec.expr.fn.impl.XXHash.hash64(in.value, seed.value);
+      out.value = org.apache.drill.exec.expr.fn.impl.HashHelper.hash64(in.value, seed.value);
     }
   }
 
@@ -474,7 +474,7 @@ public class Hash64FunctionsWithSeed {
       if (in.isSet == 0) {
         out.value = seed.value;
       } else {
-        out.value = org.apache.drill.exec.expr.fn.impl.XXHash.hash64(in.value, seed.value);
+        out.value = org.apache.drill.exec.expr.fn.impl.HashHelper.hash64(in.value, seed.value);
       }
     }
   }
@@ -490,7 +490,7 @@ public class Hash64FunctionsWithSeed {
     }
 
     public void eval() {
-      out.value = org.apache.drill.exec.expr.fn.impl.XXHash.hash64(in.start, in.start + Decimal28SparseHolder.WIDTH, in.buffer, seed.value);
+      out.value = org.apache.drill.exec.expr.fn.impl.HashHelper.hash64(in.start, in.start + Decimal28SparseHolder.WIDTH, in.buffer, seed.value);
     }
   }
 
@@ -508,7 +508,7 @@ public class Hash64FunctionsWithSeed {
       if (in.isSet == 0) {
         out.value = seed.value;
       } else {
-        out.value = org.apache.drill.exec.expr.fn.impl.XXHash.hash64(in.start, in.start + NullableDecimal28SparseHolder.WIDTH, in.buffer, seed.value);
+        out.value = org.apache.drill.exec.expr.fn.impl.HashHelper.hash64(in.start, in.start + NullableDecimal28SparseHolder.WIDTH, in.buffer, seed.value);
       }
     }
   }
@@ -524,7 +524,7 @@ public class Hash64FunctionsWithSeed {
     }
 
     public void eval() {
-      out.value = org.apache.drill.exec.expr.fn.impl.XXHash.hash64(in.start, in.start + Decimal38SparseHolder.WIDTH, in.buffer, seed.value);
+      out.value = org.apache.drill.exec.expr.fn.impl.HashHelper.hash64(in.start, in.start + Decimal38SparseHolder.WIDTH, in.buffer, seed.value);
     }
   }
 
@@ -542,7 +542,7 @@ public class Hash64FunctionsWithSeed {
       if (in.isSet == 0) {
         out.value = seed.value;
       } else {
-        out.value = org.apache.drill.exec.expr.fn.impl.XXHash.hash64(in.start, in.start + NullableDecimal38SparseHolder.WIDTH, in.buffer, seed.value);
+        out.value = org.apache.drill.exec.expr.fn.impl.HashHelper.hash64(in.start, in.start + NullableDecimal38SparseHolder.WIDTH, in.buffer, seed.value);
       }
     }
   }
@@ -562,7 +562,7 @@ public class Hash64FunctionsWithSeed {
       if (in.isSet == 0) {
         out.value = seed.value;
       } else {
-        out.value = org.apache.drill.exec.expr.fn.impl.XXHash.hash64(in.value, seed.value);
+        out.value = org.apache.drill.exec.expr.fn.impl.HashHelper.hash64(in.value, seed.value);
       }
     }
   }
@@ -579,6 +579,6 @@ public class Hash64FunctionsWithSeed {
     }
 
     public void eval() {
-      out.value = org.apache.drill.exec.expr.fn.impl.XXHash.hash64(in.value, seed.value);
+      out.value = org.apache.drill.exec.expr.fn.impl.HashHelper.hash64(in.value, seed.value);
     }
   }}

http://git-wip-us.apache.org/repos/asf/drill/blob/c6a03eb1/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/impl/Hash64WithSeedAsDouble.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/impl/Hash64WithSeedAsDouble.java b/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/impl/Hash64WithSeedAsDouble.java
index fbe1868..0079d98 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/impl/Hash64WithSeedAsDouble.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/impl/Hash64WithSeedAsDouble.java
@@ -63,7 +63,7 @@ public class Hash64WithSeedAsDouble {
       if (in.isSet == 0) {
         out.value = seed.value;
       } else {
-        out.value = org.apache.drill.exec.expr.fn.impl.XXHash.hash64((double) in.value, seed.value);
+        out.value = org.apache.drill.exec.expr.fn.impl.HashHelper.hash64((double) in.value, seed.value);
       }
     }
   }
@@ -80,7 +80,7 @@ public class Hash64WithSeedAsDouble {
     }
 
     public void eval() {
-      out.value = org.apache.drill.exec.expr.fn.impl.XXHash.hash64((double) in.value, seed.value);
+      out.value = org.apache.drill.exec.expr.fn.impl.HashHelper.hash64((double) in.value, seed.value);
     }
   }
 
@@ -99,7 +99,7 @@ public class Hash64WithSeedAsDouble {
       if (in.isSet == 0) {
         out.value = seed.value;
       } else {
-        out.value = org.apache.drill.exec.expr.fn.impl.XXHash.hash64(in.value, seed.value);
+        out.value = org.apache.drill.exec.expr.fn.impl.HashHelper.hash64(in.value, seed.value);
       }
     }
   }
@@ -116,7 +116,7 @@ public class Hash64WithSeedAsDouble {
     }
 
     public void eval() {
-      out.value = org.apache.drill.exec.expr.fn.impl.XXHash.hash64(in.value, seed.value);
+      out.value = org.apache.drill.exec.expr.fn.impl.HashHelper.hash64(in.value, seed.value);
     }
   }
 
@@ -136,7 +136,7 @@ public class Hash64WithSeedAsDouble {
         out.value = seed.value;
       }
       else {
-        out.value = org.apache.drill.exec.expr.fn.impl.XXHash.hash64((double) in.value, seed.value);
+        out.value = org.apache.drill.exec.expr.fn.impl.HashHelper.hash64((double) in.value, seed.value);
       }
     }
   }
@@ -156,7 +156,7 @@ public class Hash64WithSeedAsDouble {
         out.value = seed.value;
       }
       else {
-        out.value = org.apache.drill.exec.expr.fn.impl.XXHash.hash64((double) in.value, seed.value);
+        out.value = org.apache.drill.exec.expr.fn.impl.HashHelper.hash64((double) in.value, seed.value);
       }
     }
   }
@@ -173,7 +173,7 @@ public class Hash64WithSeedAsDouble {
     }
 
     public void eval() {
-      out.value = org.apache.drill.exec.expr.fn.impl.XXHash.hash64((double) in.value, seed.value);
+      out.value = org.apache.drill.exec.expr.fn.impl.HashHelper.hash64((double) in.value, seed.value);
     }
   }
 
@@ -189,7 +189,7 @@ public class Hash64WithSeedAsDouble {
 
     public void eval() {
       // TODO: implement hash function for other types
-      out.value = org.apache.drill.exec.expr.fn.impl.XXHash.hash64((double) in.value, seed.value);
+      out.value = org.apache.drill.exec.expr.fn.impl.HashHelper.hash64((double) in.value, seed.value);
     }
   }
 
@@ -205,7 +205,7 @@ public class Hash64WithSeedAsDouble {
 
     public void eval() {
       java.math.BigDecimal input = new java.math.BigDecimal(java.math.BigInteger.valueOf(in.value), in.scale);
-      out.value = org.apache.drill.exec.expr.fn.impl.XXHash.hash64(input.doubleValue(), seed.value);
+      out.value = org.apache.drill.exec.expr.fn.impl.HashHelper.hash64(input.doubleValue(), seed.value);
     }
   }
 
@@ -224,7 +224,7 @@ public class Hash64WithSeedAsDouble {
         out.value = seed.value;
       } else {
         java.math.BigDecimal input = new java.math.BigDecimal(java.math.BigInteger.valueOf(in.value), in.scale);
-        out.value = org.apache.drill.exec.expr.fn.impl.XXHash.hash64(input.doubleValue(), seed.value);
+        out.value = org.apache.drill.exec.expr.fn.impl.HashHelper.hash64(input.doubleValue(), seed.value);
       }
     }
   }
@@ -241,7 +241,7 @@ public class Hash64WithSeedAsDouble {
 
     public void eval() {
       java.math.BigDecimal input = new java.math.BigDecimal(java.math.BigInteger.valueOf(in.value), in.scale);
-      out.value = org.apache.drill.exec.expr.fn.impl.XXHash.hash64(input.doubleValue(), seed.value);
+      out.value = org.apache.drill.exec.expr.fn.impl.HashHelper.hash64(input.doubleValue(), seed.value);
     }
   }
 
@@ -260,7 +260,7 @@ public class Hash64WithSeedAsDouble {
         out.value = seed.value;
       } else {
         java.math.BigDecimal input = new java.math.BigDecimal(java.math.BigInteger.valueOf(in.value), in.scale);
-        out.value = org.apache.drill.exec.expr.fn.impl.XXHash.hash64(input.doubleValue(), seed.value);
+        out.value = org.apache.drill.exec.expr.fn.impl.HashHelper.hash64(input.doubleValue(), seed.value);
       }
     }
   }
@@ -278,7 +278,7 @@ public class Hash64WithSeedAsDouble {
     public void eval() {
       java.math.BigDecimal input = org.apache.drill.exec.util.DecimalUtility.getBigDecimalFromSparse(in.buffer,
           in.start, in.nDecimalDigits, in.scale);
-      out.value = org.apache.drill.exec.expr.fn.impl.XXHash.hash64(input.doubleValue(), 0);
+      out.value = org.apache.drill.exec.expr.fn.impl.HashHelper.hash64(input.doubleValue(), seed.value);
     }
   }
 
@@ -298,7 +298,7 @@ public class Hash64WithSeedAsDouble {
       } else {
         java.math.BigDecimal input = org.apache.drill.exec.util.DecimalUtility.getBigDecimalFromSparse(in.buffer,
             in.start, in.nDecimalDigits, in.scale);
-        out.value = org.apache.drill.exec.expr.fn.impl.XXHash.hash64(input.doubleValue(), 0);
+        out.value = org.apache.drill.exec.expr.fn.impl.HashHelper.hash64(input.doubleValue(), seed.value);
       }
     }
   }
@@ -316,7 +316,7 @@ public class Hash64WithSeedAsDouble {
     public void eval() {
       java.math.BigDecimal input = org.apache.drill.exec.util.DecimalUtility.getBigDecimalFromSparse(in.buffer,
           in.start, in.nDecimalDigits, in.scale);
-      out.value = org.apache.drill.exec.expr.fn.impl.XXHash.hash64(input.doubleValue(), 0);
+      out.value = org.apache.drill.exec.expr.fn.impl.HashHelper.hash64(input.doubleValue(), seed.value);
     }
   }
 
@@ -336,7 +336,7 @@ public class Hash64WithSeedAsDouble {
       } else {
         java.math.BigDecimal input = org.apache.drill.exec.util.DecimalUtility.getBigDecimalFromSparse(in.buffer,
             in.start, in.nDecimalDigits, in.scale);
-        out.value = org.apache.drill.exec.expr.fn.impl.XXHash.hash64(input.doubleValue(), 0);
+        out.value = org.apache.drill.exec.expr.fn.impl.HashHelper.hash64(input.doubleValue(), seed.value);
       }
     }
   }

http://git-wip-us.apache.org/repos/asf/drill/blob/c6a03eb1/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/impl/HashHelper.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/impl/HashHelper.java b/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/impl/HashHelper.java
index 0e9509a..22d0d82 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/impl/HashHelper.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/impl/HashHelper.java
@@ -17,12 +17,12 @@
  */
 package org.apache.drill.exec.expr.fn.impl;
 
+import io.netty.buffer.DrillBuf;
+
 import java.nio.ByteBuffer;
 import java.nio.ByteOrder;
 
 public class HashHelper {
-  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(HashHelper.class);
-
 
   /** taken from mahout **/
   public static int hash(ByteBuffer buf, int seed) {
@@ -60,4 +60,45 @@ public class HashHelper {
     return h;
   }
 
+  public static int hash32(int val, long seed) {
+    double converted = val;
+    return hash32(converted, seed);
+  }
+  public static int hash32(long val, long seed) {
+    double converted = val;
+    return hash32(converted, seed);
+  }
+  public static int hash32(float val, long seed){
+    double converted = val;
+    return hash32(converted, seed);
+  }
+
+
+  public static long hash64(float val, long seed){
+    double converted = val;
+    return hash64(converted, seed);
+  }
+  public static long hash64(long val, long seed){
+    double converted = val;
+    return hash64(converted, seed);
+  }
+
+  public static long hash64(double val, long seed){
+    return MurmurHash3.hash64(val, (int)seed);
+  }
+
+  public static long hash64(long start, long end, DrillBuf buffer, long seed){
+    return MurmurHash3.hash64(start, end, buffer, (int)seed);
+  }
+
+  public static int hash32(double val, long seed) {
+    //return com.google.common.hash.Hashing.murmur3_128().hashLong(Double.doubleToLongBits(val)).asInt();
+    return MurmurHash3.hash32(val, (int)seed);
+  }
+
+  public static int hash32(int start, int end, DrillBuf buffer, int seed){
+    return MurmurHash3.hash32(start, end, buffer, seed);
+  }
+
 }
+

http://git-wip-us.apache.org/repos/asf/drill/blob/c6a03eb1/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/impl/MurmurHash3.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/impl/MurmurHash3.java b/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/impl/MurmurHash3.java
new file mode 100644
index 0000000..b21117b
--- /dev/null
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/impl/MurmurHash3.java
@@ -0,0 +1,280 @@
+/**
+ * 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.drill.exec.expr.fn.impl;
+
+import io.netty.buffer.DrillBuf;
+import io.netty.util.internal.PlatformDependent;
+
+
+/**
+ *
+ * MurmurHash3 was written by Austin Appleby, and is placed in the public
+ * domain.
+ * See http://smhasher.googlecode.com/svn/trunk/MurmurHash3.cpp
+ * MurmurHash3_x64_128
+ * MurmurHash3_x86_32
+ */
+public final class MurmurHash3 extends DrillHash{
+
+   public static final long fmix64(long k) {
+    k ^= k >>> 33;
+    k *= 0xff51afd7ed558ccdL;
+    k ^= k >>> 33;
+    k *= 0xc4ceb9fe1a85ec53L;
+    k ^= k >>> 33;
+    return k;
+  }
+
+  /*
+  Take 64 bit of murmur3_128's output
+   */
+  public static long murmur3_64(long bStart, long bEnd, DrillBuf buffer, int seed) {
+
+    long h1 = seed & 0x00000000FFFFFFFFL;
+    long h2 = seed & 0x00000000FFFFFFFFL;
+
+    final long c1 = 0x87c37b91114253d5L;
+    final long c2 = 0x4cf5ad432745937fL;
+    long start = buffer.memoryAddress() + bStart;
+    long end = buffer.memoryAddress() + bEnd;
+    long length = bEnd - bStart;
+    long roundedEnd = start + ( length & 0xFFFFFFF0);  // round down to 16 byte block
+    for (long i=start; i<roundedEnd; i+=16) {
+      long k1 = getLongLittleEndian(i);
+      long k2 = getLongLittleEndian(i+8);
+      k1 *= c1;
+      k1  = Long.rotateLeft(k1,31);
+      k1 *= c2;
+      h1 ^= k1;
+      h1 = Long.rotateLeft(h1,27);
+      h1 += h2;
+      h1 = h1*5+0x52dce729;
+      k2 *= c2;
+      k2  = Long.rotateLeft(k2,33);
+      k2 *= c1;
+      h2 ^= k2;
+      h2 = Long.rotateLeft(h2,31);
+      h2 += h1;
+      h2 = h2*5+0x38495ab5;
+    }
+
+    long k1 = 0;
+    long k2 = 0;
+
+    // tail
+    switch ((int)length & 15) {
+      case 15: k2  = (PlatformDependent.getByte(roundedEnd+14) & 0xffL) << 48;
+      case 14: k2 ^= (PlatformDependent.getByte(roundedEnd+13) & 0xffL) << 40;
+      case 13: k2 ^= (PlatformDependent.getByte(roundedEnd+12) & 0xffL) << 32;
+      case 12: k2 ^= (PlatformDependent.getByte(roundedEnd+11) & 0xffL) << 24;
+      case 11: k2 ^= (PlatformDependent.getByte(roundedEnd+10) & 0xffL) << 16;
+      case 10: k2 ^= (PlatformDependent.getByte(roundedEnd+ 9) & 0xffL) << 8;
+      case  9: k2 ^= (PlatformDependent.getByte(roundedEnd+ 8) & 0xffL);
+        k2 *= c2;
+        k2  = Long.rotateLeft(k2, 33);
+        k2 *= c1;
+        h2 ^= k2;
+      case  8: k1  = (long)PlatformDependent.getByte(roundedEnd+7) << 56;
+      case  7: k1 ^= (PlatformDependent.getByte(roundedEnd+6) & 0xffL) << 48;
+      case  6: k1 ^= (PlatformDependent.getByte(roundedEnd+5) & 0xffL) << 40;
+      case  5: k1 ^= (PlatformDependent.getByte(roundedEnd+4) & 0xffL) << 32;
+      case  4: k1 ^= (PlatformDependent.getByte(roundedEnd+3) & 0xffL) << 24;
+      case  3: k1 ^= (PlatformDependent.getByte(roundedEnd+2) & 0xffL) << 16;
+      case  2: k1 ^= (PlatformDependent.getByte(roundedEnd+1) & 0xffL) << 8;
+      case  1: k1 ^= (PlatformDependent.getByte(roundedEnd ) & 0xffL);
+        k1 *= c1;
+        k1  = Long.rotateLeft(k1,31);
+        k1 *= c2;
+        h1 ^= k1;
+    }
+
+    h1 ^= length;
+    h2 ^= length;
+
+    h1 += h2;
+    h2 += h1;
+
+    h1 = fmix64(h1);
+    h2 = fmix64(h2);
+
+    h1 += h2;
+    h2 += h1;
+    // murmur3_128 should return 128 bit (h1,h2), now we return only 64bits,
+    return h1;
+  }
+
+  public static long murmur3_64(long val, int seed) {
+
+    long h1 = seed & 0x00000000FFFFFFFFL;
+    long h2 = seed & 0x00000000FFFFFFFFL;
+
+    final long c1 = 0x87c37b91114253d5L;
+    final long c2 = 0x4cf5ad432745937fL;
+
+    int length = 8;
+    long k1 = 0;
+
+    k1 = val;
+    k1 *= c1;
+    k1  = Long.rotateLeft(k1,31);
+    k1 *= c2;
+    h1 ^= k1;
+
+    h1 ^= length;
+    h2 ^= length;
+
+    h1 += h2;
+    h2 += h1;
+
+    h1 = fmix64(h1);
+    h2 = fmix64(h2);
+
+    h1 += h2;
+
+    //h2 += h1;
+    // murmur3_128 should return 128 bit (h1,h2), now we return only 64bits,
+    return h1;
+
+  }
+
+  public static int murmur3_32(int bStart, int bEnd, DrillBuf buffer, int seed) {
+
+    final long c1 = 0xcc9e2d51L;
+    final long c2 = 0x1b873593L;
+    long start = buffer.memoryAddress() + bStart;
+    long length = bEnd - bStart;
+    long UINT_MASK=0xffffffffL;
+    long lh1 = seed;
+    long roundedEnd = start + (length & 0xfffffffc);  // round down to 4 byte block
+
+    for (long i=start; i<roundedEnd; i+=4) {
+      // little endian load order
+      long lk1 = (PlatformDependent.getByte(i) & 0xff) | ((PlatformDependent.getByte(i+1) & 0xff) << 8) |
+              ((PlatformDependent.getByte(i+2) & 0xff) << 16) | (PlatformDependent.getByte(i+3) << 24);
+
+      //k1 *= c1;
+      lk1 *= c1;
+      lk1 &= UINT_MASK;
+
+      lk1 = ((lk1 << 15) & UINT_MASK) | (lk1 >>> 17);
+
+      lk1 *= c2;
+      lk1 = lk1 & UINT_MASK;
+      lh1 ^= lk1;
+      lh1 = ((lh1 << 13) & UINT_MASK) | (lh1 >>> 19);
+
+      lh1 = lh1*5+0xe6546b64L;
+      lh1 = UINT_MASK & lh1;
+    }
+
+    // tail
+    long lk1 = 0;
+
+    switch((byte)length & 0x03) {
+      case 3:
+        lk1 = (PlatformDependent.getByte(roundedEnd + 2) & 0xff) << 16;
+      case 2:
+        lk1 |= (PlatformDependent.getByte(roundedEnd + 1) & 0xff) << 8;
+      case 1:
+        lk1 |= (PlatformDependent.getByte(roundedEnd) & 0xff);
+        lk1 *= c1;
+        lk1 = UINT_MASK & lk1;
+        lk1 = ((lk1 << 15) & UINT_MASK) | (lk1 >>> 17);
+
+        lk1 *= c2;
+        lk1 = lk1 & UINT_MASK;
+
+        lh1 ^= lk1;
+    }
+
+    // finalization
+    lh1 ^= length;
+
+    lh1 ^= lh1 >>> 16;
+    lh1 *= 0x85ebca6b;
+    lh1 = UINT_MASK & lh1;
+    lh1 ^= lh1 >>> 13;
+
+    lh1 *= 0xc2b2ae35;
+    lh1 = UINT_MASK & lh1;
+    lh1 ^= lh1 >>> 16;
+
+    return (int)(lh1 & UINT_MASK);
+  }
+
+  public static int murmur3_32(long val, int seed) {
+    final long c1 = 0xcc9e2d51L;
+    final long c2 = 0x1b873593;
+    long length = 8;
+    long UINT_MASK=0xffffffffL;
+    long lh1 = seed & UINT_MASK;
+    for (int i=0; i<2; i++) {
+      //int ik1 = (int)((val >> i*32) & UINT_MASK);
+      long lk1 = ((val >> i*32) & UINT_MASK);
+
+      //k1 *= c1;
+      lk1 *= c1;
+      lk1 &= UINT_MASK;
+
+      lk1 = ((lk1 << 15) & UINT_MASK) | (lk1 >>> 17);
+
+      lk1 *= c2;
+      lk1 &= UINT_MASK;
+
+      lh1 ^= lk1;
+      lh1 = ((lh1 << 13) & UINT_MASK) | (lh1 >>> 19);
+
+      lh1 = lh1*5+0xe6546b64L;
+      lh1 = UINT_MASK & lh1;
+    }
+    // finalization
+    lh1 ^= length;
+
+    lh1 ^= lh1 >>> 16;
+    lh1 *= 0x85ebca6bL;
+    lh1 = UINT_MASK & lh1;
+    lh1 ^= lh1 >>> 13;
+    lh1 *= 0xc2b2ae35L;
+    lh1 = UINT_MASK & lh1;
+    lh1 ^= lh1 >>> 16;
+
+    return (int)lh1;
+  }
+
+  public static long hash64(double val, long seed){
+    return murmur3_64(Double.doubleToLongBits(val), (int)seed);
+  }
+
+  public static long hash64(long start, long end, DrillBuf buffer, long seed){
+    return murmur3_64(start, end, buffer, (int)seed);
+  }
+
+  public static int hash32(double val, long seed) {
+    //return com.google.common.hash.Hashing.murmur3_128().hashLong(Double.doubleToLongBits(val)).asInt();
+    return (int)murmur3_64(Double.doubleToLongBits(val), (int)seed);
+  }
+
+  public static int hash32(int start, int end, DrillBuf buffer, int seed){
+    return (int)murmur3_64(start, end, buffer, seed);
+  }
+
+}
+