You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hive.apache.org by we...@apache.org on 2017/05/08 22:17:30 UTC

[13/50] [abbrv] hive git commit: HIVE-16513 : width_bucket issues (Sahil Takiar via Ashutosh Chauhan)

HIVE-16513 : width_bucket issues (Sahil Takiar via Ashutosh Chauhan)

Signed-off-by: Ashutosh Chauhan <ha...@apache.org>


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

Branch: refs/heads/hive-14535
Commit: f8f9155dab346a2a112ba2bb4b9162b98cab404c
Parents: c6b5ad6
Author: Sahil Takiar <ta...@gmail.com>
Authored: Thu May 4 10:01:02 2017 -0700
Committer: Ashutosh Chauhan <ha...@apache.org>
Committed: Thu May 4 10:01:02 2017 -0700

----------------------------------------------------------------------
 .../ql/udf/generic/GenericUDFWidthBucket.java   | 279 ++++++++-
 .../queries/clientpositive/udf_width_bucket.q   | 175 ++++++
 .../clientpositive/udf_width_bucket.q.out       | 569 +++++++++++++++++++
 3 files changed, 995 insertions(+), 28 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hive/blob/f8f9155d/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDFWidthBucket.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDFWidthBucket.java b/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDFWidthBucket.java
index 3ba24ed..bcf5c49 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDFWidthBucket.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDFWidthBucket.java
@@ -18,13 +18,25 @@
 package org.apache.hadoop.hive.ql.udf.generic;
 
 import com.google.common.base.Preconditions;
+
+import org.apache.hadoop.hive.common.type.HiveDecimal;
 import org.apache.hadoop.hive.ql.exec.Description;
+import org.apache.hadoop.hive.ql.exec.FunctionRegistry;
 import org.apache.hadoop.hive.ql.exec.UDFArgumentException;
 import org.apache.hadoop.hive.ql.metadata.HiveException;
+import org.apache.hadoop.hive.serde2.io.ByteWritable;
+import org.apache.hadoop.hive.serde2.io.DoubleWritable;
+import org.apache.hadoop.hive.serde2.io.HiveDecimalWritable;
+import org.apache.hadoop.hive.serde2.io.ShortWritable;
 import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspector;
 import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspectorConverters;
 import org.apache.hadoop.hive.serde2.objectinspector.PrimitiveObjectInspector;
 import org.apache.hadoop.hive.serde2.objectinspector.primitive.PrimitiveObjectInspectorFactory;
+import org.apache.hadoop.hive.serde2.objectinspector.primitive.PrimitiveObjectInspectorUtils;
+import org.apache.hadoop.hive.serde2.typeinfo.TypeInfo;
+import org.apache.hadoop.hive.serde2.typeinfo.TypeInfoUtils;
+
+import org.apache.hadoop.io.FloatWritable;
 import org.apache.hadoop.io.IntWritable;
 import org.apache.hadoop.io.LongWritable;
 
@@ -33,22 +45,27 @@ import static org.apache.hadoop.hive.serde2.objectinspector.primitive.PrimitiveO
 
 
 @Description(name = "width_bucket",
-    value = "_FUNC_(expr, min_value, max_value, num_buckets) - Returns an integer between 0 and num_buckets+1 by "
-        + "mapping the expr into buckets defined by the range [min_value, max_value]",
-    extended = "Returns an integer between 0 and num_buckets+1 by "
-        + "mapping expr into the ith equally sized bucket. Buckets are made by dividing [min_value, max_value] into "
-        + "equally sized regions. If expr < min_value, return 1, if expr > max_value return num_buckets+1\n"
-        + "Example: expr is an integer column withs values 1, 10, 20, 30.\n"
-        + "  > SELECT _FUNC_(expr, 5, 25, 4) FROM src;\n1\n1\n3\n5")
+        value = "_FUNC_(expr, min_value, max_value, num_buckets) - Returns an integer between 0 and num_buckets+1 by "
+                + "mapping the expr into buckets defined by the range [min_value, max_value]",
+        extended = "Returns an integer between 0 and num_buckets+1 by "
+                + "mapping expr into the ith equally sized bucket. Buckets are made by dividing [min_value, max_value] into "
+                + "equally sized regions. If expr < min_value, return 1, if expr > max_value return num_buckets+1\n"
+                + "Example: expr is an integer column withs values 1, 10, 20, 30.\n"
+                + "  > SELECT _FUNC_(expr, 5, 25, 4) FROM src;\n1\n1\n3\n5")
 public class GenericUDFWidthBucket extends GenericUDF {
 
-  private transient PrimitiveObjectInspector.PrimitiveCategory[] inputTypes = new PrimitiveObjectInspector.PrimitiveCategory[4];
-  private transient ObjectInspectorConverters.Converter[] converters = new ObjectInspectorConverters.Converter[4];
+  private transient ObjectInspector[] objectInspectors;
+  private transient ObjectInspector commonExprMinMaxOI;
+  private transient ObjectInspectorConverters.Converter epxrConverterOI;
+  private transient ObjectInspectorConverters.Converter minValueConverterOI;
+  private transient ObjectInspectorConverters.Converter maxValueConverterOI;
 
   private final IntWritable output = new IntWritable();
 
   @Override
   public ObjectInspector initialize(ObjectInspector[] arguments) throws UDFArgumentException {
+    this.objectInspectors = arguments;
+
     checkArgsSize(arguments, 4, 4);
 
     checkArgPrimitive(arguments, 0);
@@ -56,43 +73,249 @@ public class GenericUDFWidthBucket extends GenericUDF {
     checkArgPrimitive(arguments, 2);
     checkArgPrimitive(arguments, 3);
 
+    PrimitiveObjectInspector.PrimitiveCategory[] inputTypes = new PrimitiveObjectInspector.PrimitiveCategory[4];
     checkArgGroups(arguments, 0, inputTypes, NUMERIC_GROUP, VOID_GROUP);
     checkArgGroups(arguments, 1, inputTypes, NUMERIC_GROUP, VOID_GROUP);
     checkArgGroups(arguments, 2, inputTypes, NUMERIC_GROUP, VOID_GROUP);
     checkArgGroups(arguments, 3, inputTypes, NUMERIC_GROUP, VOID_GROUP);
 
-    obtainLongConverter(arguments, 0, inputTypes, converters);
-    obtainLongConverter(arguments, 1, inputTypes, converters);
-    obtainLongConverter(arguments, 2, inputTypes, converters);
-    obtainIntConverter(arguments, 3, inputTypes, converters);
+    TypeInfo exprTypeInfo = TypeInfoUtils.getTypeInfoFromObjectInspector(this.objectInspectors[0]);
+    TypeInfo minValueTypeInfo = TypeInfoUtils.getTypeInfoFromObjectInspector(this.objectInspectors[1]);
+    TypeInfo maxValueTypeInfo = TypeInfoUtils.getTypeInfoFromObjectInspector(this.objectInspectors[2]);
+
+    TypeInfo commonExprMinMaxTypeInfo = FunctionRegistry.getCommonClassForComparison(exprTypeInfo,
+            FunctionRegistry.getCommonClassForComparison(minValueTypeInfo, maxValueTypeInfo));
+
+    this.commonExprMinMaxOI = TypeInfoUtils.getStandardWritableObjectInspectorFromTypeInfo(commonExprMinMaxTypeInfo);
+
+    this.epxrConverterOI = ObjectInspectorConverters.getConverter(this.objectInspectors[0], this.commonExprMinMaxOI);
+    this.minValueConverterOI = ObjectInspectorConverters.getConverter(this.objectInspectors[1], this.commonExprMinMaxOI);
+    this.maxValueConverterOI = ObjectInspectorConverters.getConverter(this.objectInspectors[2], this.commonExprMinMaxOI);
 
     return PrimitiveObjectInspectorFactory.writableIntObjectInspector;
   }
 
   @Override
   public Object evaluate(DeferredObject[] arguments) throws HiveException {
-    Long exprValue = getLongValue(arguments, 0, converters);
-    Long minValue = getLongValue(arguments, 1, converters);
-    Long maxValue = getLongValue(arguments, 2, converters);
-    Integer numBuckets = getIntValue(arguments, 3, converters);
-
-    if (exprValue == null || minValue == null || maxValue == null || numBuckets == null) {
+    if (arguments[0].get() == null || arguments[1].get() == null || arguments[2].get() == null || arguments[3].get() == null) {
       return null;
     }
 
+    Object exprValue = this.epxrConverterOI.convert(arguments[0].get());
+    Object minValue = this.minValueConverterOI.convert(arguments[1].get());
+    Object maxValue = this.maxValueConverterOI.convert(arguments[2].get());
+
+    int numBuckets = PrimitiveObjectInspectorUtils.getInt(arguments[3].get(),
+            (PrimitiveObjectInspector) this.objectInspectors[3]);
+
+    switch (((PrimitiveObjectInspector) this.commonExprMinMaxOI).getPrimitiveCategory()) {
+      case SHORT:
+        return evaluate(((ShortWritable) exprValue).get(), ((ShortWritable) minValue).get(),
+                ((ShortWritable) maxValue).get(), numBuckets);
+      case INT:
+        return evaluate(((IntWritable) exprValue).get(), ((IntWritable) minValue).get(),
+                ((IntWritable) maxValue).get(), numBuckets);
+      case LONG:
+        return evaluate(((LongWritable) exprValue).get(), ((LongWritable) minValue).get(),
+                ((LongWritable) maxValue).get(), numBuckets);
+      case FLOAT:
+        return evaluate(((FloatWritable) exprValue).get(), ((FloatWritable) minValue).get(),
+                ((FloatWritable) maxValue).get(), numBuckets);
+      case DOUBLE:
+        return evaluate(((DoubleWritable) exprValue).get(), ((DoubleWritable) minValue).get(),
+                ((DoubleWritable) maxValue).get(), numBuckets);
+      case DECIMAL:
+        return evaluate(((HiveDecimalWritable) exprValue).getHiveDecimal(),
+                ((HiveDecimalWritable) minValue).getHiveDecimal(), ((HiveDecimalWritable) maxValue).getHiveDecimal(),
+                numBuckets);
+      case BYTE:
+        return evaluate(((ByteWritable) exprValue).get(), ((ByteWritable) minValue).get(),
+                ((ByteWritable) maxValue).get(), numBuckets);
+      default:
+        throw new IllegalStateException(
+                "Error: width_bucket could not determine a common primitive type for all inputs");
+    }
+  }
+
+  private IntWritable evaluate(short exprValue, short minValue, short maxValue, int numBuckets) {
+
+    Preconditions.checkArgument(numBuckets > 0, "numBuckets in width_bucket function must be above 0");
+    Preconditions.checkArgument(maxValue != minValue, "maxValue cannot be equal to minValue in width_bucket function");
+
+    if (maxValue > minValue) {
+      if (exprValue < minValue) {
+        output.set(0);
+      } else if (exprValue >= maxValue) {
+        output.set(numBuckets + 1);
+      } else {
+        output.set((int) Math.floor((numBuckets * (exprValue - minValue) / (maxValue - minValue)) + 1));
+      }
+    } else {
+      if (exprValue > minValue) {
+        output.set(0);
+      } else if (exprValue <= maxValue) {
+        output.set(numBuckets + 1);
+      } else {
+        output.set((int) Math.floor((numBuckets * (minValue - exprValue) / (minValue - maxValue)) + 1));
+      }
+    }
+
+    return output;
+  }
+
+  private IntWritable evaluate(int exprValue, int minValue, int maxValue, int numBuckets) {
+
+    Preconditions.checkArgument(numBuckets > 0, "numBuckets in width_bucket function must be above 0");
+    Preconditions.checkArgument(maxValue != minValue, "maxValue cannot be equal to minValue in width_bucket function");
+
+    if (maxValue > minValue) {
+      if (exprValue < minValue) {
+        output.set(0);
+      } else if (exprValue >= maxValue) {
+        output.set(numBuckets + 1);
+      } else {
+        output.set((int) Math.floor((numBuckets * (exprValue - minValue) / (maxValue - minValue)) + 1));
+      }
+    } else {
+      if (exprValue > minValue) {
+        output.set(0);
+      } else if (exprValue <= maxValue) {
+        output.set(numBuckets + 1);
+      } else {
+        output.set((int) Math.floor((numBuckets * (minValue - exprValue) / (minValue - maxValue)) + 1));
+      }
+    }
+
+    return output;
+  }
+
+  private IntWritable evaluate(long exprValue, long minValue, long maxValue, int numBuckets) {
+
     Preconditions.checkArgument(numBuckets > 0, "numBuckets in width_bucket function must be above 0");
-    long intervalSize = (maxValue - minValue) / numBuckets;
+    Preconditions.checkArgument(maxValue != minValue, "maxValue cannot be equal to minValue in width_bucket function");
+
+    if (maxValue > minValue) {
+      if (exprValue < minValue) {
+        output.set(0);
+      } else if (exprValue >= maxValue) {
+        output.set(numBuckets + 1);
+      } else {
+        output.set((int) Math.floor((numBuckets * (exprValue - minValue) / (maxValue - minValue)) + 1));
+      }
+    } else {
+      if (exprValue > minValue) {
+        output.set(0);
+      } else if (exprValue <= maxValue) {
+        output.set(numBuckets + 1);
+      } else {
+        output.set((int) Math.floor((numBuckets * (minValue - exprValue) / (minValue - maxValue)) + 1));
+      }
+    }
 
-    if (exprValue < minValue) {
-      output.set(0);
-    } else if (exprValue > maxValue) {
-      output.set(numBuckets + 1);
+    return output;
+  }
+
+  private IntWritable evaluate(float exprValue, float minValue, float maxValue, int numBuckets) {
+
+    Preconditions.checkArgument(numBuckets > 0, "numBuckets in width_bucket function must be above 0");
+    Preconditions.checkArgument(maxValue != minValue, "maxValue cannot be equal to minValue in width_bucket function");
+
+    if (maxValue > minValue) {
+      if (exprValue < minValue) {
+        output.set(0);
+      } else if (exprValue >= maxValue) {
+        output.set(numBuckets + 1);
+      } else {
+        output.set((int) Math.floor((numBuckets * (exprValue - minValue) / (maxValue - minValue)) + 1));
+      }
+    } else {
+      if (exprValue > minValue) {
+        output.set(0);
+      } else if (exprValue <= maxValue) {
+        output.set(numBuckets + 1);
+      } else {
+        output.set((int) Math.floor((numBuckets * (minValue - exprValue) / (minValue - maxValue)) + 1));
+      }
+    }
+
+    return output;
+  }
+
+  private IntWritable evaluate(double exprValue, double minValue, double maxValue, int numBuckets) {
+
+    Preconditions.checkArgument(numBuckets > 0, "numBuckets in width_bucket function must be above 0");
+    Preconditions.checkArgument(maxValue != minValue, "maxValue cannot be equal to minValue in width_bucket function");
+
+    if (maxValue > minValue) {
+      if (exprValue < minValue) {
+        output.set(0);
+      } else if (exprValue >= maxValue) {
+        output.set(numBuckets + 1);
+      } else {
+        output.set((int) Math.floor((numBuckets * (exprValue - minValue) / (maxValue - minValue)) + 1));
+      }
+    } else {
+      if (exprValue > minValue) {
+        output.set(0);
+      } else if (exprValue <= maxValue) {
+        output.set(numBuckets + 1);
+      } else {
+        output.set((int) Math.floor((numBuckets * (minValue - exprValue) / (minValue - maxValue)) + 1));
+      }
+    }
+
+    return output;
+  }
+
+  private IntWritable evaluate(HiveDecimal exprValue, HiveDecimal minValue, HiveDecimal maxValue,
+                                      int numBuckets) {
+
+    Preconditions.checkArgument(numBuckets > 0, "numBuckets in width_bucket function must be above 0");
+    Preconditions.checkArgument(!maxValue.equals(minValue),
+            "maxValue cannot be equal to minValue in width_bucket function");
+
+    if (maxValue.compareTo(minValue) > 0) {
+      if (exprValue.compareTo(minValue) < 0) {
+        output.set(0);
+      } else if (exprValue.compareTo(maxValue) >= 0) {
+        output.set(numBuckets + 1);
+      } else {
+        output.set(HiveDecimal.create(numBuckets).multiply(exprValue.subtract(minValue)).divide(
+                maxValue.subtract(minValue)).add(HiveDecimal.ONE).intValue());
+      }
+    } else {
+      if (exprValue.compareTo(minValue) > 0) {
+        output.set(0);
+      } else if (exprValue.compareTo(maxValue) <= 0) {
+        output.set(numBuckets + 1);
+      } else {
+        output.set(HiveDecimal.create(numBuckets).multiply(minValue.subtract(exprValue)).divide(
+                minValue.subtract(maxValue)).add(HiveDecimal.ONE).intValue());
+      }
+    }
+
+    return output;
+  }
+
+   private Object evaluate(byte exprValue, byte minValue, byte maxValue, int numBuckets) {
+         Preconditions.checkArgument(numBuckets > 0, "numBuckets in width_bucket function must be above 0");
+    Preconditions.checkArgument(maxValue != minValue, "maxValue cannot be equal to minValue in width_bucket function");
+
+    if (maxValue > minValue) {
+      if (exprValue < minValue) {
+        output.set(0);
+      } else if (exprValue >= maxValue) {
+        output.set(numBuckets + 1);
+      } else {
+        output.set((int) Math.floor((numBuckets * (exprValue - minValue) / (maxValue - minValue)) + 1));
+      }
     } else {
-      long diff = exprValue - minValue;
-      if (diff % intervalSize == 0) {
-        output.set((int) (diff/intervalSize + 1));
+      if (exprValue > minValue) {
+        output.set(0);
+      } else if (exprValue <= maxValue) {
+        output.set(numBuckets + 1);
       } else {
-        output.set((int) Math.ceil((double) (diff) / intervalSize));
+        output.set((int) Math.floor((numBuckets * (minValue - exprValue) / (minValue - maxValue)) + 1));
       }
     }
 

http://git-wip-us.apache.org/repos/asf/hive/blob/f8f9155d/ql/src/test/queries/clientpositive/udf_width_bucket.q
----------------------------------------------------------------------
diff --git a/ql/src/test/queries/clientpositive/udf_width_bucket.q b/ql/src/test/queries/clientpositive/udf_width_bucket.q
index 6ac60d6..dfdbee1 100644
--- a/ql/src/test/queries/clientpositive/udf_width_bucket.q
+++ b/ql/src/test/queries/clientpositive/udf_width_bucket.q
@@ -3,12 +3,14 @@ desc function extended width_bucket;
 
 explain select width_bucket(10, 5, 25, 4);
 
+-- Test with constants
 select
 width_bucket(1, 5, 25, 4),
 width_bucket(10, 5, 25, 4),
 width_bucket(20, 5, 25, 4),
 width_bucket(30, 5, 25, 4);
 
+-- Test with NULLs
 select
 width_bucket(1, NULL, 25, 4),
 width_bucket(NULL, 5, 25, 4),
@@ -16,14 +18,187 @@ width_bucket(20, 5, NULL, 4),
 width_bucket(30, 5, 25, NULL),
 width_bucket(NULL, NULL, NULL, NULL);
 
+-- Test with negative values
 select
 width_bucket(-1, -25, -5, 4),
 width_bucket(-10, -25, -5, 4),
 width_bucket(-20, -25, -5, 4),
 width_bucket(-30, -25, -5, 4);
 
+-- Test with positive and negative values
 select
 width_bucket(-10, -5, 15, 4),
 width_bucket(0, -5, 15, 4),
 width_bucket(10, -5, 15, 4),
 width_bucket(20, -5, 15, 4);
+
+-- Test with decimals
+select
+width_bucket(0.1, 0, 1, 10),
+width_bucket(0.25, 0, 1, 10),
+width_bucket(0.3456, 0, 1, 10),
+width_bucket(0.654321, 0, 1, 10);
+
+-- Test with negative decimals
+select
+width_bucket(-0.5, -1.5, 1.5, 10),
+width_bucket(-0.3, -1.5, 1.5, 10),
+width_bucket(-0.25, -1.5, 1.5, 10),
+width_bucket(0, -1.5, 1.5, 10),
+width_bucket(0.75, -1.5, 1.5, 10),
+width_bucket(1.25, -1.5, 1.5, 10),
+width_bucket(1.5, -1.5, 1.5, 10);
+
+-- Test with minValue > maxValue
+select
+width_bucket(1, 25, 5, 4),
+width_bucket(10, 25, 5, 4),
+width_bucket(20, 25, 5, 4),
+width_bucket(30, 25, 5, 4);
+
+-- Test with minValue > maxValue, with positive and negative values
+select
+width_bucket(-10, 15, -5, 4),
+width_bucket(0, 15, -5, 4),
+width_bucket(10, 15, -5, 4),
+width_bucket(20, 15, -5, 4);
+
+-- Test with minValue > maxValue, with decimals
+select
+width_bucket(0.1, 1, 0, 10),
+width_bucket(0.25, 1, 0, 10),
+width_bucket(0.3456, 1, 0, 10),
+width_bucket(0.654321, 1, 0, 10);
+
+-- Test with small decimal values
+create table alldecimaltypes(
+    cfloat FLOAT,
+    cdouble DOUBLE);
+
+insert into table alldecimaltypes values (0.1, 0.1), (0.25, 0.25), (0.3456, 0.3456), (0.654321, 0.654321);
+
+select
+width_bucket(cfloat, 0, 1, 10),
+width_bucket(cdouble, 0, 1, 10)
+from alldecimaltypes;
+
+select
+width_bucket(cfloat, 0, 1.5, 10),
+width_bucket(cdouble, -1.5, 0, 10),
+width_bucket(0.25, cfloat, 2, 10),
+width_bucket(0.25, 0, cdouble, 10)
+from alldecimaltypes;
+
+-- Test with all numeric types
+create table alltypes(
+    ctinyint TINYINT,
+    csmallint SMALLINT,
+    cint INT,
+    cbigint BIGINT,
+    cfloat FLOAT,
+    cdouble DOUBLE);
+
+insert into table alltypes values
+(0, 0, 0, 0, 0.0, 0.0),
+(1, 1, 1, 1, 1.0, 1.0),
+(25, 25, 25, 25, 25.0, 25.0),
+(60, 60, 60, 60, 60.0, 60.0),
+(72, 72, 72, 72, 72.0, 72.0),
+(100, 100, 100, 100, 100.0, 100.0);
+
+-- Test each numeric type individually
+select
+width_bucket(ctinyint, 0, 100, 10),
+width_bucket(csmallint, 0, 100, 10),
+width_bucket(cint, 0, 100, 10),
+width_bucket(cbigint, 0, 100, 10),
+width_bucket(cfloat, 0, 100, 10),
+width_bucket(cdouble, 0, 100, 10)
+from alltypes;
+
+truncate table alltypes;
+
+insert into table alltypes values (5, 5, 5, 10, 4.5, 7.25);
+
+-- Test different numeric types in a single query
+select
+width_bucket(cdouble, ctinyint, cbigint, 10),
+width_bucket(cdouble, csmallint, cbigint, 10),
+width_bucket(cdouble, cint, cbigint, 10),
+width_bucket(cdouble, cfloat, cbigint, 10)
+from alltypes;
+
+-- Test all tinyints
+create table alltinyints (
+    ctinyint1 TINYINT,
+    ctinyint2 TINYINT,
+    ctinyint3 TINYINT,
+    cint INT);
+
+insert into table alltinyints values (5, 1, 10, 2);
+select width_bucket(ctinyint1, ctinyint2, ctinyint3, cint) from alltinyints;
+
+-- Test all smallints
+create table allsmallints (
+    csmallint1 SMALLINT,
+    csmallint2 SMALLINT,
+    csmallint3 SMALLINT,
+    cint INT);
+
+insert into table allsmallints values (5, 1, 10, 2);
+select width_bucket(csmallint1, csmallint2, csmallint3, cint) from allsmallints;
+
+-- Test all ints
+create table allints (
+    cint1 INT,
+    cint2 INT,
+    cint3 INT,
+    cint4 INT);
+
+insert into table allints values (5, 1, 10, 2);
+select width_bucket(cint1, cint2, cint3, cint4) from allints;
+
+-- Test all bigints
+create table allbigints (
+    cbigint1 BIGINT,
+    cbigint2 BIGINT,
+    cbigint3 BIGINT,
+    cint INT);
+
+insert into table allbigints values (5, 1, 10, 2);
+select width_bucket(cbigint1, cbigint2, cbigint3, cint) from allbigints;
+
+-- Test all floats
+create table allfloats (
+    cfloat1 FLOAT,
+    cfloat2 FLOAT,
+    cfloat3 FLOAT,
+    cint INT);
+
+insert into table allfloats values (5.0, 1.0, 10.0, 2);
+select width_bucket(cfloat1, cfloat2, cfloat3, cint) from allfloats;
+
+-- Test all doubles
+create table alldoubles (
+    cdouble1 DOUBLE,
+    cdouble2 DOUBLE,
+    cdouble3 DOUBLE,
+    cint INT);
+
+insert into table alldoubles values (5.0, 1.0, 10.0, 2);
+select width_bucket(cdouble1, cdouble2, cdouble3, cint) from alldoubles;
+
+-- Test with grouping sets
+create table testgroupingsets (c1 int, c2 int);
+insert into table testgroupingsets values (1, 1), (2, 2);
+select c1, c2, width_bucket(5, c1, 10, case when grouping(c2) = 0 then 10 else 5 end) from testgroupingsets group by cube(c1, c2);
+
+drop table alldecimaltype;
+drop table alltypes;
+drop table alltinyints;
+drop table allsmallints;
+drop table allints;
+drop table allbigints;
+drop table allfloats;
+drop table alldoubles;
+drop table testgroupingsets;

http://git-wip-us.apache.org/repos/asf/hive/blob/f8f9155d/ql/src/test/results/clientpositive/udf_width_bucket.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/udf_width_bucket.q.out b/ql/src/test/results/clientpositive/udf_width_bucket.q.out
index a72e977..6879631 100644
--- a/ql/src/test/results/clientpositive/udf_width_bucket.q.out
+++ b/ql/src/test/results/clientpositive/udf_width_bucket.q.out
@@ -109,3 +109,572 @@ POSTHOOK: type: QUERY
 POSTHOOK: Input: _dummy_database@_dummy_table
 #### A masked pattern was here ####
 0	2	4	5
+PREHOOK: query: select
+width_bucket(0.1, 0, 1, 10),
+width_bucket(0.25, 0, 1, 10),
+width_bucket(0.3456, 0, 1, 10),
+width_bucket(0.654321, 0, 1, 10)
+PREHOOK: type: QUERY
+PREHOOK: Input: _dummy_database@_dummy_table
+#### A masked pattern was here ####
+POSTHOOK: query: select
+width_bucket(0.1, 0, 1, 10),
+width_bucket(0.25, 0, 1, 10),
+width_bucket(0.3456, 0, 1, 10),
+width_bucket(0.654321, 0, 1, 10)
+POSTHOOK: type: QUERY
+POSTHOOK: Input: _dummy_database@_dummy_table
+#### A masked pattern was here ####
+2	3	4	7
+PREHOOK: query: select
+width_bucket(-0.5, -1.5, 1.5, 10),
+width_bucket(-0.3, -1.5, 1.5, 10),
+width_bucket(-0.25, -1.5, 1.5, 10),
+width_bucket(0, -1.5, 1.5, 10),
+width_bucket(0.75, -1.5, 1.5, 10),
+width_bucket(1.25, -1.5, 1.5, 10),
+width_bucket(1.5, -1.5, 1.5, 10)
+PREHOOK: type: QUERY
+PREHOOK: Input: _dummy_database@_dummy_table
+#### A masked pattern was here ####
+POSTHOOK: query: select
+width_bucket(-0.5, -1.5, 1.5, 10),
+width_bucket(-0.3, -1.5, 1.5, 10),
+width_bucket(-0.25, -1.5, 1.5, 10),
+width_bucket(0, -1.5, 1.5, 10),
+width_bucket(0.75, -1.5, 1.5, 10),
+width_bucket(1.25, -1.5, 1.5, 10),
+width_bucket(1.5, -1.5, 1.5, 10)
+POSTHOOK: type: QUERY
+POSTHOOK: Input: _dummy_database@_dummy_table
+#### A masked pattern was here ####
+4	5	5	6	8	10	11
+PREHOOK: query: select
+width_bucket(1, 25, 5, 4),
+width_bucket(10, 25, 5, 4),
+width_bucket(20, 25, 5, 4),
+width_bucket(30, 25, 5, 4)
+PREHOOK: type: QUERY
+PREHOOK: Input: _dummy_database@_dummy_table
+#### A masked pattern was here ####
+POSTHOOK: query: select
+width_bucket(1, 25, 5, 4),
+width_bucket(10, 25, 5, 4),
+width_bucket(20, 25, 5, 4),
+width_bucket(30, 25, 5, 4)
+POSTHOOK: type: QUERY
+POSTHOOK: Input: _dummy_database@_dummy_table
+#### A masked pattern was here ####
+5	4	2	0
+PREHOOK: query: select
+width_bucket(-10, 15, -5, 4),
+width_bucket(0, 15, -5, 4),
+width_bucket(10, 15, -5, 4),
+width_bucket(20, 15, -5, 4)
+PREHOOK: type: QUERY
+PREHOOK: Input: _dummy_database@_dummy_table
+#### A masked pattern was here ####
+POSTHOOK: query: select
+width_bucket(-10, 15, -5, 4),
+width_bucket(0, 15, -5, 4),
+width_bucket(10, 15, -5, 4),
+width_bucket(20, 15, -5, 4)
+POSTHOOK: type: QUERY
+POSTHOOK: Input: _dummy_database@_dummy_table
+#### A masked pattern was here ####
+5	4	2	0
+PREHOOK: query: select
+width_bucket(0.1, 1, 0, 10),
+width_bucket(0.25, 1, 0, 10),
+width_bucket(0.3456, 1, 0, 10),
+width_bucket(0.654321, 1, 0, 10)
+PREHOOK: type: QUERY
+PREHOOK: Input: _dummy_database@_dummy_table
+#### A masked pattern was here ####
+POSTHOOK: query: select
+width_bucket(0.1, 1, 0, 10),
+width_bucket(0.25, 1, 0, 10),
+width_bucket(0.3456, 1, 0, 10),
+width_bucket(0.654321, 1, 0, 10)
+POSTHOOK: type: QUERY
+POSTHOOK: Input: _dummy_database@_dummy_table
+#### A masked pattern was here ####
+10	8	7	4
+PREHOOK: query: create table alldecimaltypes(
+    cfloat FLOAT,
+    cdouble DOUBLE)
+PREHOOK: type: CREATETABLE
+PREHOOK: Output: database:default
+PREHOOK: Output: default@alldecimaltypes
+POSTHOOK: query: create table alldecimaltypes(
+    cfloat FLOAT,
+    cdouble DOUBLE)
+POSTHOOK: type: CREATETABLE
+POSTHOOK: Output: database:default
+POSTHOOK: Output: default@alldecimaltypes
+PREHOOK: query: insert into table alldecimaltypes values (0.1, 0.1), (0.25, 0.25), (0.3456, 0.3456), (0.654321, 0.654321)
+PREHOOK: type: QUERY
+PREHOOK: Output: default@alldecimaltypes
+POSTHOOK: query: insert into table alldecimaltypes values (0.1, 0.1), (0.25, 0.25), (0.3456, 0.3456), (0.654321, 0.654321)
+POSTHOOK: type: QUERY
+POSTHOOK: Output: default@alldecimaltypes
+POSTHOOK: Lineage: alldecimaltypes.cdouble EXPRESSION [(values__tmp__table__1)values__tmp__table__1.FieldSchema(name:tmp_values_col2, type:string, comment:), ]
+POSTHOOK: Lineage: alldecimaltypes.cfloat EXPRESSION [(values__tmp__table__1)values__tmp__table__1.FieldSchema(name:tmp_values_col1, type:string, comment:), ]
+PREHOOK: query: select
+width_bucket(cfloat, 0, 1, 10),
+width_bucket(cdouble, 0, 1, 10)
+from alldecimaltypes
+PREHOOK: type: QUERY
+PREHOOK: Input: default@alldecimaltypes
+#### A masked pattern was here ####
+POSTHOOK: query: select
+width_bucket(cfloat, 0, 1, 10),
+width_bucket(cdouble, 0, 1, 10)
+from alldecimaltypes
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@alldecimaltypes
+#### A masked pattern was here ####
+2	2
+3	3
+4	4
+7	7
+PREHOOK: query: select
+width_bucket(cfloat, 0, 1.5, 10),
+width_bucket(cdouble, -1.5, 0, 10),
+width_bucket(0.25, cfloat, 2, 10),
+width_bucket(0.25, 0, cdouble, 10)
+from alldecimaltypes
+PREHOOK: type: QUERY
+PREHOOK: Input: default@alldecimaltypes
+#### A masked pattern was here ####
+POSTHOOK: query: select
+width_bucket(cfloat, 0, 1.5, 10),
+width_bucket(cdouble, -1.5, 0, 10),
+width_bucket(0.25, cfloat, 2, 10),
+width_bucket(0.25, 0, cdouble, 10)
+from alldecimaltypes
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@alldecimaltypes
+#### A masked pattern was here ####
+1	11	1	11
+2	11	1	11
+3	11	0	8
+5	11	0	4
+PREHOOK: query: create table alltypes(
+    ctinyint TINYINT,
+    csmallint SMALLINT,
+    cint INT,
+    cbigint BIGINT,
+    cfloat FLOAT,
+    cdouble DOUBLE)
+PREHOOK: type: CREATETABLE
+PREHOOK: Output: database:default
+PREHOOK: Output: default@alltypes
+POSTHOOK: query: create table alltypes(
+    ctinyint TINYINT,
+    csmallint SMALLINT,
+    cint INT,
+    cbigint BIGINT,
+    cfloat FLOAT,
+    cdouble DOUBLE)
+POSTHOOK: type: CREATETABLE
+POSTHOOK: Output: database:default
+POSTHOOK: Output: default@alltypes
+PREHOOK: query: insert into table alltypes values
+(0, 0, 0, 0, 0.0, 0.0),
+(1, 1, 1, 1, 1.0, 1.0),
+(25, 25, 25, 25, 25.0, 25.0),
+(60, 60, 60, 60, 60.0, 60.0),
+(72, 72, 72, 72, 72.0, 72.0),
+(100, 100, 100, 100, 100.0, 100.0)
+PREHOOK: type: QUERY
+PREHOOK: Output: default@alltypes
+POSTHOOK: query: insert into table alltypes values
+(0, 0, 0, 0, 0.0, 0.0),
+(1, 1, 1, 1, 1.0, 1.0),
+(25, 25, 25, 25, 25.0, 25.0),
+(60, 60, 60, 60, 60.0, 60.0),
+(72, 72, 72, 72, 72.0, 72.0),
+(100, 100, 100, 100, 100.0, 100.0)
+POSTHOOK: type: QUERY
+POSTHOOK: Output: default@alltypes
+POSTHOOK: Lineage: alltypes.cbigint EXPRESSION [(values__tmp__table__2)values__tmp__table__2.FieldSchema(name:tmp_values_col4, type:string, comment:), ]
+POSTHOOK: Lineage: alltypes.cdouble EXPRESSION [(values__tmp__table__2)values__tmp__table__2.FieldSchema(name:tmp_values_col6, type:string, comment:), ]
+POSTHOOK: Lineage: alltypes.cfloat EXPRESSION [(values__tmp__table__2)values__tmp__table__2.FieldSchema(name:tmp_values_col5, type:string, comment:), ]
+POSTHOOK: Lineage: alltypes.cint EXPRESSION [(values__tmp__table__2)values__tmp__table__2.FieldSchema(name:tmp_values_col3, type:string, comment:), ]
+POSTHOOK: Lineage: alltypes.csmallint EXPRESSION [(values__tmp__table__2)values__tmp__table__2.FieldSchema(name:tmp_values_col2, type:string, comment:), ]
+POSTHOOK: Lineage: alltypes.ctinyint EXPRESSION [(values__tmp__table__2)values__tmp__table__2.FieldSchema(name:tmp_values_col1, type:string, comment:), ]
+PREHOOK: query: select
+width_bucket(ctinyint, 0, 100, 10),
+width_bucket(csmallint, 0, 100, 10),
+width_bucket(cint, 0, 100, 10),
+width_bucket(cbigint, 0, 100, 10),
+width_bucket(cfloat, 0, 100, 10),
+width_bucket(cdouble, 0, 100, 10)
+from alltypes
+PREHOOK: type: QUERY
+PREHOOK: Input: default@alltypes
+#### A masked pattern was here ####
+POSTHOOK: query: select
+width_bucket(ctinyint, 0, 100, 10),
+width_bucket(csmallint, 0, 100, 10),
+width_bucket(cint, 0, 100, 10),
+width_bucket(cbigint, 0, 100, 10),
+width_bucket(cfloat, 0, 100, 10),
+width_bucket(cdouble, 0, 100, 10)
+from alltypes
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@alltypes
+#### A masked pattern was here ####
+1	1	1	1	1	1
+1	1	1	1	1	1
+3	3	3	3	3	3
+7	7	7	7	7	7
+8	8	8	8	8	8
+11	11	11	11	11	11
+PREHOOK: query: truncate table alltypes
+PREHOOK: type: TRUNCATETABLE
+PREHOOK: Output: default@alltypes
+POSTHOOK: query: truncate table alltypes
+POSTHOOK: type: TRUNCATETABLE
+POSTHOOK: Output: default@alltypes
+PREHOOK: query: insert into table alltypes values (5, 5, 5, 10, 4.5, 7.25)
+PREHOOK: type: QUERY
+PREHOOK: Output: default@alltypes
+POSTHOOK: query: insert into table alltypes values (5, 5, 5, 10, 4.5, 7.25)
+POSTHOOK: type: QUERY
+POSTHOOK: Output: default@alltypes
+POSTHOOK: Lineage: alltypes.cbigint EXPRESSION [(values__tmp__table__3)values__tmp__table__3.FieldSchema(name:tmp_values_col4, type:string, comment:), ]
+POSTHOOK: Lineage: alltypes.cdouble EXPRESSION [(values__tmp__table__3)values__tmp__table__3.FieldSchema(name:tmp_values_col6, type:string, comment:), ]
+POSTHOOK: Lineage: alltypes.cfloat EXPRESSION [(values__tmp__table__3)values__tmp__table__3.FieldSchema(name:tmp_values_col5, type:string, comment:), ]
+POSTHOOK: Lineage: alltypes.cint EXPRESSION [(values__tmp__table__3)values__tmp__table__3.FieldSchema(name:tmp_values_col3, type:string, comment:), ]
+POSTHOOK: Lineage: alltypes.csmallint EXPRESSION [(values__tmp__table__3)values__tmp__table__3.FieldSchema(name:tmp_values_col2, type:string, comment:), ]
+POSTHOOK: Lineage: alltypes.ctinyint EXPRESSION [(values__tmp__table__3)values__tmp__table__3.FieldSchema(name:tmp_values_col1, type:string, comment:), ]
+PREHOOK: query: select
+width_bucket(cdouble, ctinyint, cbigint, 10),
+width_bucket(cdouble, csmallint, cbigint, 10),
+width_bucket(cdouble, cint, cbigint, 10),
+width_bucket(cdouble, cfloat, cbigint, 10)
+from alltypes
+PREHOOK: type: QUERY
+PREHOOK: Input: default@alltypes
+#### A masked pattern was here ####
+POSTHOOK: query: select
+width_bucket(cdouble, ctinyint, cbigint, 10),
+width_bucket(cdouble, csmallint, cbigint, 10),
+width_bucket(cdouble, cint, cbigint, 10),
+width_bucket(cdouble, cfloat, cbigint, 10)
+from alltypes
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@alltypes
+#### A masked pattern was here ####
+5	5	5	6
+PREHOOK: query: create table alltinyints (
+    ctinyint1 TINYINT,
+    ctinyint2 TINYINT,
+    ctinyint3 TINYINT,
+    cint INT)
+PREHOOK: type: CREATETABLE
+PREHOOK: Output: database:default
+PREHOOK: Output: default@alltinyints
+POSTHOOK: query: create table alltinyints (
+    ctinyint1 TINYINT,
+    ctinyint2 TINYINT,
+    ctinyint3 TINYINT,
+    cint INT)
+POSTHOOK: type: CREATETABLE
+POSTHOOK: Output: database:default
+POSTHOOK: Output: default@alltinyints
+PREHOOK: query: insert into table alltinyints values (5, 1, 10, 2)
+PREHOOK: type: QUERY
+PREHOOK: Output: default@alltinyints
+POSTHOOK: query: insert into table alltinyints values (5, 1, 10, 2)
+POSTHOOK: type: QUERY
+POSTHOOK: Output: default@alltinyints
+POSTHOOK: Lineage: alltinyints.cint EXPRESSION [(values__tmp__table__4)values__tmp__table__4.FieldSchema(name:tmp_values_col4, type:string, comment:), ]
+POSTHOOK: Lineage: alltinyints.ctinyint1 EXPRESSION [(values__tmp__table__4)values__tmp__table__4.FieldSchema(name:tmp_values_col1, type:string, comment:), ]
+POSTHOOK: Lineage: alltinyints.ctinyint2 EXPRESSION [(values__tmp__table__4)values__tmp__table__4.FieldSchema(name:tmp_values_col2, type:string, comment:), ]
+POSTHOOK: Lineage: alltinyints.ctinyint3 EXPRESSION [(values__tmp__table__4)values__tmp__table__4.FieldSchema(name:tmp_values_col3, type:string, comment:), ]
+PREHOOK: query: select width_bucket(ctinyint1, ctinyint2, ctinyint3, cint) from alltinyints
+PREHOOK: type: QUERY
+PREHOOK: Input: default@alltinyints
+#### A masked pattern was here ####
+POSTHOOK: query: select width_bucket(ctinyint1, ctinyint2, ctinyint3, cint) from alltinyints
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@alltinyints
+#### A masked pattern was here ####
+1
+PREHOOK: query: create table allsmallints (
+    csmallint1 SMALLINT,
+    csmallint2 SMALLINT,
+    csmallint3 SMALLINT,
+    cint INT)
+PREHOOK: type: CREATETABLE
+PREHOOK: Output: database:default
+PREHOOK: Output: default@allsmallints
+POSTHOOK: query: create table allsmallints (
+    csmallint1 SMALLINT,
+    csmallint2 SMALLINT,
+    csmallint3 SMALLINT,
+    cint INT)
+POSTHOOK: type: CREATETABLE
+POSTHOOK: Output: database:default
+POSTHOOK: Output: default@allsmallints
+PREHOOK: query: insert into table allsmallints values (5, 1, 10, 2)
+PREHOOK: type: QUERY
+PREHOOK: Output: default@allsmallints
+POSTHOOK: query: insert into table allsmallints values (5, 1, 10, 2)
+POSTHOOK: type: QUERY
+POSTHOOK: Output: default@allsmallints
+POSTHOOK: Lineage: allsmallints.cint EXPRESSION [(values__tmp__table__5)values__tmp__table__5.FieldSchema(name:tmp_values_col4, type:string, comment:), ]
+POSTHOOK: Lineage: allsmallints.csmallint1 EXPRESSION [(values__tmp__table__5)values__tmp__table__5.FieldSchema(name:tmp_values_col1, type:string, comment:), ]
+POSTHOOK: Lineage: allsmallints.csmallint2 EXPRESSION [(values__tmp__table__5)values__tmp__table__5.FieldSchema(name:tmp_values_col2, type:string, comment:), ]
+POSTHOOK: Lineage: allsmallints.csmallint3 EXPRESSION [(values__tmp__table__5)values__tmp__table__5.FieldSchema(name:tmp_values_col3, type:string, comment:), ]
+PREHOOK: query: select width_bucket(csmallint1, csmallint2, csmallint3, cint) from allsmallints
+PREHOOK: type: QUERY
+PREHOOK: Input: default@allsmallints
+#### A masked pattern was here ####
+POSTHOOK: query: select width_bucket(csmallint1, csmallint2, csmallint3, cint) from allsmallints
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@allsmallints
+#### A masked pattern was here ####
+1
+PREHOOK: query: create table allints (
+    cint1 INT,
+    cint2 INT,
+    cint3 INT,
+    cint4 INT)
+PREHOOK: type: CREATETABLE
+PREHOOK: Output: database:default
+PREHOOK: Output: default@allints
+POSTHOOK: query: create table allints (
+    cint1 INT,
+    cint2 INT,
+    cint3 INT,
+    cint4 INT)
+POSTHOOK: type: CREATETABLE
+POSTHOOK: Output: database:default
+POSTHOOK: Output: default@allints
+PREHOOK: query: insert into table allints values (5, 1, 10, 2)
+PREHOOK: type: QUERY
+PREHOOK: Output: default@allints
+POSTHOOK: query: insert into table allints values (5, 1, 10, 2)
+POSTHOOK: type: QUERY
+POSTHOOK: Output: default@allints
+POSTHOOK: Lineage: allints.cint1 EXPRESSION [(values__tmp__table__6)values__tmp__table__6.FieldSchema(name:tmp_values_col1, type:string, comment:), ]
+POSTHOOK: Lineage: allints.cint2 EXPRESSION [(values__tmp__table__6)values__tmp__table__6.FieldSchema(name:tmp_values_col2, type:string, comment:), ]
+POSTHOOK: Lineage: allints.cint3 EXPRESSION [(values__tmp__table__6)values__tmp__table__6.FieldSchema(name:tmp_values_col3, type:string, comment:), ]
+POSTHOOK: Lineage: allints.cint4 EXPRESSION [(values__tmp__table__6)values__tmp__table__6.FieldSchema(name:tmp_values_col4, type:string, comment:), ]
+PREHOOK: query: select width_bucket(cint1, cint2, cint3, cint4) from allints
+PREHOOK: type: QUERY
+PREHOOK: Input: default@allints
+#### A masked pattern was here ####
+POSTHOOK: query: select width_bucket(cint1, cint2, cint3, cint4) from allints
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@allints
+#### A masked pattern was here ####
+1
+PREHOOK: query: create table allbigints (
+    cbigint1 BIGINT,
+    cbigint2 BIGINT,
+    cbigint3 BIGINT,
+    cint INT)
+PREHOOK: type: CREATETABLE
+PREHOOK: Output: database:default
+PREHOOK: Output: default@allbigints
+POSTHOOK: query: create table allbigints (
+    cbigint1 BIGINT,
+    cbigint2 BIGINT,
+    cbigint3 BIGINT,
+    cint INT)
+POSTHOOK: type: CREATETABLE
+POSTHOOK: Output: database:default
+POSTHOOK: Output: default@allbigints
+PREHOOK: query: insert into table allbigints values (5, 1, 10, 2)
+PREHOOK: type: QUERY
+PREHOOK: Output: default@allbigints
+POSTHOOK: query: insert into table allbigints values (5, 1, 10, 2)
+POSTHOOK: type: QUERY
+POSTHOOK: Output: default@allbigints
+POSTHOOK: Lineage: allbigints.cbigint1 EXPRESSION [(values__tmp__table__7)values__tmp__table__7.FieldSchema(name:tmp_values_col1, type:string, comment:), ]
+POSTHOOK: Lineage: allbigints.cbigint2 EXPRESSION [(values__tmp__table__7)values__tmp__table__7.FieldSchema(name:tmp_values_col2, type:string, comment:), ]
+POSTHOOK: Lineage: allbigints.cbigint3 EXPRESSION [(values__tmp__table__7)values__tmp__table__7.FieldSchema(name:tmp_values_col3, type:string, comment:), ]
+POSTHOOK: Lineage: allbigints.cint EXPRESSION [(values__tmp__table__7)values__tmp__table__7.FieldSchema(name:tmp_values_col4, type:string, comment:), ]
+PREHOOK: query: select width_bucket(cbigint1, cbigint2, cbigint3, cint) from allbigints
+PREHOOK: type: QUERY
+PREHOOK: Input: default@allbigints
+#### A masked pattern was here ####
+POSTHOOK: query: select width_bucket(cbigint1, cbigint2, cbigint3, cint) from allbigints
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@allbigints
+#### A masked pattern was here ####
+1
+PREHOOK: query: create table allfloats (
+    cfloat1 FLOAT,
+    cfloat2 FLOAT,
+    cfloat3 FLOAT,
+    cint INT)
+PREHOOK: type: CREATETABLE
+PREHOOK: Output: database:default
+PREHOOK: Output: default@allfloats
+POSTHOOK: query: create table allfloats (
+    cfloat1 FLOAT,
+    cfloat2 FLOAT,
+    cfloat3 FLOAT,
+    cint INT)
+POSTHOOK: type: CREATETABLE
+POSTHOOK: Output: database:default
+POSTHOOK: Output: default@allfloats
+PREHOOK: query: insert into table allfloats values (5.0, 1.0, 10.0, 2)
+PREHOOK: type: QUERY
+PREHOOK: Output: default@allfloats
+POSTHOOK: query: insert into table allfloats values (5.0, 1.0, 10.0, 2)
+POSTHOOK: type: QUERY
+POSTHOOK: Output: default@allfloats
+POSTHOOK: Lineage: allfloats.cfloat1 EXPRESSION [(values__tmp__table__8)values__tmp__table__8.FieldSchema(name:tmp_values_col1, type:string, comment:), ]
+POSTHOOK: Lineage: allfloats.cfloat2 EXPRESSION [(values__tmp__table__8)values__tmp__table__8.FieldSchema(name:tmp_values_col2, type:string, comment:), ]
+POSTHOOK: Lineage: allfloats.cfloat3 EXPRESSION [(values__tmp__table__8)values__tmp__table__8.FieldSchema(name:tmp_values_col3, type:string, comment:), ]
+POSTHOOK: Lineage: allfloats.cint EXPRESSION [(values__tmp__table__8)values__tmp__table__8.FieldSchema(name:tmp_values_col4, type:string, comment:), ]
+PREHOOK: query: select width_bucket(cfloat1, cfloat2, cfloat3, cint) from allfloats
+PREHOOK: type: QUERY
+PREHOOK: Input: default@allfloats
+#### A masked pattern was here ####
+POSTHOOK: query: select width_bucket(cfloat1, cfloat2, cfloat3, cint) from allfloats
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@allfloats
+#### A masked pattern was here ####
+1
+PREHOOK: query: create table alldoubles (
+    cdouble1 DOUBLE,
+    cdouble2 DOUBLE,
+    cdouble3 DOUBLE,
+    cint INT)
+PREHOOK: type: CREATETABLE
+PREHOOK: Output: database:default
+PREHOOK: Output: default@alldoubles
+POSTHOOK: query: create table alldoubles (
+    cdouble1 DOUBLE,
+    cdouble2 DOUBLE,
+    cdouble3 DOUBLE,
+    cint INT)
+POSTHOOK: type: CREATETABLE
+POSTHOOK: Output: database:default
+POSTHOOK: Output: default@alldoubles
+PREHOOK: query: insert into table alldoubles values (5.0, 1.0, 10.0, 2)
+PREHOOK: type: QUERY
+PREHOOK: Output: default@alldoubles
+POSTHOOK: query: insert into table alldoubles values (5.0, 1.0, 10.0, 2)
+POSTHOOK: type: QUERY
+POSTHOOK: Output: default@alldoubles
+POSTHOOK: Lineage: alldoubles.cdouble1 EXPRESSION [(values__tmp__table__9)values__tmp__table__9.FieldSchema(name:tmp_values_col1, type:string, comment:), ]
+POSTHOOK: Lineage: alldoubles.cdouble2 EXPRESSION [(values__tmp__table__9)values__tmp__table__9.FieldSchema(name:tmp_values_col2, type:string, comment:), ]
+POSTHOOK: Lineage: alldoubles.cdouble3 EXPRESSION [(values__tmp__table__9)values__tmp__table__9.FieldSchema(name:tmp_values_col3, type:string, comment:), ]
+POSTHOOK: Lineage: alldoubles.cint EXPRESSION [(values__tmp__table__9)values__tmp__table__9.FieldSchema(name:tmp_values_col4, type:string, comment:), ]
+PREHOOK: query: select width_bucket(cdouble1, cdouble2, cdouble3, cint) from alldoubles
+PREHOOK: type: QUERY
+PREHOOK: Input: default@alldoubles
+#### A masked pattern was here ####
+POSTHOOK: query: select width_bucket(cdouble1, cdouble2, cdouble3, cint) from alldoubles
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@alldoubles
+#### A masked pattern was here ####
+1
+PREHOOK: query: create table testgroupingsets (c1 int, c2 int)
+PREHOOK: type: CREATETABLE
+PREHOOK: Output: database:default
+PREHOOK: Output: default@testgroupingsets
+POSTHOOK: query: create table testgroupingsets (c1 int, c2 int)
+POSTHOOK: type: CREATETABLE
+POSTHOOK: Output: database:default
+POSTHOOK: Output: default@testgroupingsets
+PREHOOK: query: insert into table testgroupingsets values (1, 1), (2, 2)
+PREHOOK: type: QUERY
+PREHOOK: Output: default@testgroupingsets
+POSTHOOK: query: insert into table testgroupingsets values (1, 1), (2, 2)
+POSTHOOK: type: QUERY
+POSTHOOK: Output: default@testgroupingsets
+POSTHOOK: Lineage: testgroupingsets.c1 EXPRESSION [(values__tmp__table__10)values__tmp__table__10.FieldSchema(name:tmp_values_col1, type:string, comment:), ]
+POSTHOOK: Lineage: testgroupingsets.c2 EXPRESSION [(values__tmp__table__10)values__tmp__table__10.FieldSchema(name:tmp_values_col2, type:string, comment:), ]
+PREHOOK: query: select c1, c2, width_bucket(5, c1, 10, case when grouping(c2) = 0 then 10 else 5 end) from testgroupingsets group by cube(c1, c2)
+PREHOOK: type: QUERY
+PREHOOK: Input: default@testgroupingsets
+#### A masked pattern was here ####
+POSTHOOK: query: select c1, c2, width_bucket(5, c1, 10, case when grouping(c2) = 0 then 10 else 5 end) from testgroupingsets group by cube(c1, c2)
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@testgroupingsets
+#### A masked pattern was here ####
+NULL	NULL	NULL
+NULL	1	NULL
+NULL	2	NULL
+1	NULL	3
+1	1	5
+2	NULL	2
+2	2	4
+PREHOOK: query: drop table alldecimaltype
+PREHOOK: type: DROPTABLE
+POSTHOOK: query: drop table alldecimaltype
+POSTHOOK: type: DROPTABLE
+PREHOOK: query: drop table alltypes
+PREHOOK: type: DROPTABLE
+PREHOOK: Input: default@alltypes
+PREHOOK: Output: default@alltypes
+POSTHOOK: query: drop table alltypes
+POSTHOOK: type: DROPTABLE
+POSTHOOK: Input: default@alltypes
+POSTHOOK: Output: default@alltypes
+PREHOOK: query: drop table alltinyints
+PREHOOK: type: DROPTABLE
+PREHOOK: Input: default@alltinyints
+PREHOOK: Output: default@alltinyints
+POSTHOOK: query: drop table alltinyints
+POSTHOOK: type: DROPTABLE
+POSTHOOK: Input: default@alltinyints
+POSTHOOK: Output: default@alltinyints
+PREHOOK: query: drop table allsmallints
+PREHOOK: type: DROPTABLE
+PREHOOK: Input: default@allsmallints
+PREHOOK: Output: default@allsmallints
+POSTHOOK: query: drop table allsmallints
+POSTHOOK: type: DROPTABLE
+POSTHOOK: Input: default@allsmallints
+POSTHOOK: Output: default@allsmallints
+PREHOOK: query: drop table allints
+PREHOOK: type: DROPTABLE
+PREHOOK: Input: default@allints
+PREHOOK: Output: default@allints
+POSTHOOK: query: drop table allints
+POSTHOOK: type: DROPTABLE
+POSTHOOK: Input: default@allints
+POSTHOOK: Output: default@allints
+PREHOOK: query: drop table allbigints
+PREHOOK: type: DROPTABLE
+PREHOOK: Input: default@allbigints
+PREHOOK: Output: default@allbigints
+POSTHOOK: query: drop table allbigints
+POSTHOOK: type: DROPTABLE
+POSTHOOK: Input: default@allbigints
+POSTHOOK: Output: default@allbigints
+PREHOOK: query: drop table allfloats
+PREHOOK: type: DROPTABLE
+PREHOOK: Input: default@allfloats
+PREHOOK: Output: default@allfloats
+POSTHOOK: query: drop table allfloats
+POSTHOOK: type: DROPTABLE
+POSTHOOK: Input: default@allfloats
+POSTHOOK: Output: default@allfloats
+PREHOOK: query: drop table alldoubles
+PREHOOK: type: DROPTABLE
+PREHOOK: Input: default@alldoubles
+PREHOOK: Output: default@alldoubles
+POSTHOOK: query: drop table alldoubles
+POSTHOOK: type: DROPTABLE
+POSTHOOK: Input: default@alldoubles
+POSTHOOK: Output: default@alldoubles
+PREHOOK: query: drop table testgroupingsets
+PREHOOK: type: DROPTABLE
+PREHOOK: Input: default@testgroupingsets
+PREHOOK: Output: default@testgroupingsets
+POSTHOOK: query: drop table testgroupingsets
+POSTHOOK: type: DROPTABLE
+POSTHOOK: Input: default@testgroupingsets
+POSTHOOK: Output: default@testgroupingsets