You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hive.apache.org by gu...@apache.org on 2014/11/01 22:10:34 UTC

svn commit: r1636055 [1/12] - in /hive/branches/branch-0.14: common/src/java/org/apache/hadoop/hive/common/type/ itests/src/test/resources/ ql/src/gen/vectorization/ExpressionTemplates/ ql/src/gen/vectorization/UDAFTemplates/ ql/src/java/org/apache/had...

Author: gunther
Date: Sat Nov  1 21:10:32 2014
New Revision: 1636055

URL: http://svn.apache.org/r1636055
Log:
HIVE-8461: Make Vectorized Decimal query results match Non-Vectorized query results with respect to trailing zeroes... .0000 (Matt McCline reviewed by Ashutosh Chauhan)

Added:
    hive/branches/branch-0.14/ql/src/test/queries/clientpositive/vector_aggregate_9.q
    hive/branches/branch-0.14/ql/src/test/queries/clientpositive/vector_decimal_1.q
    hive/branches/branch-0.14/ql/src/test/queries/clientpositive/vector_decimal_10_0.q
    hive/branches/branch-0.14/ql/src/test/queries/clientpositive/vector_decimal_2.q
    hive/branches/branch-0.14/ql/src/test/queries/clientpositive/vector_decimal_3.q
    hive/branches/branch-0.14/ql/src/test/queries/clientpositive/vector_decimal_4.q
    hive/branches/branch-0.14/ql/src/test/queries/clientpositive/vector_decimal_5.q
    hive/branches/branch-0.14/ql/src/test/queries/clientpositive/vector_decimal_6.q
    hive/branches/branch-0.14/ql/src/test/queries/clientpositive/vector_decimal_precision.q
    hive/branches/branch-0.14/ql/src/test/queries/clientpositive/vector_decimal_trailing.q
    hive/branches/branch-0.14/ql/src/test/queries/clientpositive/vector_decimal_udf.q
    hive/branches/branch-0.14/ql/src/test/queries/clientpositive/vector_decimal_udf2.q
    hive/branches/branch-0.14/ql/src/test/results/clientpositive/tez/acid_vectorization_partition.q.out
    hive/branches/branch-0.14/ql/src/test/results/clientpositive/tez/acid_vectorization_project.q.out
    hive/branches/branch-0.14/ql/src/test/results/clientpositive/tez/vector_aggregate_9.q.out
    hive/branches/branch-0.14/ql/src/test/results/clientpositive/tez/vector_coalesce.q.out
    hive/branches/branch-0.14/ql/src/test/results/clientpositive/tez/vector_decimal_1.q.out
    hive/branches/branch-0.14/ql/src/test/results/clientpositive/tez/vector_decimal_10_0.q.out
    hive/branches/branch-0.14/ql/src/test/results/clientpositive/tez/vector_decimal_2.q.out
    hive/branches/branch-0.14/ql/src/test/results/clientpositive/tez/vector_decimal_3.q.out
    hive/branches/branch-0.14/ql/src/test/results/clientpositive/tez/vector_decimal_4.q.out
    hive/branches/branch-0.14/ql/src/test/results/clientpositive/tez/vector_decimal_5.q.out
    hive/branches/branch-0.14/ql/src/test/results/clientpositive/tez/vector_decimal_6.q.out
    hive/branches/branch-0.14/ql/src/test/results/clientpositive/tez/vector_decimal_cast.q.out
    hive/branches/branch-0.14/ql/src/test/results/clientpositive/tez/vector_decimal_expressions.q.out
    hive/branches/branch-0.14/ql/src/test/results/clientpositive/tez/vector_decimal_mapjoin.q.out
    hive/branches/branch-0.14/ql/src/test/results/clientpositive/tez/vector_decimal_math_funcs.q.out
    hive/branches/branch-0.14/ql/src/test/results/clientpositive/tez/vector_decimal_precision.q.out
    hive/branches/branch-0.14/ql/src/test/results/clientpositive/tez/vector_decimal_trailing.q.out
    hive/branches/branch-0.14/ql/src/test/results/clientpositive/tez/vector_decimal_udf.q.out
    hive/branches/branch-0.14/ql/src/test/results/clientpositive/tez/vector_decimal_udf2.q.out
    hive/branches/branch-0.14/ql/src/test/results/clientpositive/tez/vectorization_limit.q.out
    hive/branches/branch-0.14/ql/src/test/results/clientpositive/tez/vectorized_casts.q.out
    hive/branches/branch-0.14/ql/src/test/results/clientpositive/tez/vectorized_date_funcs.q.out
    hive/branches/branch-0.14/ql/src/test/results/clientpositive/tez/vectorized_distinct_gby.q.out
    hive/branches/branch-0.14/ql/src/test/results/clientpositive/vector_aggregate_9.q.out
    hive/branches/branch-0.14/ql/src/test/results/clientpositive/vector_decimal_1.q.out
    hive/branches/branch-0.14/ql/src/test/results/clientpositive/vector_decimal_10_0.q.out
    hive/branches/branch-0.14/ql/src/test/results/clientpositive/vector_decimal_2.q.out
    hive/branches/branch-0.14/ql/src/test/results/clientpositive/vector_decimal_3.q.out
    hive/branches/branch-0.14/ql/src/test/results/clientpositive/vector_decimal_4.q.out
    hive/branches/branch-0.14/ql/src/test/results/clientpositive/vector_decimal_5.q.out
    hive/branches/branch-0.14/ql/src/test/results/clientpositive/vector_decimal_6.q.out
    hive/branches/branch-0.14/ql/src/test/results/clientpositive/vector_decimal_precision.q.out
    hive/branches/branch-0.14/ql/src/test/results/clientpositive/vector_decimal_trailing.q.out
    hive/branches/branch-0.14/ql/src/test/results/clientpositive/vector_decimal_udf.q.out
    hive/branches/branch-0.14/ql/src/test/results/clientpositive/vector_decimal_udf2.q.out
Modified:
    hive/branches/branch-0.14/common/src/java/org/apache/hadoop/hive/common/type/HiveDecimal.java
    hive/branches/branch-0.14/itests/src/test/resources/testconfiguration.properties
    hive/branches/branch-0.14/ql/src/gen/vectorization/ExpressionTemplates/ColumnArithmeticColumnDecimal.txt
    hive/branches/branch-0.14/ql/src/gen/vectorization/ExpressionTemplates/ColumnArithmeticScalarDecimal.txt
    hive/branches/branch-0.14/ql/src/gen/vectorization/ExpressionTemplates/ColumnDivideColumnDecimal.txt
    hive/branches/branch-0.14/ql/src/gen/vectorization/ExpressionTemplates/ColumnDivideScalarDecimal.txt
    hive/branches/branch-0.14/ql/src/gen/vectorization/ExpressionTemplates/DecimalColumnUnaryFunc.txt
    hive/branches/branch-0.14/ql/src/gen/vectorization/ExpressionTemplates/FilterDecimalColumnBetween.txt
    hive/branches/branch-0.14/ql/src/gen/vectorization/ExpressionTemplates/FilterDecimalColumnCompareColumn.txt
    hive/branches/branch-0.14/ql/src/gen/vectorization/ExpressionTemplates/FilterDecimalColumnCompareScalar.txt
    hive/branches/branch-0.14/ql/src/gen/vectorization/ExpressionTemplates/FilterDecimalScalarCompareColumn.txt
    hive/branches/branch-0.14/ql/src/gen/vectorization/ExpressionTemplates/ScalarArithmeticColumnDecimal.txt
    hive/branches/branch-0.14/ql/src/gen/vectorization/ExpressionTemplates/ScalarDivideColumnDecimal.txt
    hive/branches/branch-0.14/ql/src/gen/vectorization/UDAFTemplates/VectorUDAFMinMaxDecimal.txt
    hive/branches/branch-0.14/ql/src/gen/vectorization/UDAFTemplates/VectorUDAFVarDecimal.txt
    hive/branches/branch-0.14/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/DecimalColumnVector.java
    hive/branches/branch-0.14/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/VectorColumnAssignFactory.java
    hive/branches/branch-0.14/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/VectorGroupKeyHelper.java
    hive/branches/branch-0.14/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/VectorHashKeyWrapper.java
    hive/branches/branch-0.14/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/VectorHashKeyWrapperBatch.java
    hive/branches/branch-0.14/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/VectorizationContext.java
    hive/branches/branch-0.14/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/VectorizedBatchUtil.java
    hive/branches/branch-0.14/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/VectorizedRowBatchCtx.java
    hive/branches/branch-0.14/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/CastDecimalToBoolean.java
    hive/branches/branch-0.14/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/CastDecimalToDecimal.java
    hive/branches/branch-0.14/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/CastDecimalToDouble.java
    hive/branches/branch-0.14/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/CastDecimalToLong.java
    hive/branches/branch-0.14/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/CastDecimalToString.java
    hive/branches/branch-0.14/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/CastDecimalToTimestamp.java
    hive/branches/branch-0.14/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/CastDoubleToDecimal.java
    hive/branches/branch-0.14/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/CastLongToDecimal.java
    hive/branches/branch-0.14/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/CastStringToDecimal.java
    hive/branches/branch-0.14/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/CastTimestampToDecimal.java
    hive/branches/branch-0.14/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/ConstantVectorExpression.java
    hive/branches/branch-0.14/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/DecimalColumnInList.java
    hive/branches/branch-0.14/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/DecimalUtil.java
    hive/branches/branch-0.14/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/FilterDecimalColumnInList.java
    hive/branches/branch-0.14/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/FuncDecimalToLong.java
    hive/branches/branch-0.14/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/FuncDoubleToDecimal.java
    hive/branches/branch-0.14/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/FuncLongToDecimal.java
    hive/branches/branch-0.14/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/FuncRoundWithNumDigitsDecimalToDecimal.java
    hive/branches/branch-0.14/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/IDecimalInExpr.java
    hive/branches/branch-0.14/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/NullUtil.java
    hive/branches/branch-0.14/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/VectorExpressionWriter.java
    hive/branches/branch-0.14/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/VectorExpressionWriterFactory.java
    hive/branches/branch-0.14/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/aggregates/VectorUDAFAvgDecimal.java
    hive/branches/branch-0.14/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/aggregates/VectorUDAFSumDecimal.java
    hive/branches/branch-0.14/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/udf/VectorUDFAdaptor.java
    hive/branches/branch-0.14/ql/src/java/org/apache/hadoop/hive/ql/io/orc/RecordReaderImpl.java
    hive/branches/branch-0.14/ql/src/test/org/apache/hadoop/hive/ql/exec/vector/TestVectorGroupByOperator.java
    hive/branches/branch-0.14/ql/src/test/org/apache/hadoop/hive/ql/exec/vector/expressions/TestConstantVectorExpression.java
    hive/branches/branch-0.14/ql/src/test/org/apache/hadoop/hive/ql/exec/vector/expressions/TestDecimalUtil.java
    hive/branches/branch-0.14/ql/src/test/org/apache/hadoop/hive/ql/exec/vector/expressions/TestVectorArithmeticExpressions.java
    hive/branches/branch-0.14/ql/src/test/org/apache/hadoop/hive/ql/exec/vector/expressions/TestVectorExpressionWriters.java
    hive/branches/branch-0.14/ql/src/test/org/apache/hadoop/hive/ql/exec/vector/expressions/TestVectorFilterExpressions.java
    hive/branches/branch-0.14/ql/src/test/org/apache/hadoop/hive/ql/exec/vector/expressions/TestVectorTypeCasts.java
    hive/branches/branch-0.14/ql/src/test/org/apache/hadoop/hive/ql/exec/vector/util/FakeVectorRowBatchFromObjectIterables.java
    hive/branches/branch-0.14/ql/src/test/org/apache/hadoop/hive/ql/exec/vector/util/VectorizedRowGroupGenUtil.java
    hive/branches/branch-0.14/ql/src/test/org/apache/hadoop/hive/ql/io/orc/TestInputOutputFormat.java
    hive/branches/branch-0.14/ql/src/test/results/clientpositive/tez/vector_between_in.q.out
    hive/branches/branch-0.14/ql/src/test/results/clientpositive/tez/vector_decimal_aggregate.q.out
    hive/branches/branch-0.14/ql/src/test/results/clientpositive/vector_between_in.q.out
    hive/branches/branch-0.14/ql/src/test/results/clientpositive/vector_decimal_aggregate.q.out
    hive/branches/branch-0.14/ql/src/test/results/clientpositive/vector_decimal_cast.q.out
    hive/branches/branch-0.14/ql/src/test/results/clientpositive/vector_decimal_expressions.q.out
    hive/branches/branch-0.14/ql/src/test/results/clientpositive/vector_decimal_math_funcs.q.out
    hive/branches/branch-0.14/serde/src/java/org/apache/hadoop/hive/serde2/io/HiveDecimalWritable.java

Modified: hive/branches/branch-0.14/common/src/java/org/apache/hadoop/hive/common/type/HiveDecimal.java
URL: http://svn.apache.org/viewvc/hive/branches/branch-0.14/common/src/java/org/apache/hadoop/hive/common/type/HiveDecimal.java?rev=1636055&r1=1636054&r2=1636055&view=diff
==============================================================================
--- hive/branches/branch-0.14/common/src/java/org/apache/hadoop/hive/common/type/HiveDecimal.java (original)
+++ hive/branches/branch-0.14/common/src/java/org/apache/hadoop/hive/common/type/HiveDecimal.java Sat Nov  1 21:10:32 2014
@@ -266,4 +266,17 @@ public class HiveDecimal implements Comp
 
     return bd;
   }
+
+  public static HiveDecimal enforcePrecisionScale(HiveDecimal dec, int maxPrecision, int maxScale) {
+    if (dec == null) {
+      return null;
+    }
+
+    BigDecimal bd = enforcePrecisionScale(dec.bd, maxPrecision, maxScale);
+    if (bd == null) {
+      return null;
+    }
+
+    return HiveDecimal.create(bd);
+  }
 }

Modified: hive/branches/branch-0.14/itests/src/test/resources/testconfiguration.properties
URL: http://svn.apache.org/viewvc/hive/branches/branch-0.14/itests/src/test/resources/testconfiguration.properties?rev=1636055&r1=1636054&r2=1636055&view=diff
==============================================================================
--- hive/branches/branch-0.14/itests/src/test/resources/testconfiguration.properties (original)
+++ hive/branches/branch-0.14/itests/src/test/resources/testconfiguration.properties Sat Nov  1 21:10:32 2014
@@ -161,9 +161,25 @@ minitez.query.files.shared=alter_merge_2
   vector_cast_constant.q,\
   vector_char_4.q,\
   vector_char_simple.q,\
+  vector_coalesce.q,\
   vector_count_distinct.q,\
   vector_data_types.q,\
+  vector_decimal_1.q,\
+  vector_decimal_10_0.q,\
+  vector_decimal_2.q,\
+  vector_decimal_3.q,\
+  vector_decimal_4.q,\
+  vector_decimal_5.q,\
+  vector_decimal_6.q,\
   vector_decimal_aggregate.q,\
+  vector_decimal_cast.q,\
+  vector_decimal_expressions.q,\
+  vector_decimal_mapjoin.q,\
+  vector_decimal_math_funcs.q,\
+  vector_decimal_precision.q,\
+  vector_decimal_trailing.q,\
+  vector_decimal_udf.q,\
+  vector_decimal_udf2.q,\
   vector_distinct_2.q,\
   vector_elt.q,\
   vector_groupby_3.q,\
@@ -196,6 +212,7 @@ minitez.query.files.shared=alter_merge_2
   vectorization_9.q,\
   vectorization_decimal_date.q,\
   vectorization_div0.q,\
+  vectorization_limit.q,\
   vectorization_nested_udf.q,\
   vectorization_not.q,\
   vectorization_part.q,\
@@ -204,7 +221,10 @@ minitez.query.files.shared=alter_merge_2
   vectorization_short_regress.q,\
   vectorized_bucketmapjoin1.q,\
   vectorized_case.q,\
+  vectorized_casts.q,\
   vectorized_context.q,\
+  vectorized_date_funcs.q,\
+  vectorized_distinct_gby.q,\
   vectorized_mapjoin.q,\
   vectorized_math_funcs.q,\
   vectorized_nested_mapjoin.q,\

Modified: hive/branches/branch-0.14/ql/src/gen/vectorization/ExpressionTemplates/ColumnArithmeticColumnDecimal.txt
URL: http://svn.apache.org/viewvc/hive/branches/branch-0.14/ql/src/gen/vectorization/ExpressionTemplates/ColumnArithmeticColumnDecimal.txt?rev=1636055&r1=1636054&r2=1636055&view=diff
==============================================================================
--- hive/branches/branch-0.14/ql/src/gen/vectorization/ExpressionTemplates/ColumnArithmeticColumnDecimal.txt (original)
+++ hive/branches/branch-0.14/ql/src/gen/vectorization/ExpressionTemplates/ColumnArithmeticColumnDecimal.txt Sat Nov  1 21:10:32 2014
@@ -24,7 +24,7 @@ import org.apache.hadoop.hive.ql.exec.ve
 import org.apache.hadoop.hive.ql.exec.vector.expressions.NullUtil;
 import org.apache.hadoop.hive.ql.exec.vector.expressions.DecimalUtil;
 import org.apache.hadoop.hive.ql.exec.vector.VectorExpressionDescriptor;
-import org.apache.hadoop.hive.common.type.Decimal128;
+import org.apache.hadoop.hive.serde2.io.HiveDecimalWritable;
 
 /**
  * Generated from template ColumnArithmeticColumnDecimal.txt, which covers binary arithmetic 
@@ -61,8 +61,8 @@ public class <ClassName> extends VectorE
     DecimalColumnVector outputColVector = (DecimalColumnVector) batch.cols[outputColumn];
     int[] sel = batch.selected;
     int n = batch.size;
-    Decimal128[] vector1 = inputColVector1.vector;
-    Decimal128[] vector2 = inputColVector2.vector;
+    HiveDecimalWritable[] vector1 = inputColVector1.vector;
+    HiveDecimalWritable[] vector2 = inputColVector2.vector;
 
     // return immediately if batch is empty
     if (n == 0) {

Modified: hive/branches/branch-0.14/ql/src/gen/vectorization/ExpressionTemplates/ColumnArithmeticScalarDecimal.txt
URL: http://svn.apache.org/viewvc/hive/branches/branch-0.14/ql/src/gen/vectorization/ExpressionTemplates/ColumnArithmeticScalarDecimal.txt?rev=1636055&r1=1636054&r2=1636055&view=diff
==============================================================================
--- hive/branches/branch-0.14/ql/src/gen/vectorization/ExpressionTemplates/ColumnArithmeticScalarDecimal.txt (original)
+++ hive/branches/branch-0.14/ql/src/gen/vectorization/ExpressionTemplates/ColumnArithmeticScalarDecimal.txt Sat Nov  1 21:10:32 2014
@@ -24,7 +24,8 @@ import org.apache.hadoop.hive.ql.exec.ve
 import org.apache.hadoop.hive.ql.exec.vector.expressions.NullUtil;
 import org.apache.hadoop.hive.ql.exec.vector.expressions.DecimalUtil;
 import org.apache.hadoop.hive.ql.exec.vector.VectorExpressionDescriptor;
-import org.apache.hadoop.hive.common.type.Decimal128;
+import org.apache.hadoop.hive.serde2.io.HiveDecimalWritable;
+import org.apache.hadoop.hive.common.type.HiveDecimal;
 
 /**
  * Generated from template ColumnArithmeticScalarDecimal.txt, which covers binary arithmetic 
@@ -35,10 +36,10 @@ public class <ClassName> extends VectorE
   private static final long serialVersionUID = 1L;
 
   private int colNum;
-  private Decimal128 value;
+  private HiveDecimal value;
   private int outputColumn;
 
-  public <ClassName>(int colNum, Decimal128 value, int outputColumn) {
+  public <ClassName>(int colNum, HiveDecimal value, int outputColumn) {
     this.colNum = colNum;
     this.value = value;
     this.outputColumn = outputColumn;
@@ -64,7 +65,7 @@ public class <ClassName> extends VectorE
     outputColVector.noNulls = inputColVector.noNulls;
     outputColVector.isRepeating = inputColVector.isRepeating;
     int n = batch.size;
-    Decimal128[] vector = inputColVector.vector;
+    HiveDecimalWritable[] vector = inputColVector.vector;
     
     // return immediately if batch is empty
     if (n == 0) {
@@ -129,26 +130,6 @@ public class <ClassName> extends VectorE
   public int getOutputColumn() {
     return outputColumn;
   }
-  
-  public int getColNum() {
-    return colNum;
-  }
-  
-  public void setColNum(int colNum) {
-    this.colNum = colNum;
-  }
-
-  public Decimal128 getValue() {
-    return value;
-  }
-
-  public void setValue(Decimal128 value) {
-    this.value = value;
-  }
-
-  public void setOutputColumn(int outputColumn) {
-    this.outputColumn = outputColumn;
-  }
 
   @Override
   public VectorExpressionDescriptor.Descriptor getDescriptor() {

Modified: hive/branches/branch-0.14/ql/src/gen/vectorization/ExpressionTemplates/ColumnDivideColumnDecimal.txt
URL: http://svn.apache.org/viewvc/hive/branches/branch-0.14/ql/src/gen/vectorization/ExpressionTemplates/ColumnDivideColumnDecimal.txt?rev=1636055&r1=1636054&r2=1636055&view=diff
==============================================================================
--- hive/branches/branch-0.14/ql/src/gen/vectorization/ExpressionTemplates/ColumnDivideColumnDecimal.txt (original)
+++ hive/branches/branch-0.14/ql/src/gen/vectorization/ExpressionTemplates/ColumnDivideColumnDecimal.txt Sat Nov  1 21:10:32 2014
@@ -24,7 +24,7 @@ import org.apache.hadoop.hive.ql.exec.ve
 import org.apache.hadoop.hive.ql.exec.vector.expressions.NullUtil;
 import org.apache.hadoop.hive.ql.exec.vector.expressions.DecimalUtil;
 import org.apache.hadoop.hive.ql.exec.vector.VectorExpressionDescriptor;
-import org.apache.hadoop.hive.common.type.Decimal128;
+import org.apache.hadoop.hive.serde2.io.HiveDecimalWritable;
 
 /**
  * Generated from template ColumnArithmeticColumnDecimal.txt, which covers binary arithmetic
@@ -61,8 +61,8 @@ public class <ClassName> extends VectorE
     DecimalColumnVector outputColVector = (DecimalColumnVector) batch.cols[outputColumn];
     int[] sel = batch.selected;
     int n = batch.size;
-    Decimal128[] vector1 = inputColVector1.vector;
-    Decimal128[] vector2 = inputColVector2.vector;
+    HiveDecimalWritable[] vector1 = inputColVector1.vector;
+    HiveDecimalWritable[] vector2 = inputColVector2.vector;
 
     // return immediately if batch is empty
     if (n == 0) {
@@ -138,26 +138,6 @@ public class <ClassName> extends VectorE
     return outputColumn;
   }
 
-  public int getColNum1() {
-    return colNum1;
-  }
-
-  public void setColNum1(int colNum1) {
-    this.colNum1 = colNum1;
-  }
-
-  public int getColNum2() {
-    return colNum2;
-  }
-
-  public void setColNum2(int colNum2) {
-    this.colNum2 = colNum2;
-  }
-
-  public void setOutputColumn(int outputColumn) {
-    this.outputColumn = outputColumn;
-  }
-
   @Override
   public VectorExpressionDescriptor.Descriptor getDescriptor() {
     return (new VectorExpressionDescriptor.Builder())

Modified: hive/branches/branch-0.14/ql/src/gen/vectorization/ExpressionTemplates/ColumnDivideScalarDecimal.txt
URL: http://svn.apache.org/viewvc/hive/branches/branch-0.14/ql/src/gen/vectorization/ExpressionTemplates/ColumnDivideScalarDecimal.txt?rev=1636055&r1=1636054&r2=1636055&view=diff
==============================================================================
--- hive/branches/branch-0.14/ql/src/gen/vectorization/ExpressionTemplates/ColumnDivideScalarDecimal.txt (original)
+++ hive/branches/branch-0.14/ql/src/gen/vectorization/ExpressionTemplates/ColumnDivideScalarDecimal.txt Sat Nov  1 21:10:32 2014
@@ -24,7 +24,8 @@ import org.apache.hadoop.hive.ql.exec.ve
 import org.apache.hadoop.hive.ql.exec.vector.expressions.NullUtil;
 import org.apache.hadoop.hive.ql.exec.vector.expressions.DecimalUtil;
 import org.apache.hadoop.hive.ql.exec.vector.VectorExpressionDescriptor;
-import org.apache.hadoop.hive.common.type.Decimal128;
+import org.apache.hadoop.hive.serde2.io.HiveDecimalWritable;
+import org.apache.hadoop.hive.common.type.HiveDecimal;
 
 /**
  * Generated from template ColumnDivideScalarDecimal.txt, which covers binary arithmetic
@@ -35,11 +36,10 @@ public class <ClassName> extends VectorE
   private static final long serialVersionUID = 1L;
 
   private int colNum;
-  private Decimal128 value;
+  private HiveDecimal value;
   private int outputColumn;
-  private transient Decimal128 zero;  // to hold constant 0 for later use
 
-  public <ClassName>(int colNum, Decimal128 value, int outputColumn) {
+  public <ClassName>(int colNum, HiveDecimal value, int outputColumn) {
     this.colNum = colNum;
     this.value = value;
     this.outputColumn = outputColumn;
@@ -65,13 +65,8 @@ public class <ClassName> extends VectorE
     outputColVector.noNulls = inputColVector.noNulls;
     outputColVector.isRepeating = inputColVector.isRepeating;
     int n = batch.size;
-    Decimal128[] vector = inputColVector.vector;
-    Decimal128[] outputVector = outputColVector.vector;
-
-    // Initialize local variable to use as 0 value on first use.
-    if (zero == null) {
-      this.zero = new Decimal128(0, inputColVector.scale);
-    }
+    HiveDecimalWritable[] vector = inputColVector.vector;
+    HiveDecimalWritable[] outputVector = outputColVector.vector;
 
     // return immediately if batch is empty
     if (n == 0) {
@@ -90,7 +85,7 @@ public class <ClassName> extends VectorE
     }
 
 
-    if (value.compareTo(zero) == 0) {
+    if (value.compareTo(HiveDecimal.ZERO) == 0) {
 
       // Denominator is zero, convert the batch to nulls
       outputColVector.noNulls = false;
@@ -142,26 +137,6 @@ public class <ClassName> extends VectorE
     return outputColumn;
   }
 
-  public int getColNum() {
-    return colNum;
-  }
-
-  public void setColNum(int colNum) {
-    this.colNum = colNum;
-  }
-
-  public Decimal128 getValue() {
-    return value;
-  }
-
-  public void setValue(Decimal128 value) {
-    this.value = value;
-  }
-
-  public void setOutputColumn(int outputColumn) {
-    this.outputColumn = outputColumn;
-  }
-
   @Override
   public VectorExpressionDescriptor.Descriptor getDescriptor() {
     return (new VectorExpressionDescriptor.Builder())

Modified: hive/branches/branch-0.14/ql/src/gen/vectorization/ExpressionTemplates/DecimalColumnUnaryFunc.txt
URL: http://svn.apache.org/viewvc/hive/branches/branch-0.14/ql/src/gen/vectorization/ExpressionTemplates/DecimalColumnUnaryFunc.txt?rev=1636055&r1=1636054&r2=1636055&view=diff
==============================================================================
--- hive/branches/branch-0.14/ql/src/gen/vectorization/ExpressionTemplates/DecimalColumnUnaryFunc.txt (original)
+++ hive/branches/branch-0.14/ql/src/gen/vectorization/ExpressionTemplates/DecimalColumnUnaryFunc.txt Sat Nov  1 21:10:32 2014
@@ -23,7 +23,7 @@ import org.apache.hadoop.hive.ql.exec.ve
 import org.apache.hadoop.hive.ql.exec.vector.*;
 import org.apache.hadoop.hive.ql.exec.vector.VectorizedRowBatch;
 import org.apache.hadoop.hive.ql.exec.vector.VectorExpressionDescriptor;
-import org.apache.hadoop.hive.common.type.Decimal128;
+import org.apache.hadoop.hive.serde2.io.HiveDecimalWritable;
 import org.apache.hadoop.hive.ql.exec.vector.DecimalColumnVector;
 import org.apache.hadoop.hive.ql.exec.vector.LongColumnVector;
 import org.apache.hadoop.hive.ql.exec.vector.expressions.DecimalUtil;
@@ -59,7 +59,7 @@ public class <ClassName> extends VectorE
     boolean[] outputIsNull = outputColVector.isNull;
     outputColVector.noNulls = inputColVector.noNulls;
     int n = batch.size;
-    Decimal128[] vector = inputColVector.vector;
+    HiveDecimalWritable[] vector = inputColVector.vector;
 
     // return immediately if batch is empty
     if (n == 0) {
@@ -117,18 +117,6 @@ public class <ClassName> extends VectorE
   public String getOutputType() {
     return outputType;
   }
-  
-  public int getColNum() {
-    return colNum;
-  }
-
-  public void setColNum(int colNum) {
-    this.colNum = colNum;
-  }
-
-  public void setOutputColumn(int outputColumn) {
-    this.outputColumn = outputColumn;
-  }
 
   @Override
   public VectorExpressionDescriptor.Descriptor getDescriptor() {

Modified: hive/branches/branch-0.14/ql/src/gen/vectorization/ExpressionTemplates/FilterDecimalColumnBetween.txt
URL: http://svn.apache.org/viewvc/hive/branches/branch-0.14/ql/src/gen/vectorization/ExpressionTemplates/FilterDecimalColumnBetween.txt?rev=1636055&r1=1636054&r2=1636055&view=diff
==============================================================================
--- hive/branches/branch-0.14/ql/src/gen/vectorization/ExpressionTemplates/FilterDecimalColumnBetween.txt (original)
+++ hive/branches/branch-0.14/ql/src/gen/vectorization/ExpressionTemplates/FilterDecimalColumnBetween.txt Sat Nov  1 21:10:32 2014
@@ -18,8 +18,10 @@
 
 package org.apache.hadoop.hive.ql.exec.vector.expressions.gen;
 
-import org.apache.hadoop.hive.common.type.Decimal128;
+import org.apache.hadoop.hive.serde2.io.HiveDecimalWritable;
+import org.apache.hadoop.hive.common.type.HiveDecimal;
 import org.apache.hadoop.hive.ql.exec.vector.DecimalColumnVector;
+import org.apache.hadoop.hive.ql.exec.vector.expressions.DecimalUtil;
 import org.apache.hadoop.hive.ql.exec.vector.expressions.VectorExpression;
 import org.apache.hadoop.hive.ql.exec.vector.VectorizedRowBatch;
 import org.apache.hadoop.hive.ql.exec.vector.VectorExpressionDescriptor;
@@ -37,10 +39,10 @@ public class <ClassName> extends VectorE
   private int colNum;
 
   // The comparison is of the form "column BETWEEN leftValue AND rightValue"
-  private Decimal128 leftValue;
-  private Decimal128 rightValue;
+  private HiveDecimal leftValue;
+  private HiveDecimal rightValue;
 
-  public <ClassName>(int colNum, Decimal128 leftValue, Decimal128 rightValue) {
+  public <ClassName>(int colNum, HiveDecimal leftValue, HiveDecimal rightValue) {
     this.colNum = colNum;
     this.leftValue = leftValue;
     this.rightValue = rightValue;
@@ -60,7 +62,7 @@ public class <ClassName> extends VectorE
     int[] sel = batch.selected;
     boolean[] nullPos = inputColVector.isNull;
     int n = batch.size;
-    Decimal128[] vector = inputColVector.vector;
+    HiveDecimalWritable[] vector = inputColVector.vector;
 
     // return immediately if batch is empty
     if (n == 0) {
@@ -72,7 +74,7 @@ public class <ClassName> extends VectorE
 
         // All must be selected otherwise size would be zero.
         // Repeating property will not change.
-        if (<OptionalNot>(vector[0].compareTo(leftValue) < 0 || vector[0].compareTo(rightValue) > 0)) {
+        if (<OptionalNot>(DecimalUtil.compare(vector[0], leftValue) < 0 || DecimalUtil.compare(vector[0], rightValue) > 0)) {
 
           // Entire batch is filtered out.
           batch.size = 0;
@@ -81,7 +83,7 @@ public class <ClassName> extends VectorE
         int newSize = 0;
         for(int j = 0; j != n; j++) {
           int i = sel[j];
-          if (<OptionalNot>(leftValue.compareTo(vector[i]) <= 0 && vector[i].compareTo(rightValue) <= 0)) {
+          if (<OptionalNot>(DecimalUtil.compare(leftValue, vector[i]) <= 0 && DecimalUtil.compare(vector[i], rightValue) <= 0)) {
             sel[newSize++] = i;
           }
         }
@@ -89,7 +91,7 @@ public class <ClassName> extends VectorE
       } else {
         int newSize = 0;
         for(int i = 0; i != n; i++) {
-          if (<OptionalNot>(leftValue.compareTo(vector[i]) <= 0 && vector[i].compareTo(rightValue) <= 0)) {
+          if (<OptionalNot>(DecimalUtil.compare(leftValue, vector[i]) <= 0 && DecimalUtil.compare(vector[i], rightValue) <= 0)) {
             sel[newSize++] = i;
           }
         }
@@ -104,7 +106,7 @@ public class <ClassName> extends VectorE
         // All must be selected otherwise size would be zero.
         // Repeating property will not change.
         if (!nullPos[0]) {
-          if (<OptionalNot>(vector[0].compareTo(leftValue) < 0 || vector[0].compareTo(rightValue) > 0)) {
+          if (<OptionalNot>(DecimalUtil.compare(vector[0], leftValue) < 0 || DecimalUtil.compare(vector[0], rightValue) > 0)) {
 
             // Entire batch is filtered out.
             batch.size = 0;
@@ -117,7 +119,7 @@ public class <ClassName> extends VectorE
         for(int j = 0; j != n; j++) {
           int i = sel[j];
           if (!nullPos[i]) {
-            if (<OptionalNot>(leftValue.compareTo(vector[i]) <= 0 && vector[i].compareTo(rightValue) <= 0)) {
+            if (<OptionalNot>(DecimalUtil.compare(leftValue, vector[i]) <= 0 && DecimalUtil.compare(vector[i], rightValue) <= 0)) {
              sel[newSize++] = i;
             }
           }
@@ -129,7 +131,7 @@ public class <ClassName> extends VectorE
         int newSize = 0;
         for(int i = 0; i != n; i++) {
           if (!nullPos[i]) {
-            if (<OptionalNot>(leftValue.compareTo(vector[i]) <= 0 && vector[i].compareTo(rightValue) <= 0)) {
+            if (<OptionalNot>(DecimalUtil.compare(leftValue, vector[i]) <= 0 && DecimalUtil.compare(vector[i], rightValue) <= 0)) {
               sel[newSize++] = i;
             }
           }
@@ -152,30 +154,6 @@ public class <ClassName> extends VectorE
     return "boolean";
   }
 
-  public int getColNum() {
-    return colNum;
-  }
-
-  public void setColNum(int colNum) {
-    this.colNum = colNum;
-  }
-
-  public Decimal128 getLeftValue() {
-    return leftValue;
-  }
-
-  public void setLeftValue(Decimal128 value) {
-    this.leftValue = value;
-  }
-
-  public Decimal128 getRightValue() {
-    return rightValue;
-  }
-
-  public void setRightValue(Decimal128 value) {
-    this.leftValue = value;
-  }
-
   @Override
   public VectorExpressionDescriptor.Descriptor getDescriptor() {
     return (new VectorExpressionDescriptor.Builder())

Modified: hive/branches/branch-0.14/ql/src/gen/vectorization/ExpressionTemplates/FilterDecimalColumnCompareColumn.txt
URL: http://svn.apache.org/viewvc/hive/branches/branch-0.14/ql/src/gen/vectorization/ExpressionTemplates/FilterDecimalColumnCompareColumn.txt?rev=1636055&r1=1636054&r2=1636055&view=diff
==============================================================================
--- hive/branches/branch-0.14/ql/src/gen/vectorization/ExpressionTemplates/FilterDecimalColumnCompareColumn.txt (original)
+++ hive/branches/branch-0.14/ql/src/gen/vectorization/ExpressionTemplates/FilterDecimalColumnCompareColumn.txt Sat Nov  1 21:10:32 2014
@@ -22,7 +22,7 @@ import org.apache.hadoop.hive.ql.exec.ve
 import org.apache.hadoop.hive.ql.exec.vector.DecimalColumnVector;
 import org.apache.hadoop.hive.ql.exec.vector.VectorizedRowBatch;
 import org.apache.hadoop.hive.ql.exec.vector.VectorExpressionDescriptor;
-import org.apache.hadoop.hive.common.type.Decimal128;
+import org.apache.hadoop.hive.serde2.io.HiveDecimalWritable;
 
 /**
  * Generated from template FilterDecimalColumnCompareColumn.txt, which covers binary comparison 
@@ -57,8 +57,8 @@ public class <ClassName> extends VectorE
     boolean[] nullPos1 = inputColVector1.isNull;
     boolean[] nullPos2 = inputColVector2.isNull;
     int n = batch.size;
-    Decimal128[] vector1 = inputColVector1.vector;
-    Decimal128[] vector2 = inputColVector2.vector;
+    HiveDecimalWritable[] vector1 = inputColVector1.vector;
+    HiveDecimalWritable[] vector2 = inputColVector2.vector;
 
     // return immediately if batch is empty
     if (n == 0) {
@@ -428,22 +428,6 @@ public class <ClassName> extends VectorE
   public int getOutputColumn() {
     return -1;
   }
-  
-  public int getColNum1() {
-    return colNum1;
-  }
-
-  public void setColNum1(int colNum1) {
-    this.colNum1 = colNum1;
-  }
-
-  public int getColNum2() {
-    return colNum2;
-  }
-
-  public void setColNum2(int colNum2) {
-    this.colNum2 = colNum2;
-  }
 
   @Override
   public VectorExpressionDescriptor.Descriptor getDescriptor() {

Modified: hive/branches/branch-0.14/ql/src/gen/vectorization/ExpressionTemplates/FilterDecimalColumnCompareScalar.txt
URL: http://svn.apache.org/viewvc/hive/branches/branch-0.14/ql/src/gen/vectorization/ExpressionTemplates/FilterDecimalColumnCompareScalar.txt?rev=1636055&r1=1636054&r2=1636055&view=diff
==============================================================================
--- hive/branches/branch-0.14/ql/src/gen/vectorization/ExpressionTemplates/FilterDecimalColumnCompareScalar.txt (original)
+++ hive/branches/branch-0.14/ql/src/gen/vectorization/ExpressionTemplates/FilterDecimalColumnCompareScalar.txt Sat Nov  1 21:10:32 2014
@@ -20,9 +20,11 @@ package org.apache.hadoop.hive.ql.exec.v
 
 import org.apache.hadoop.hive.ql.exec.vector.expressions.VectorExpression;
 import org.apache.hadoop.hive.ql.exec.vector.DecimalColumnVector;
+import org.apache.hadoop.hive.ql.exec.vector.expressions.DecimalUtil;
 import org.apache.hadoop.hive.ql.exec.vector.VectorizedRowBatch;
 import org.apache.hadoop.hive.ql.exec.vector.VectorExpressionDescriptor;
-import org.apache.hadoop.hive.common.type.Decimal128;
+import org.apache.hadoop.hive.serde2.io.HiveDecimalWritable;
+import org.apache.hadoop.hive.common.type.HiveDecimal;
 
 /**
  * This is a generated class to evaluate a <OperatorSymbol> comparison on a vector of decimal
@@ -33,9 +35,9 @@ public class <ClassName> extends VectorE
   private static final long serialVersionUID = 1L;
 
   private int colNum;
-  private Decimal128 value;
+  private HiveDecimal value;
 
-  public <ClassName>(int colNum, Decimal128 value) {
+  public <ClassName>(int colNum, HiveDecimal value) {
     this.colNum = colNum;
     this.value = value;
   }
@@ -52,7 +54,7 @@ public class <ClassName> extends VectorE
     int[] sel = batch.selected;
     boolean[] nullPos = inputColVector.isNull;
     int n = batch.size;
-    Decimal128[] vector = inputColVector.vector;
+    HiveDecimalWritable[] vector = inputColVector.vector;
 
     // return immediately if batch is empty
     if (n == 0) {
@@ -63,7 +65,7 @@ public class <ClassName> extends VectorE
       if (inputColVector.isRepeating) {
 
         // All must be selected otherwise size would be zero. Repeating property will not change.
-        if (!(vector[0].compareTo(value) <OperatorSymbol> 0)) {
+        if (!(DecimalUtil.compare(vector[0], value) <OperatorSymbol> 0)) {
 
           // Entire batch is filtered out.
           batch.size = 0;
@@ -72,7 +74,7 @@ public class <ClassName> extends VectorE
         int newSize = 0;
         for(int j = 0; j != n; j++) {
           int i = sel[j];
-          if (vector[i].compareTo(value) <OperatorSymbol> 0) {
+          if (DecimalUtil.compare(vector[i], value) <OperatorSymbol> 0) {
             sel[newSize++] = i;
           }
         }
@@ -80,7 +82,7 @@ public class <ClassName> extends VectorE
       } else {
         int newSize = 0;
         for(int i = 0; i != n; i++) {
-          if (vector[i].compareTo(value) <OperatorSymbol> 0) {
+          if (DecimalUtil.compare(vector[i], value) <OperatorSymbol> 0) {
             sel[newSize++] = i;
           }
         }
@@ -94,7 +96,7 @@ public class <ClassName> extends VectorE
 
         // All must be selected otherwise size would be zero. Repeating property will not change.
         if (!nullPos[0]) {
-          if (!(vector[0].compareTo(value) <OperatorSymbol> 0)) {
+          if (!(DecimalUtil.compare(vector[0], value) <OperatorSymbol> 0)) {
 
             // Entire batch is filtered out.
             batch.size = 0;
@@ -107,7 +109,7 @@ public class <ClassName> extends VectorE
         for(int j = 0; j != n; j++) {
           int i = sel[j];
           if (!nullPos[i]) {
-           if (vector[i].compareTo(value) <OperatorSymbol> 0) {
+           if (DecimalUtil.compare(vector[i], value) <OperatorSymbol> 0) {
              sel[newSize++] = i;
            }
           }
@@ -119,7 +121,7 @@ public class <ClassName> extends VectorE
         int newSize = 0;
         for(int i = 0; i != n; i++) {
           if (!nullPos[i]) {
-            if (vector[i].compareTo(value) <OperatorSymbol> 0) {
+            if (DecimalUtil.compare(vector[i], value) <OperatorSymbol> 0) {
               sel[newSize++] = i;
             }
           }
@@ -142,22 +144,6 @@ public class <ClassName> extends VectorE
     return "boolean";
   }
 
-  public int getColNum() {
-    return colNum;
-  }
-
-  public void setColNum(int colNum) {
-    this.colNum = colNum;
-  }
-
-  public Decimal128 getValue() {
-    return value;
-  }
-
-  public void setValue(Decimal128 value) {
-    this.value = value;
-  }
-
   @Override
   public VectorExpressionDescriptor.Descriptor getDescriptor() {
     return (new VectorExpressionDescriptor.Builder())

Modified: hive/branches/branch-0.14/ql/src/gen/vectorization/ExpressionTemplates/FilterDecimalScalarCompareColumn.txt
URL: http://svn.apache.org/viewvc/hive/branches/branch-0.14/ql/src/gen/vectorization/ExpressionTemplates/FilterDecimalScalarCompareColumn.txt?rev=1636055&r1=1636054&r2=1636055&view=diff
==============================================================================
--- hive/branches/branch-0.14/ql/src/gen/vectorization/ExpressionTemplates/FilterDecimalScalarCompareColumn.txt (original)
+++ hive/branches/branch-0.14/ql/src/gen/vectorization/ExpressionTemplates/FilterDecimalScalarCompareColumn.txt Sat Nov  1 21:10:32 2014
@@ -20,9 +20,11 @@ package org.apache.hadoop.hive.ql.exec.v
 
 import org.apache.hadoop.hive.ql.exec.vector.expressions.VectorExpression;
 import org.apache.hadoop.hive.ql.exec.vector.DecimalColumnVector;
+import org.apache.hadoop.hive.ql.exec.vector.expressions.DecimalUtil;
 import org.apache.hadoop.hive.ql.exec.vector.VectorizedRowBatch;
 import org.apache.hadoop.hive.ql.exec.vector.VectorExpressionDescriptor;
-import org.apache.hadoop.hive.common.type.Decimal128;
+import org.apache.hadoop.hive.serde2.io.HiveDecimalWritable;
+import org.apache.hadoop.hive.common.type.HiveDecimal;
 
 /**
  * This is a generated class to evaluate a <OperatorSymbol> comparison on a vector of decimal
@@ -33,9 +35,9 @@ public class <ClassName> extends VectorE
   private static final long serialVersionUID = 1L;
 
   private int colNum;
-  private Decimal128 value;
+  private HiveDecimal value;
 
-  public <ClassName>(Decimal128 value, int colNum) {
+  public <ClassName>(HiveDecimal value, int colNum) {
     this.colNum = colNum;
     this.value = value;
   }
@@ -52,7 +54,7 @@ public class <ClassName> extends VectorE
     int[] sel = batch.selected;
     boolean[] nullPos = inputColVector.isNull;
     int n = batch.size;
-    Decimal128[] vector = inputColVector.vector;
+    HiveDecimalWritable[] vector = inputColVector.vector;
 
     // return immediately if batch is empty
     if (n == 0) {
@@ -63,7 +65,7 @@ public class <ClassName> extends VectorE
       if (inputColVector.isRepeating) {
 
         // All must be selected otherwise size would be zero. Repeating property will not change.
-        if (!(value.compareTo(vector[0]) <OperatorSymbol> 0)) {
+        if (!(DecimalUtil.compare(value, vector[0]) <OperatorSymbol> 0)) {
 
           // Entire batch is filtered out.
           batch.size = 0;
@@ -72,7 +74,7 @@ public class <ClassName> extends VectorE
         int newSize = 0;
         for(int j = 0; j != n; j++) {
           int i = sel[j];
-          if (value.compareTo(vector[i]) <OperatorSymbol> 0) {
+          if (DecimalUtil.compare(value, vector[i]) <OperatorSymbol> 0) {
             sel[newSize++] = i;
           }
         }
@@ -80,7 +82,7 @@ public class <ClassName> extends VectorE
       } else {
         int newSize = 0;
         for(int i = 0; i != n; i++) {
-          if (value.compareTo(vector[i]) <OperatorSymbol> 0) {
+          if (DecimalUtil.compare(value, vector[i]) <OperatorSymbol> 0) {
             sel[newSize++] = i;
           }
         }
@@ -94,7 +96,7 @@ public class <ClassName> extends VectorE
 
         // All must be selected otherwise size would be zero. Repeating property will not change.
         if (!nullPos[0]) {
-          if (!(value.compareTo(vector[0]) <OperatorSymbol> 0)) {
+          if (!(DecimalUtil.compare(value, vector[0]) <OperatorSymbol> 0)) {
 
             // Entire batch is filtered out.
             batch.size = 0;
@@ -107,7 +109,7 @@ public class <ClassName> extends VectorE
         for(int j = 0; j != n; j++) {
           int i = sel[j];
           if (!nullPos[i]) {
-           if (value.compareTo(vector[i]) <OperatorSymbol> 0) {
+           if (DecimalUtil.compare(value, vector[i]) <OperatorSymbol> 0) {
              sel[newSize++] = i;
            }
           }
@@ -119,7 +121,7 @@ public class <ClassName> extends VectorE
         int newSize = 0;
         for(int i = 0; i != n; i++) {
           if (!nullPos[i]) {
-            if (value.compareTo(vector[i]) <OperatorSymbol> 0) {
+            if (DecimalUtil.compare(value, vector[i]) <OperatorSymbol> 0) {
               sel[newSize++] = i;
             }
           }
@@ -142,22 +144,6 @@ public class <ClassName> extends VectorE
     return "boolean";
   }
 
-  public int getColNum() {
-    return colNum;
-  }
-
-  public void setColNum(int colNum) {
-    this.colNum = colNum;
-  }
-
-  public Decimal128 getValue() {
-    return value;
-  }
-
-  public void setValue(Decimal128 value) {
-    this.value = value;
-  }
-
   @Override
   public VectorExpressionDescriptor.Descriptor getDescriptor() {
     return (new VectorExpressionDescriptor.Builder())

Modified: hive/branches/branch-0.14/ql/src/gen/vectorization/ExpressionTemplates/ScalarArithmeticColumnDecimal.txt
URL: http://svn.apache.org/viewvc/hive/branches/branch-0.14/ql/src/gen/vectorization/ExpressionTemplates/ScalarArithmeticColumnDecimal.txt?rev=1636055&r1=1636054&r2=1636055&view=diff
==============================================================================
--- hive/branches/branch-0.14/ql/src/gen/vectorization/ExpressionTemplates/ScalarArithmeticColumnDecimal.txt (original)
+++ hive/branches/branch-0.14/ql/src/gen/vectorization/ExpressionTemplates/ScalarArithmeticColumnDecimal.txt Sat Nov  1 21:10:32 2014
@@ -24,7 +24,8 @@ import org.apache.hadoop.hive.ql.exec.ve
 import org.apache.hadoop.hive.ql.exec.vector.expressions.NullUtil;
 import org.apache.hadoop.hive.ql.exec.vector.expressions.DecimalUtil;
 import org.apache.hadoop.hive.ql.exec.vector.VectorExpressionDescriptor;
-import org.apache.hadoop.hive.common.type.Decimal128;
+import org.apache.hadoop.hive.serde2.io.HiveDecimalWritable;
+import org.apache.hadoop.hive.common.type.HiveDecimal;
 
 /**
  * Generated from template ScalarArithmeticColumnDecimal.txt, which covers binary arithmetic 
@@ -35,10 +36,10 @@ public class <ClassName> extends VectorE
   private static final long serialVersionUID = 1L;
 
   private int colNum;
-  private Decimal128 value;
+  private HiveDecimal value;
   private int outputColumn;
 
-  public <ClassName>(Decimal128 value, int colNum, int outputColumn) {
+  public <ClassName>(HiveDecimal value, int colNum, int outputColumn) {
     this.colNum = colNum;
     this.value = value;
     this.outputColumn = outputColumn;
@@ -64,7 +65,7 @@ public class <ClassName> extends VectorE
     outputColVector.noNulls = inputColVector.noNulls;
     outputColVector.isRepeating = inputColVector.isRepeating;
     int n = batch.size;
-    Decimal128[] vector = inputColVector.vector;
+    HiveDecimalWritable[] vector = inputColVector.vector;
     
     // return immediately if batch is empty
     if (n == 0) {
@@ -126,26 +127,6 @@ public class <ClassName> extends VectorE
   public int getOutputColumn() {
     return outputColumn;
   }
-  
-  public int getColNum() {
-    return colNum;
-  }
-  
-  public void setColNum(int colNum) {
-    this.colNum = colNum;
-  }
-
-  public Decimal128 getValue() {
-    return value;
-  }
-
-  public void setValue(Decimal128 value) {
-    this.value = value;
-  }
-
-  public void setOutputColumn(int outputColumn) {
-    this.outputColumn = outputColumn;
-  }
 
   @Override
   public VectorExpressionDescriptor.Descriptor getDescriptor() {

Modified: hive/branches/branch-0.14/ql/src/gen/vectorization/ExpressionTemplates/ScalarDivideColumnDecimal.txt
URL: http://svn.apache.org/viewvc/hive/branches/branch-0.14/ql/src/gen/vectorization/ExpressionTemplates/ScalarDivideColumnDecimal.txt?rev=1636055&r1=1636054&r2=1636055&view=diff
==============================================================================
--- hive/branches/branch-0.14/ql/src/gen/vectorization/ExpressionTemplates/ScalarDivideColumnDecimal.txt (original)
+++ hive/branches/branch-0.14/ql/src/gen/vectorization/ExpressionTemplates/ScalarDivideColumnDecimal.txt Sat Nov  1 21:10:32 2014
@@ -24,7 +24,8 @@ import org.apache.hadoop.hive.ql.exec.ve
 import org.apache.hadoop.hive.ql.exec.vector.expressions.NullUtil;
 import org.apache.hadoop.hive.ql.exec.vector.expressions.DecimalUtil;
 import org.apache.hadoop.hive.ql.exec.vector.VectorExpressionDescriptor;
-import org.apache.hadoop.hive.common.type.Decimal128;
+import org.apache.hadoop.hive.serde2.io.HiveDecimalWritable;
+import org.apache.hadoop.hive.common.type.HiveDecimal;
 
 /**
  * Generated from template ScalarDivideColumnDecimal.txt, which covers binary arithmetic
@@ -35,10 +36,10 @@ public class <ClassName> extends VectorE
   private static final long serialVersionUID = 1L;
 
   private int colNum;
-  private Decimal128 value;
+  private HiveDecimal value;
   private int outputColumn;
 
-  public <ClassName>(Decimal128 value, int colNum, int outputColumn) {
+  public <ClassName>(HiveDecimal value, int colNum, int outputColumn) {
     this.colNum = colNum;
     this.value = value;
     this.outputColumn = outputColumn;
@@ -64,8 +65,8 @@ public class <ClassName> extends VectorE
     outputColVector.noNulls = inputColVector.noNulls;
     outputColVector.isRepeating = inputColVector.isRepeating;
     int n = batch.size;
-    Decimal128[] vector = inputColVector.vector;
-    Decimal128[] outputVector = outputColVector.vector;
+    HiveDecimalWritable[] vector = inputColVector.vector;
+    HiveDecimalWritable[] outputVector = outputColVector.vector;
 
     // return immediately if batch is empty
     if (n == 0) {
@@ -129,26 +130,6 @@ public class <ClassName> extends VectorE
     return outputColumn;
   }
 
-  public int getColNum() {
-    return colNum;
-  }
-
-  public void setColNum(int colNum) {
-    this.colNum = colNum;
-  }
-
-  public Decimal128 getValue() {
-    return value;
-  }
-
-  public void setValue(Decimal128 value) {
-    this.value = value;
-  }
-
-  public void setOutputColumn(int outputColumn) {
-    this.outputColumn = outputColumn;
-  }
-
   @Override
   public VectorExpressionDescriptor.Descriptor getDescriptor() {
     return (new VectorExpressionDescriptor.Builder())

Modified: hive/branches/branch-0.14/ql/src/gen/vectorization/UDAFTemplates/VectorUDAFMinMaxDecimal.txt
URL: http://svn.apache.org/viewvc/hive/branches/branch-0.14/ql/src/gen/vectorization/UDAFTemplates/VectorUDAFMinMaxDecimal.txt?rev=1636055&r1=1636054&r2=1636055&view=diff
==============================================================================
--- hive/branches/branch-0.14/ql/src/gen/vectorization/UDAFTemplates/VectorUDAFMinMaxDecimal.txt (original)
+++ hive/branches/branch-0.14/ql/src/gen/vectorization/UDAFTemplates/VectorUDAFMinMaxDecimal.txt Sat Nov  1 21:10:32 2014
@@ -18,7 +18,6 @@
 
 package org.apache.hadoop.hive.ql.exec.vector.expressions.aggregates.gen;
 
-import org.apache.hadoop.hive.common.type.Decimal128;
 import org.apache.hadoop.hive.common.type.HiveDecimal;
 import org.apache.hadoop.hive.ql.exec.Description;
 import org.apache.hadoop.hive.ql.exec.vector.expressions.VectorExpression;
@@ -31,6 +30,7 @@ import org.apache.hadoop.hive.ql.exec.ve
 import org.apache.hadoop.hive.ql.metadata.HiveException;
 import org.apache.hadoop.hive.ql.plan.AggregationDesc;
 import org.apache.hadoop.hive.ql.util.JavaDataModel;
+import org.apache.hadoop.hive.serde2.io.HiveDecimalWritable;
 import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspector;
 
 /**
@@ -49,7 +49,7 @@ public class <ClassName> extends VectorA
 
       private static final long serialVersionUID = 1L;
 
-      transient private final Decimal128 value;
+      transient private final HiveDecimalWritable value;
 
       /**
       * Value is explicitly (re)initialized in reset()
@@ -57,15 +57,16 @@ public class <ClassName> extends VectorA
       transient private boolean isNull = true;
 
       public Aggregation() {
-        value = new Decimal128();
+        value = new HiveDecimalWritable();
       }
 
-      public void checkValue(Decimal128 value, short scale) {
+      public void checkValue(HiveDecimalWritable writable, short scale) {
+        HiveDecimal value = writable.getHiveDecimal();
         if (isNull) {
           isNull = false;
-          this.value.update(value);
-        } else if (this.value.compareTo(value) <OperatorSymbol> 0) {
-          this.value.update(value, scale);
+          this.value.set(value);
+        } else if (this.value.getHiveDecimal().compareTo(value) <OperatorSymbol> 0) {
+          this.value.set(value);
         }
       }
 
@@ -77,7 +78,7 @@ public class <ClassName> extends VectorA
       @Override
       public void reset () {
         isNull = true;
-        value.zeroClear();
+        value.set(HiveDecimal.ZERO);
       }
     }
 
@@ -124,7 +125,7 @@ public class <ClassName> extends VectorA
 
       DecimalColumnVector inputVector = (DecimalColumnVector)batch.
         cols[this.inputExpression.getOutputColumn()];
-      Decimal128[] vector = inputVector.vector;
+      HiveDecimalWritable[] vector = inputVector.vector;
 
       if (inputVector.noNulls) {
         if (inputVector.isRepeating) {
@@ -170,7 +171,7 @@ public class <ClassName> extends VectorA
     private void iterateNoNullsRepeatingWithAggregationSelection(
       VectorAggregationBufferRow[] aggregationBufferSets,
       int aggregrateIndex,
-      Decimal128 value,
+      HiveDecimalWritable value,
       short scale,
       int batchSize) {
 
@@ -186,7 +187,7 @@ public class <ClassName> extends VectorA
     private void iterateNoNullsSelectionWithAggregationSelection(
       VectorAggregationBufferRow[] aggregationBufferSets,
       int aggregrateIndex,
-      Decimal128[] values,
+      HiveDecimalWritable[] values,
       short scale,
       int[] selection,
       int batchSize) {
@@ -203,7 +204,7 @@ public class <ClassName> extends VectorA
     private void iterateNoNullsWithAggregationSelection(
       VectorAggregationBufferRow[] aggregationBufferSets,
       int aggregrateIndex,
-      Decimal128[] values,
+      HiveDecimalWritable[] values,
       short scale,
       int batchSize) {
       for (int i=0; i < batchSize; ++i) {
@@ -218,7 +219,7 @@ public class <ClassName> extends VectorA
     private void iterateHasNullsRepeatingSelectionWithAggregationSelection(
       VectorAggregationBufferRow[] aggregationBufferSets,
       int aggregrateIndex,
-      Decimal128 value,
+      HiveDecimalWritable value,
       short scale,
       int batchSize,
       int[] selection,
@@ -239,7 +240,7 @@ public class <ClassName> extends VectorA
     private void iterateHasNullsRepeatingWithAggregationSelection(
       VectorAggregationBufferRow[] aggregationBufferSets,
       int aggregrateIndex,
-      Decimal128 value,
+      HiveDecimalWritable value,
       short scale,
       int batchSize,
       boolean[] isNull) {
@@ -258,7 +259,7 @@ public class <ClassName> extends VectorA
     private void iterateHasNullsSelectionWithAggregationSelection(
       VectorAggregationBufferRow[] aggregationBufferSets,
       int aggregrateIndex,
-      Decimal128[] values,
+      HiveDecimalWritable[] values,
       short scale,
       int batchSize,
       int[] selection,
@@ -279,7 +280,7 @@ public class <ClassName> extends VectorA
     private void iterateHasNullsWithAggregationSelection(
       VectorAggregationBufferRow[] aggregationBufferSets,
       int aggregrateIndex,
-      Decimal128[] values,
+      HiveDecimalWritable[] values,
       short scale,
       int batchSize,
       boolean[] isNull) {
@@ -312,13 +313,14 @@ public class <ClassName> extends VectorA
 
         Aggregation myagg = (Aggregation)agg;
 
-        Decimal128[] vector = inputVector.vector;
+        HiveDecimalWritable[] vector = inputVector.vector;
 
         if (inputVector.isRepeating) {
           if (inputVector.noNulls &&
             (myagg.isNull || (myagg.value.compareTo(vector[0]) <OperatorSymbol> 0))) {
             myagg.isNull = false;
-            myagg.value.update(vector[0], inputVector.scale);
+            HiveDecimal value = vector[0].getHiveDecimal();
+            myagg.value.set(value);
           }
           return;
         }
@@ -341,7 +343,7 @@ public class <ClassName> extends VectorA
 
     private void iterateSelectionHasNulls(
         Aggregation myagg,
-        Decimal128[] vector,
+        HiveDecimalWritable[] vector,
         short scale,
         int batchSize,
         boolean[] isNull,
@@ -350,13 +352,13 @@ public class <ClassName> extends VectorA
       for (int j=0; j< batchSize; ++j) {
         int i = selected[j];
         if (!isNull[i]) {
-          Decimal128 value = vector[i];
+          HiveDecimal value = vector[i].getHiveDecimal();
           if (myagg.isNull) {
             myagg.isNull = false;
-            myagg.value.update(value);
+            myagg.value.set(value);
           }
-          else if (myagg.value.compareTo(value) <OperatorSymbol> 0) {
-            myagg.value.update(value, scale);
+          else if (myagg.value.getHiveDecimal().compareTo(value) <OperatorSymbol> 0) {
+            myagg.value.set(value);
           }
         }
       }
@@ -364,40 +366,41 @@ public class <ClassName> extends VectorA
 
     private void iterateSelectionNoNulls(
         Aggregation myagg,
-        Decimal128[] vector,
+        HiveDecimalWritable[] vector,
         short scale,
         int batchSize,
         int[] selected) {
 
       if (myagg.isNull) {
-        myagg.value.update(vector[selected[0]]);
+        HiveDecimal value = vector[selected[0]].getHiveDecimal();
+        myagg.value.set(value);
         myagg.isNull = false;
       }
 
       for (int i=0; i< batchSize; ++i) {
-        Decimal128 value = vector[selected[i]];
-        if (myagg.value.compareTo(value) <OperatorSymbol> 0) {
-          myagg.value.update(value, scale);
+        HiveDecimal value = vector[selected[i]].getHiveDecimal();
+        if (myagg.value.getHiveDecimal().compareTo(value) <OperatorSymbol> 0) {
+          myagg.value.set(value);
         }
       }
     }
 
     private void iterateNoSelectionHasNulls(
         Aggregation myagg,
-        Decimal128[] vector,
+        HiveDecimalWritable[] vector,
         short scale,
         int batchSize,
         boolean[] isNull) {
 
       for(int i=0;i<batchSize;++i) {
         if (!isNull[i]) {
-          Decimal128 value = vector[i];
+          HiveDecimal value = vector[i].getHiveDecimal();
           if (myagg.isNull) {
-            myagg.value.update(value, scale);
+            myagg.value.set(value);
             myagg.isNull = false;
           }
-          else if (myagg.value.compareTo(value) <OperatorSymbol> 0) {
-            myagg.value.update(value, scale);
+          else if (myagg.value.getHiveDecimal().compareTo(value) <OperatorSymbol> 0) {
+            myagg.value.set(value);
           }
         }
       }
@@ -405,18 +408,19 @@ public class <ClassName> extends VectorA
 
     private void iterateNoSelectionNoNulls(
         Aggregation myagg,
-        Decimal128[] vector,
+        HiveDecimalWritable[] vector,
         short scale,
         int batchSize) {
       if (myagg.isNull) {
-        myagg.value.update(vector[0]);
+        HiveDecimal value = vector[0].getHiveDecimal();
+        myagg.value.set(value);
         myagg.isNull = false;
       }
 
       for (int i=0;i<batchSize;++i) {
-        Decimal128 value = vector[i];
-        if (myagg.value.compareTo(value) <OperatorSymbol> 0) {
-          myagg.value.update(value, scale);
+        HiveDecimal value = vector[i].getHiveDecimal();
+        if (myagg.value.getHiveDecimal().compareTo(value) <OperatorSymbol> 0) {
+          myagg.value.set(value);
         }
       }
     }

Modified: hive/branches/branch-0.14/ql/src/gen/vectorization/UDAFTemplates/VectorUDAFVarDecimal.txt
URL: http://svn.apache.org/viewvc/hive/branches/branch-0.14/ql/src/gen/vectorization/UDAFTemplates/VectorUDAFVarDecimal.txt?rev=1636055&r1=1636054&r2=1636055&view=diff
==============================================================================
--- hive/branches/branch-0.14/ql/src/gen/vectorization/UDAFTemplates/VectorUDAFVarDecimal.txt (original)
+++ hive/branches/branch-0.14/ql/src/gen/vectorization/UDAFTemplates/VectorUDAFVarDecimal.txt Sat Nov  1 21:10:32 2014
@@ -21,7 +21,7 @@ package org.apache.hadoop.hive.ql.exec.v
 import java.util.ArrayList;
 import java.util.List;
 
-import org.apache.hadoop.hive.common.type.Decimal128;
+import org.apache.hadoop.hive.serde2.io.HiveDecimalWritable;
 import org.apache.hadoop.hive.common.type.HiveDecimal;
 import org.apache.hadoop.hive.ql.exec.Description;
 import org.apache.hadoop.hive.ql.exec.vector.expressions.VectorExpression;
@@ -86,12 +86,12 @@ public class <ClassName> extends VectorA
         variance = 0f;
       }
 
-      public void updateValueWithCheckAndInit(Decimal128 value, short scale) {
+      public void updateValueWithCheckAndInit(HiveDecimalWritable value, short scale) {
         if (this.isNull) {
           this.init();
         }
 
-        double dval = value.doubleValue();
+        double dval = value.getHiveDecimal().doubleValue();
         this.sum += dval;
         this.count += 1;
         if(this.count > 1) {
@@ -100,8 +100,8 @@ public class <ClassName> extends VectorA
         }
       }
 
-      public void updateValueNoCheck(Decimal128 value, short scale) {
-        double dval = value.doubleValue();
+      public void updateValueNoCheck(HiveDecimalWritable value, short scale) {
+        double dval = value.getHiveDecimal().doubleValue();
         this.sum += dval;
         this.count += 1;
         double t = this.count*dval - this.sum;
@@ -176,7 +176,7 @@ public class <ClassName> extends VectorA
         return;
       }
 
-      Decimal128[] vector = inputVector.vector;
+      HiveDecimalWritable[] vector = inputVector.vector;
 
       if (inputVector.isRepeating) {
         if (inputVector.noNulls || !inputVector.isNull[0]) {
@@ -209,7 +209,7 @@ public class <ClassName> extends VectorA
     private void  iterateRepeatingNoNullsWithAggregationSelection(
         VectorAggregationBufferRow[] aggregationBufferSets,
         int aggregateIndex,
-        Decimal128 value,
+        HiveDecimalWritable value,
         short scale,
         int batchSize) {
 
@@ -225,7 +225,7 @@ public class <ClassName> extends VectorA
     private void iterateSelectionHasNullsWithAggregationSelection(
         VectorAggregationBufferRow[] aggregationBufferSets,
         int aggregateIndex,
-        Decimal128[] vector,
+        HiveDecimalWritable[] vector,
         short scale,
         int batchSize,
         boolean[] isNull,
@@ -238,7 +238,7 @@ public class <ClassName> extends VectorA
           j);
         int i = selected[j];
         if (!isNull[i]) {
-          Decimal128 value = vector[i];
+          HiveDecimalWritable value = vector[i];
           myagg.updateValueWithCheckAndInit(value, scale);
         }
       }
@@ -247,7 +247,7 @@ public class <ClassName> extends VectorA
     private void iterateSelectionNoNullsWithAggregationSelection(
         VectorAggregationBufferRow[] aggregationBufferSets,
         int aggregateIndex,
-        Decimal128[] vector,
+        HiveDecimalWritable[] vector,
         short scale,
         int batchSize,
         int[] selected) {
@@ -257,7 +257,7 @@ public class <ClassName> extends VectorA
           aggregationBufferSets,
           aggregateIndex,
           i);
-        Decimal128 value = vector[selected[i]];
+        HiveDecimalWritable value = vector[selected[i]];
         myagg.updateValueWithCheckAndInit(value, scale);
       }
     }
@@ -265,7 +265,7 @@ public class <ClassName> extends VectorA
     private void iterateNoSelectionHasNullsWithAggregationSelection(
         VectorAggregationBufferRow[] aggregationBufferSets,
         int aggregateIndex,
-        Decimal128[] vector,
+        HiveDecimalWritable[] vector,
         short scale,
         int batchSize,
         boolean[] isNull) {
@@ -276,7 +276,7 @@ public class <ClassName> extends VectorA
             aggregationBufferSets,
             aggregateIndex,
           i);
-          Decimal128 value = vector[i];
+          HiveDecimalWritable value = vector[i];
           myagg.updateValueWithCheckAndInit(value, scale);
         }
       }
@@ -285,7 +285,7 @@ public class <ClassName> extends VectorA
     private void iterateNoSelectionNoNullsWithAggregationSelection(
         VectorAggregationBufferRow[] aggregationBufferSets,
         int aggregateIndex,
-        Decimal128[] vector,
+        HiveDecimalWritable[] vector,
         short scale,
         int batchSize) {
 
@@ -294,7 +294,7 @@ public class <ClassName> extends VectorA
           aggregationBufferSets,
           aggregateIndex,
           i);
-        Decimal128 value = vector[i];
+        HiveDecimalWritable value = vector[i];
         myagg.updateValueWithCheckAndInit(value, scale);
       }
     }
@@ -316,7 +316,7 @@ public class <ClassName> extends VectorA
 
       Aggregation myagg = (Aggregation)agg;
 
-      Decimal128[] vector = inputVector.vector;
+      HiveDecimalWritable[] vector = inputVector.vector;
 
       if (inputVector.isRepeating) {
         if (inputVector.noNulls) {
@@ -340,7 +340,7 @@ public class <ClassName> extends VectorA
 
     private void  iterateRepeatingNoNulls(
         Aggregation myagg,
-        Decimal128 value,
+        HiveDecimalWritable value,
         short scale,
         int batchSize) {
 
@@ -357,7 +357,7 @@ public class <ClassName> extends VectorA
 
     private void iterateSelectionHasNulls(
         Aggregation myagg,
-        Decimal128[] vector,
+        HiveDecimalWritable[] vector,
         short scale,
         int batchSize,
         boolean[] isNull,
@@ -366,7 +366,7 @@ public class <ClassName> extends VectorA
       for (int j=0; j< batchSize; ++j) {
         int i = selected[j];
         if (!isNull[i]) {
-          Decimal128 value = vector[i];
+          HiveDecimalWritable value = vector[i];
           myagg.updateValueWithCheckAndInit(value, scale);
         }
       }
@@ -374,7 +374,7 @@ public class <ClassName> extends VectorA
 
     private void iterateSelectionNoNulls(
         Aggregation myagg,
-        Decimal128[] vector,
+        HiveDecimalWritable[] vector,
         short scale,
         int batchSize,
         int[] selected) {
@@ -383,7 +383,7 @@ public class <ClassName> extends VectorA
         myagg.init ();
       }
 
-      Decimal128 value = vector[selected[0]];
+      HiveDecimalWritable value = vector[selected[0]];
       myagg.updateValueWithCheckAndInit(value, scale);
 
       // i=0 was pulled out to remove the count > 1 check in the loop
@@ -396,14 +396,14 @@ public class <ClassName> extends VectorA
 
     private void iterateNoSelectionHasNulls(
         Aggregation myagg,
-        Decimal128[] vector,
+        HiveDecimalWritable[] vector,
         short scale,
         int batchSize,
         boolean[] isNull) {
 
       for(int i=0;i<batchSize;++i) {
         if (!isNull[i]) {
-          Decimal128 value = vector[i];
+          HiveDecimalWritable value = vector[i];
           myagg.updateValueWithCheckAndInit(value, scale);
         }
       }
@@ -411,7 +411,7 @@ public class <ClassName> extends VectorA
 
     private void iterateNoSelectionNoNulls(
         Aggregation myagg,
-        Decimal128[] vector,
+        HiveDecimalWritable[] vector,
         short scale,
         int batchSize) {
 
@@ -419,7 +419,7 @@ public class <ClassName> extends VectorA
         myagg.init ();
       }
 
-      Decimal128 value = vector[0];
+      HiveDecimalWritable value = vector[0];
       myagg.updateValueWithCheckAndInit(value, scale);
 
       // i=0 was pulled out to remove count > 1 check

Modified: hive/branches/branch-0.14/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/DecimalColumnVector.java
URL: http://svn.apache.org/viewvc/hive/branches/branch-0.14/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/DecimalColumnVector.java?rev=1636055&r1=1636054&r2=1636055&view=diff
==============================================================================
--- hive/branches/branch-0.14/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/DecimalColumnVector.java (original)
+++ hive/branches/branch-0.14/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/DecimalColumnVector.java Sat Nov  1 21:10:32 2014
@@ -17,26 +17,25 @@
  */
 
 package org.apache.hadoop.hive.ql.exec.vector;
-import org.apache.hadoop.hive.common.type.Decimal128;
+
+import java.math.BigInteger;
+
+import org.apache.hadoop.hive.serde2.io.HiveDecimalWritable;
 import org.apache.hadoop.hive.common.type.HiveDecimal;
 import org.apache.hadoop.hive.ql.exec.vector.VectorizedRowBatch;
-import org.apache.hadoop.hive.serde2.io.HiveDecimalWritable;
 import org.apache.hadoop.io.NullWritable;
 import org.apache.hadoop.io.Writable;
 
 public class DecimalColumnVector extends ColumnVector {
 
   /**
-   * A vector if Decimal128 objects. These are mutable and have fairly
-   * efficient operations on them. This will make it faster to load
-   * column vectors and perform decimal vector operations with decimal-
-   * specific VectorExpressions.
+   * A vector of HiveDecimalWritable objects.
    *
    * For high performance and easy access to this low-level structure,
    * the fields are public by design (as they are in other ColumnVector
    * types).
    */
-  public Decimal128[] vector;
+  public HiveDecimalWritable[] vector;
   public short scale;
   public short precision;
 
@@ -51,9 +50,9 @@ public class DecimalColumnVector extends
     this.precision = (short) precision;
     this.scale = (short) scale;
     final int len = size;
-    vector = new Decimal128[len];
+    vector = new HiveDecimalWritable[len];
     for (int i = 0; i < len; i++) {
-      vector[i] = new Decimal128(0, this.scale);
+      vector[i] = new HiveDecimalWritable(HiveDecimal.ZERO);
     }
   }
 
@@ -65,8 +64,7 @@ public class DecimalColumnVector extends
     if (!noNulls && isNull[index]) {
       return NullWritable.get();
     } else {
-      Decimal128 dec = vector[index];
-      writableObj.set(HiveDecimal.create(dec.toBigDecimal()));
+      writableObj.set(vector[index]);
       return writableObj;
     }
   }
@@ -78,22 +76,38 @@ public class DecimalColumnVector extends
 
   @Override
   public void setElement(int outElementNum, int inputElementNum, ColumnVector inputVector) {
-    vector[outElementNum].update(((DecimalColumnVector) inputVector).vector[inputElementNum]);
-    vector[outElementNum].changeScaleDestructive(scale);
+    HiveDecimal hiveDec = ((DecimalColumnVector) inputVector).vector[inputElementNum].getHiveDecimal(precision, scale);
+    if (hiveDec == null) {
+      noNulls = false;
+      isNull[outElementNum] = true;
+    } else {
+      vector[outElementNum].set(hiveDec);
+    }
   }
 
-  /**
-   * Check if the value at position i fits in the available precision,
-   * and convert the value to NULL if it does not.
-   */
-  public void checkPrecisionOverflow(int i) {
-    try {
-      vector[i].checkPrecisionOverflow(precision);
-    } catch (ArithmeticException e) {
+  public void set(int elementNum, HiveDecimalWritable writeable) {
+    HiveDecimal hiveDec = writeable.getHiveDecimal(precision, scale);
+    if (hiveDec == null) {
+      noNulls = false;
+      isNull[elementNum] = true;
+    } else {
+      vector[elementNum].set(hiveDec);
+    }
+  }
 
-      // If the value won't fit in the available precision, the result is NULL
+  public void set(int elementNum, HiveDecimal hiveDec) {
+    HiveDecimal checkedDec = HiveDecimal.enforcePrecisionScale(hiveDec, precision, scale);
+    if (checkedDec == null) {
       noNulls = false;
-      isNull[i] = true;
+      isNull[elementNum] = true;
+    } else {
+      vector[elementNum].set(checkedDec);
     }
   }
+
+  public void setNullDataValue(int elementNum) {
+    // E.g. For scale 2 the minimum is "0.01"
+    HiveDecimal minimumNonZeroValue = HiveDecimal.create(BigInteger.ONE, scale);
+    vector[elementNum].set(minimumNonZeroValue);
+  }
 }

Modified: hive/branches/branch-0.14/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/VectorColumnAssignFactory.java
URL: http://svn.apache.org/viewvc/hive/branches/branch-0.14/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/VectorColumnAssignFactory.java?rev=1636055&r1=1636054&r2=1636055&view=diff
==============================================================================
--- hive/branches/branch-0.14/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/VectorColumnAssignFactory.java (original)
+++ hive/branches/branch-0.14/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/VectorColumnAssignFactory.java Sat Nov  1 21:10:32 2014
@@ -23,7 +23,6 @@ import java.util.Arrays;
 import java.util.List;
 import java.util.Map;
 
-import org.apache.hadoop.hive.common.type.Decimal128;
 import org.apache.hadoop.hive.common.type.HiveChar;
 import org.apache.hadoop.hive.common.type.HiveDecimal;
 import org.apache.hadoop.hive.common.type.HiveVarchar;
@@ -152,18 +151,12 @@ public class VectorColumnAssignFactory {
 
   private static abstract class VectorDecimalColumnAssign
   extends VectorColumnAssignVectorBase<DecimalColumnVector> {
+
     protected void assignDecimal(HiveDecimal value, int index) {
-      outCol.vector[index].update(value.unscaledValue(), (byte) value.scale());
-    }
-  
-    protected void assignDecimal(Decimal128 value, int index) {
-      outCol.vector[index].update(value);
+      outCol.set(index, value);
     }
     protected void assignDecimal(HiveDecimalWritable hdw, int index) {
-        byte[] internalStorage = hdw.getInternalStorage();
-        int scale = hdw.getScale();
-  
-        outCol.vector[index].fastUpdateFromInternalStorage(internalStorage, (short)scale);
+      outCol.set(index, hdw);
     }
   }
 

Modified: hive/branches/branch-0.14/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/VectorGroupKeyHelper.java
URL: http://svn.apache.org/viewvc/hive/branches/branch-0.14/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/VectorGroupKeyHelper.java?rev=1636055&r1=1636054&r2=1636055&view=diff
==============================================================================
--- hive/branches/branch-0.14/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/VectorGroupKeyHelper.java (original)
+++ hive/branches/branch-0.14/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/VectorGroupKeyHelper.java Sat Nov  1 21:10:32 2014
@@ -113,9 +113,9 @@ public class VectorGroupKeyHelper extend
       DecimalColumnVector outputColumnVector = (DecimalColumnVector) outputBatch.cols[keyIndex];
       if (inputColumnVector.noNulls || !inputColumnVector.isNull[0]) {
 
-        // Since we store references to Decimal128 instances, we must use the update method instead
+        // Since we store references to HiveDecimalWritable instances, we must use the update method instead
         // of plain assignment.
-        outputColumnVector.vector[outputBatch.size].update(inputColumnVector.vector[0]);
+        outputColumnVector.set(outputBatch.size, inputColumnVector.vector[0]);
       } else {
         outputColumnVector.noNulls = false;
         outputColumnVector.isNull[outputBatch.size] = true;

Modified: hive/branches/branch-0.14/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/VectorHashKeyWrapper.java
URL: http://svn.apache.org/viewvc/hive/branches/branch-0.14/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/VectorHashKeyWrapper.java?rev=1636055&r1=1636054&r2=1636055&view=diff
==============================================================================
--- hive/branches/branch-0.14/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/VectorHashKeyWrapper.java (original)
+++ hive/branches/branch-0.14/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/VectorHashKeyWrapper.java Sat Nov  1 21:10:32 2014
@@ -20,7 +20,8 @@ package org.apache.hadoop.hive.ql.exec.v
 
 import java.util.Arrays;
 
-import org.apache.hadoop.hive.common.type.Decimal128;
+import org.apache.hadoop.hive.serde2.io.HiveDecimalWritable;
+import org.apache.hadoop.hive.common.type.HiveDecimal;
 import org.apache.hadoop.hive.ql.exec.KeyWrapper;
 import org.apache.hadoop.hive.ql.exec.vector.expressions.StringExpr;
 import org.apache.hadoop.hive.ql.metadata.HiveException;
@@ -40,7 +41,7 @@ public class VectorHashKeyWrapper extend
   private static final long[] EMPTY_LONG_ARRAY = new long[0];
   private static final double[] EMPTY_DOUBLE_ARRAY = new double[0];
   private static final byte[][] EMPTY_BYTES_ARRAY = new byte[0][];
-  private static final Decimal128[] EMPTY_DECIMAL_ARRAY = new Decimal128[0];
+  private static final HiveDecimalWritable[] EMPTY_DECIMAL_ARRAY = new HiveDecimalWritable[0];
 
   private long[] longValues;
   private double[] doubleValues;
@@ -49,7 +50,7 @@ public class VectorHashKeyWrapper extend
   private int[] byteStarts;
   private int[] byteLengths;
 
-  private Decimal128[] decimalValues;
+  private HiveDecimalWritable[] decimalValues;
 
   private boolean[] isNull;
   private int hashcode;
@@ -58,9 +59,9 @@ public class VectorHashKeyWrapper extend
           int byteValuesCount, int decimalValuesCount) {
     longValues = longValuesCount > 0 ? new long[longValuesCount] : EMPTY_LONG_ARRAY;
     doubleValues = doubleValuesCount > 0 ? new double[doubleValuesCount] : EMPTY_DOUBLE_ARRAY;
-    decimalValues = decimalValuesCount > 0 ? new Decimal128[decimalValuesCount] : EMPTY_DECIMAL_ARRAY;
+    decimalValues = decimalValuesCount > 0 ? new HiveDecimalWritable[decimalValuesCount] : EMPTY_DECIMAL_ARRAY;
     for(int i = 0; i < decimalValuesCount; ++i) {
-      decimalValues[i] = new Decimal128();
+      decimalValues[i] = new HiveDecimalWritable(HiveDecimal.ZERO);
     }
     if (byteValuesCount > 0) {
       byteValues = new byte[byteValuesCount][];
@@ -87,9 +88,12 @@ public class VectorHashKeyWrapper extend
   public void setHashKey() {
     hashcode = Arrays.hashCode(longValues) ^
         Arrays.hashCode(doubleValues) ^
-        Arrays.hashCode(decimalValues) ^
         Arrays.hashCode(isNull);
 
+    for (int i = 0; i < decimalValues.length; i++) {
+      hashcode ^= decimalValues[i].getHiveDecimal().hashCode();
+    }
+
     // This code, with branches and all, is not executed if there are no string keys
     for (int i = 0; i < byteValues.length; ++i) {
       /*
@@ -166,10 +170,10 @@ public class VectorHashKeyWrapper extend
     clone.isNull = isNull.clone();
 
     if (decimalValues.length > 0) {
-      // Decimal128 requires deep clone
-      clone.decimalValues = new Decimal128[decimalValues.length];
-      for (int i = 0; i < decimalValues.length; ++i) {
-        clone.decimalValues[i] = new Decimal128().update(decimalValues[i]);
+      // Decimal columns use HiveDecimalWritable.
+      clone.decimalValues = new HiveDecimalWritable[decimalValues.length];
+      for(int i = 0; i < decimalValues.length; ++i) {
+        clone.decimalValues[i] = new HiveDecimalWritable(decimalValues[i]);
       }
     } else {
       clone.decimalValues = EMPTY_DECIMAL_ARRAY;
@@ -243,8 +247,8 @@ public class VectorHashKeyWrapper extend
     isNull[longValues.length + doubleValues.length + index] = true;
   }
 
-  public void assignDecimal(int index, Decimal128 value) {
-    decimalValues[index].update(value);
+  public void assignDecimal(int index, HiveDecimalWritable value) {
+    decimalValues[index].set(value);
     isNull[longValues.length + doubleValues.length + byteValues.length + index] = false;
   }
 
@@ -308,7 +312,7 @@ public class VectorHashKeyWrapper extend
     return isNull[longValues.length + doubleValues.length + byteValues.length + i];
   }
 
-  public Decimal128 getDecimal(int i) {
+  public HiveDecimalWritable getDecimal(int i) {
     return decimalValues[i];
   }
 }

Modified: hive/branches/branch-0.14/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/VectorHashKeyWrapperBatch.java
URL: http://svn.apache.org/viewvc/hive/branches/branch-0.14/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/VectorHashKeyWrapperBatch.java?rev=1636055&r1=1636054&r2=1636055&view=diff
==============================================================================
--- hive/branches/branch-0.14/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/VectorHashKeyWrapperBatch.java (original)
+++ hive/branches/branch-0.14/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/VectorHashKeyWrapperBatch.java Sat Nov  1 21:10:32 2014
@@ -580,7 +580,7 @@ public class VectorHashKeyWrapperBatch e
     } else if (klh.decimalIndex >= 0) {
       return kw.getIsDecimalNull(klh.decimalIndex)? null :
           keyOutputWriter.writeValue(
-                kw.getDecimal(klh.decimalIndex));
+                kw.getDecimal(klh.decimalIndex).getHiveDecimal());
     }
     else {
       throw new HiveException(String.format(

Modified: hive/branches/branch-0.14/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/VectorizationContext.java
URL: http://svn.apache.org/viewvc/hive/branches/branch-0.14/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/VectorizationContext.java?rev=1636055&r1=1636054&r2=1636055&view=diff
==============================================================================
--- hive/branches/branch-0.14/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/VectorizationContext.java (original)
+++ hive/branches/branch-0.14/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/VectorizationContext.java Sat Nov  1 21:10:32 2014
@@ -19,6 +19,7 @@
 package org.apache.hadoop.hive.ql.exec.vector;
 
 import java.lang.reflect.Constructor;
+import java.math.BigDecimal;
 import java.sql.Date;
 import java.sql.Timestamp;
 import java.util.ArrayList;
@@ -34,7 +35,7 @@ import java.util.regex.Pattern;
 
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
-import org.apache.hadoop.hive.common.type.Decimal128;
+import org.apache.hadoop.hive.serde2.io.HiveDecimalWritable;
 import org.apache.hadoop.hive.common.type.HiveChar;
 import org.apache.hadoop.hive.common.type.HiveDecimal;
 import org.apache.hadoop.hive.common.type.HiveVarchar;
@@ -868,7 +869,7 @@ public class VectorizationContext {
     case FLOAT_FAMILY:
       return new ConstantVectorExpression(outCol, ((Number) constantValue).doubleValue());
     case DECIMAL:
-      VectorExpression ve = new ConstantVectorExpression(outCol, (Decimal128) constantValue);
+      VectorExpression ve = new ConstantVectorExpression(outCol, (HiveDecimal) constantValue);
       // Set type name with decimal precision, scale, etc.
       ve.setOutputType(typeName);
       return ve;
@@ -1237,9 +1238,9 @@ public class VectorizationContext {
       ((IDoubleInExpr) expr).setInListValues(inValsD);
     } else if (isDecimalFamily(colType)) {
       cl = (mode == Mode.FILTER ? FilterDecimalColumnInList.class : DecimalColumnInList.class);
-      Decimal128[] inValsD = new Decimal128[childrenForInList.size()];
+      HiveDecimal[] inValsD = new HiveDecimal[childrenForInList.size()];
       for (int i = 0; i != inValsD.length; i++) {
-        inValsD[i] = (Decimal128) getVectorTypeScalarValue(
+        inValsD[i] = (HiveDecimal) getVectorTypeScalarValue(
             (ExprNodeConstantDesc)  childrenForInList.get(i));
       }
       expr = createVectorExpression(cl, childExpr.subList(0, 1), Mode.PROJECTION, returnType);
@@ -1287,44 +1288,43 @@ public class VectorizationContext {
     return null;
   }
 
-  private Decimal128 castConstantToDecimal(Object scalar, TypeInfo type) throws HiveException {
+  private HiveDecimal castConstantToDecimal(Object scalar, TypeInfo type) throws HiveException {
     PrimitiveTypeInfo ptinfo = (PrimitiveTypeInfo) type;
-    String typename = type.getTypeName();
-    Decimal128 d = new Decimal128();
     int scale = HiveDecimalUtils.getScaleForType(ptinfo);
+    String typename = type.getTypeName();
+    HiveDecimal rawDecimal;
     switch (ptinfo.getPrimitiveCategory()) {
     case FLOAT:
-      float floatVal = ((Float) scalar).floatValue();
-      d.update(floatVal, (short) scale);
+      rawDecimal = HiveDecimal.create(String.valueOf((Float) scalar));
       break;
     case DOUBLE:
-      double doubleVal = ((Double) scalar).doubleValue();
-      d.update(doubleVal, (short) scale);
+      rawDecimal = HiveDecimal.create(String.valueOf((Double) scalar));
       break;
     case BYTE:
-      byte byteVal = ((Byte) scalar).byteValue();
-      d.update(byteVal, (short) scale);
+      rawDecimal = HiveDecimal.create((Byte) scalar);
       break;
     case SHORT:
-      short shortVal = ((Short) scalar).shortValue();
-      d.update(shortVal, (short) scale);
+      rawDecimal = HiveDecimal.create((Short) scalar);
       break;
     case INT:
-      int intVal = ((Integer) scalar).intValue();
-      d.update(intVal, (short) scale);
+      rawDecimal = HiveDecimal.create((Integer) scalar);
       break;
     case LONG:
-      long longVal = ((Long) scalar).longValue();
-      d.update(longVal, (short) scale);
+      rawDecimal = HiveDecimal.create((Long) scalar);
       break;
     case DECIMAL:
-      HiveDecimal decimalVal = (HiveDecimal) scalar;
-      d.update(decimalVal.unscaledValue(), (short) scale);
+      rawDecimal = (HiveDecimal) scalar;
       break;
     default:
-      throw new HiveException("Unsupported type "+typename+" for cast to Decimal128");
+      throw new HiveException("Unsupported type " + typename + " for cast to HiveDecimal");
+    }
+    if (rawDecimal == null) {
+      if (LOG.isDebugEnabled()) {
+        LOG.debug("Casting constant scalar " + scalar + " to HiveDecimal resulted in null");
+      }
+      return null;
     }
-    return d;
+    return rawDecimal;
   }
 
   private String castConstantToString(Object scalar, TypeInfo type) throws HiveException {
@@ -1391,7 +1391,7 @@ public class VectorizationContext {
     if (child instanceof ExprNodeConstantDesc) {
      // Return a constant vector expression
       Object constantValue = ((ExprNodeConstantDesc) child).getValue();
-      Decimal128 decimalValue = castConstantToDecimal(constantValue, child.getTypeInfo());
+      HiveDecimal decimalValue = castConstantToDecimal(constantValue, child.getTypeInfo());
       return getConstantVectorExpression(decimalValue, returnType, Mode.PROJECTION);
     } else if (child instanceof ExprNodeNullDesc) {
       return getConstantVectorExpression(null, returnType, Mode.PROJECTION);
@@ -1801,10 +1801,7 @@ public class VectorizationContext {
         return 0;
       }
     } else if (decimalTypePattern.matcher(constDesc.getTypeString()).matches()) {
-      HiveDecimal hd = (HiveDecimal) constDesc.getValue();
-      Decimal128 dvalue = new Decimal128();
-      dvalue.update(hd.unscaledValue(), (short) hd.scale());
-      return dvalue;
+      return (HiveDecimal) constDesc.getValue();
     } else {
       return constDesc.getValue();
     }

Modified: hive/branches/branch-0.14/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/VectorizedBatchUtil.java
URL: http://svn.apache.org/viewvc/hive/branches/branch-0.14/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/VectorizedBatchUtil.java?rev=1636055&r1=1636054&r2=1636055&view=diff
==============================================================================
--- hive/branches/branch-0.14/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/VectorizedBatchUtil.java (original)
+++ hive/branches/branch-0.14/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/VectorizedBatchUtil.java Sat Nov  1 21:10:32 2014
@@ -481,8 +481,7 @@ public class VectorizedBatchUtil {
       if (writableCol != null) {
         dcv.isNull[rowIndex] = false;
         HiveDecimalWritable wobj = (HiveDecimalWritable) writableCol;
-        dcv.vector[rowIndex].update(wobj.getHiveDecimal().unscaledValue(),
-            (short) wobj.getScale());
+        dcv.set(rowIndex, wobj);
       } else {
         setNullColIsNullValue(dcv, rowIndex);
       }

Modified: hive/branches/branch-0.14/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/VectorizedRowBatchCtx.java
URL: http://svn.apache.org/viewvc/hive/branches/branch-0.14/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/VectorizedRowBatchCtx.java?rev=1636055&r1=1636054&r2=1636055&view=diff
==============================================================================
--- hive/branches/branch-0.14/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/VectorizedRowBatchCtx.java (original)
+++ hive/branches/branch-0.14/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/VectorizedRowBatchCtx.java Sat Nov  1 21:10:32 2014
@@ -34,7 +34,6 @@ import java.util.regex.Pattern;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.hive.common.type.Decimal128;
 import org.apache.hadoop.hive.common.type.HiveDecimal;
 import org.apache.hadoop.hive.metastore.api.hive_metastoreConstants;
 import org.apache.hadoop.hive.ql.exec.Utilities;
@@ -539,7 +538,7 @@ public class VectorizedRowBatchCtx {
             dv.isRepeating = true;
           } else {
             HiveDecimal hd = (HiveDecimal) value;
-            dv.vector[0] = new Decimal128(hd.toString(), (short) hd.scale());
+            dv.set(0, hd);
             dv.isRepeating = true;
             dv.isNull[0] = false;      
           }

Modified: hive/branches/branch-0.14/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/CastDecimalToBoolean.java
URL: http://svn.apache.org/viewvc/hive/branches/branch-0.14/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/CastDecimalToBoolean.java?rev=1636055&r1=1636054&r2=1636055&view=diff
==============================================================================
--- hive/branches/branch-0.14/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/CastDecimalToBoolean.java (original)
+++ hive/branches/branch-0.14/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/CastDecimalToBoolean.java Sat Nov  1 21:10:32 2014
@@ -41,6 +41,6 @@ public class CastDecimalToBoolean extend
    * Otherwise, return 1 for true.
    */
   protected void func(LongColumnVector outV, DecimalColumnVector inV,  int i) {
-    outV.vector[i] = inV.vector[i].getSignum() == 0 ? 0 : 1;
+    outV.vector[i] = inV.vector[i].getHiveDecimal().signum() == 0 ? 0 : 1;
   }
 }