You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hive.apache.org by mm...@apache.org on 2018/07/16 14:27:07 UTC

[3/3] hive git commit: HIVE-20174: Vectorization: Fix NULL / Wrong Results issues in GROUP BY Aggregation Functions (Matt McCline, reviewed by Teddy Choi)

HIVE-20174: Vectorization: Fix NULL / Wrong Results issues in GROUP BY Aggregation Functions (Matt McCline, reviewed by Teddy Choi)


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

Branch: refs/heads/master
Commit: 0966a383d48348c36c270ddbcba2b4516c6f3a24
Parents: 64ceb7b
Author: Matt McCline <mm...@hortonworks.com>
Authored: Mon Jul 16 09:14:44 2018 -0500
Committer: Matt McCline <mm...@hortonworks.com>
Committed: Mon Jul 16 09:14:44 2018 -0500

----------------------------------------------------------------------
 .../UDAFTemplates/VectorUDAFAvg.txt             | 108 +--
 .../UDAFTemplates/VectorUDAFAvgDecimal.txt      |  83 +--
 .../VectorUDAFAvgDecimal64ToDecimal.txt         | 110 ++-
 .../UDAFTemplates/VectorUDAFAvgDecimalMerge.txt |  35 +-
 .../UDAFTemplates/VectorUDAFAvgMerge.txt        |  35 +-
 .../UDAFTemplates/VectorUDAFAvgTimestamp.txt    | 136 +---
 .../UDAFTemplates/VectorUDAFMinMax.txt          |  38 +-
 .../UDAFTemplates/VectorUDAFMinMaxDecimal.txt   |  98 +--
 .../VectorUDAFMinMaxIntervalDayTime.txt         | 102 +--
 .../UDAFTemplates/VectorUDAFMinMaxString.txt    |  50 +-
 .../UDAFTemplates/VectorUDAFMinMaxTimestamp.txt | 104 +--
 .../UDAFTemplates/VectorUDAFSum.txt             |  34 +-
 .../UDAFTemplates/VectorUDAFVar.txt             |   9 +-
 .../UDAFTemplates/VectorUDAFVarDecimal.txt      |  19 +-
 .../UDAFTemplates/VectorUDAFVarMerge.txt        |  60 +-
 .../UDAFTemplates/VectorUDAFVarTimestamp.txt    |  19 +-
 .../ql/exec/vector/VectorAggregationDesc.java   |  19 +-
 .../aggregates/VectorAggregateExpression.java   |   2 +-
 .../aggregates/VectorUDAFCountMerge.java        |  34 +-
 .../aggregates/VectorUDAFSumDecimal.java        |  37 +-
 .../aggregates/VectorUDAFSumDecimal64.java      |  34 +-
 .../VectorUDAFSumDecimal64ToDecimal.java        |  34 +-
 .../aggregates/VectorUDAFSumTimestamp.java      |  34 +-
 .../hive/ql/optimizer/physical/Vectorizer.java  |  54 +-
 .../hive/ql/udf/generic/GenericUDAFAverage.java |  21 +
 .../ql/udf/generic/GenericUDAFVariance.java     |  19 +-
 .../exec/vector/TestVectorGroupByOperator.java  |  52 +-
 .../ql/exec/vector/VectorRandomBatchSource.java |  51 +-
 .../ql/exec/vector/VectorRandomRowSource.java   | 253 ++++++-
 .../vector/aggregation/AggregationBase.java     | 473 +++++++++++++
 .../aggregation/TestVectorAggregation.java      | 664 +++++++++++++++++++
 .../expressions/TestVectorDateAddSub.java       |   2 +
 .../vector/expressions/TestVectorDateDiff.java  |   2 +
 .../expressions/TestVectorIfStatement.java      |   4 +
 .../vector/expressions/TestVectorNegative.java  |   5 +
 .../expressions/TestVectorStringConcat.java     |   4 +
 .../expressions/TestVectorStringUnary.java      |   2 +
 .../vector/expressions/TestVectorSubStr.java    |   2 +
 .../expressions/TestVectorTimestampExtract.java |   4 +
 .../ql/optimizer/physical/TestVectorizer.java   |   5 +-
 40 files changed, 1835 insertions(+), 1016 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hive/blob/0966a383/ql/src/gen/vectorization/UDAFTemplates/VectorUDAFAvg.txt
----------------------------------------------------------------------
diff --git a/ql/src/gen/vectorization/UDAFTemplates/VectorUDAFAvg.txt b/ql/src/gen/vectorization/UDAFTemplates/VectorUDAFAvg.txt
index fc3d01f..cf5cc69 100644
--- a/ql/src/gen/vectorization/UDAFTemplates/VectorUDAFAvg.txt
+++ b/ql/src/gen/vectorization/UDAFTemplates/VectorUDAFAvg.txt
@@ -56,20 +56,9 @@ public class <ClassName> extends VectorAggregateExpression {
       transient private double sum;
       transient private long count;
 
-      /**
-      * Value is explicitly (re)initialized in reset()
-      */
-      transient private boolean isNull = true;
-
       public void avgValue(<ValueType> value) {
-        if (isNull) {
-          sum = value;
-          count = 1;
-          isNull = false;
-        } else {
-          sum += value;
-          count++;
-        }
+        sum += value;
+        count++;
       }
 
       @Override
@@ -79,7 +68,6 @@ public class <ClassName> extends VectorAggregateExpression {
 
       @Override
       public void reset () {
-        isNull = true;
         sum = 0;
         count = 0L;
       }
@@ -151,15 +139,9 @@ public class <ClassName> extends VectorAggregateExpression {
         }
       } else {
         if (inputVector.isRepeating) {
-          if (batch.selectedInUse) {
-            iterateHasNullsRepeatingSelectionWithAggregationSelection(
-              aggregationBufferSets, bufferIndex,
-              vector[0], batchSize, batch.selected, inputVector.isNull);
-          } else {
-            iterateHasNullsRepeatingWithAggregationSelection(
-              aggregationBufferSets, bufferIndex,
-              vector[0], batchSize, inputVector.isNull);
-          }
+          iterateHasNullsRepeatingWithAggregationSelection(
+            aggregationBufferSets, bufferIndex,
+            vector[0], batchSize, inputVector.isNull);
         } else {
           if (batch.selectedInUse) {
             iterateHasNullsSelectionWithAggregationSelection(
@@ -219,28 +201,6 @@ public class <ClassName> extends VectorAggregateExpression {
       }
     }
 
-    private void iterateHasNullsRepeatingSelectionWithAggregationSelection(
-      VectorAggregationBufferRow[] aggregationBufferSets,
-      int bufferIndex,
-      <ValueType> value,
-      int batchSize,
-      int[] selection,
-      boolean[] isNull) {
-
-      if (isNull[0]) {
-        return;
-      }
-
-      for (int i=0; i < batchSize; ++i) {
-        Aggregation myagg = getCurrentAggregationBuffer(
-          aggregationBufferSets,
-          bufferIndex,
-          i);
-        myagg.avgValue(value);
-      }
-
-    }
-
     private void iterateHasNullsRepeatingWithAggregationSelection(
       VectorAggregationBufferRow[] aggregationBufferSets,
       int bufferIndex,
@@ -321,11 +281,6 @@ public class <ClassName> extends VectorAggregateExpression {
 
       if (inputVector.isRepeating) {
         if (inputVector.noNulls || !inputVector.isNull[0]) {
-          if (myagg.isNull) {
-            myagg.isNull = false;
-            myagg.sum = 0;
-            myagg.count = 0;
-          }
           myagg.sum += vector[0]*batchSize;
           myagg.count += batchSize;
         }
@@ -353,14 +308,8 @@ public class <ClassName> extends VectorAggregateExpression {
       for (int j=0; j< batchSize; ++j) {
         int i = selected[j];
         if (!isNull[i]) {
-          <ValueType> value = vector[i];
-          if (myagg.isNull) {
-            myagg.isNull = false;
-            myagg.sum = 0;
-            myagg.count = 0;
-          }
-          myagg.sum += value;
-          myagg.count += 1;
+          myagg.sum += vector[i];
+          myagg.count++;
         }
       }
     }
@@ -371,16 +320,9 @@ public class <ClassName> extends VectorAggregateExpression {
         int batchSize,
         int[] selected) {
 
-      if (myagg.isNull) {
-        myagg.isNull = false;
-        myagg.sum = 0;
-        myagg.count = 0;
-      }
-
       for (int i=0; i< batchSize; ++i) {
-        <ValueType> value = vector[selected[i]];
-        myagg.sum += value;
-        myagg.count += 1;
+        myagg.sum += vector[selected[i]];
+        myagg.count++;
       }
     }
 
@@ -392,13 +334,7 @@ public class <ClassName> extends VectorAggregateExpression {
 
       for(int i=0;i<batchSize;++i) {
         if (!isNull[i]) {
-          <ValueType> value = vector[i];
-          if (myagg.isNull) {
-            myagg.isNull = false;
-            myagg.sum = 0;
-            myagg.count = 0;
-          }
-          myagg.sum += value;
+          myagg.sum += vector[i];
           myagg.count += 1;
         }
       }
@@ -408,15 +344,9 @@ public class <ClassName> extends VectorAggregateExpression {
         Aggregation myagg,
         <ValueType>[] vector,
         int batchSize) {
-      if (myagg.isNull) {
-        myagg.isNull = false;
-        myagg.sum = 0;
-        myagg.count = 0;
-      }
 
       for (int i=0;i<batchSize;++i) {
-        <ValueType> value = vector[i];
-        myagg.sum += value;
+        myagg.sum += vector[i];
         myagg.count += 1;
       }
     }
@@ -483,15 +413,11 @@ public class <ClassName> extends VectorAggregateExpression {
 #ENDIF COMPLETE
 
     Aggregation myagg = (Aggregation) agg;
-    if (myagg.isNull) {
-      outputColVector.noNulls = false;
-      outputColVector.isNull[batchIndex] = true;
-      return;
-    }
-    Preconditions.checkState(myagg.count > 0);
-    outputColVector.isNull[batchIndex] = false;
 
 #IF PARTIAL1
+    // For AVG, we do not mark NULL if all inputs were NULL.
+    outputColVector.isNull[batchIndex] = false;
+
     ColumnVector[] fields = outputColVector.fields;
     fields[AVERAGE_COUNT_FIELD_INDEX].isNull[batchIndex] = false;
     ((LongColumnVector) fields[AVERAGE_COUNT_FIELD_INDEX]).vector[batchIndex] = myagg.count;
@@ -506,6 +432,12 @@ public class <ClassName> extends VectorAggregateExpression {
 
 #ENDIF PARTIAL1
 #IF COMPLETE
+    if (myagg.count == 0) {
+      outputColVector.noNulls = false;
+      outputColVector.isNull[batchIndex] = true;
+      return;
+    }
+    outputColVector.isNull[batchIndex] = false;
     outputColVector.vector[batchIndex] = myagg.sum / myagg.count;
 #ENDIF COMPLETE
   }

http://git-wip-us.apache.org/repos/asf/hive/blob/0966a383/ql/src/gen/vectorization/UDAFTemplates/VectorUDAFAvgDecimal.txt
----------------------------------------------------------------------
diff --git a/ql/src/gen/vectorization/UDAFTemplates/VectorUDAFAvgDecimal.txt b/ql/src/gen/vectorization/UDAFTemplates/VectorUDAFAvgDecimal.txt
index f512639..3caeecd 100644
--- a/ql/src/gen/vectorization/UDAFTemplates/VectorUDAFAvgDecimal.txt
+++ b/ql/src/gen/vectorization/UDAFTemplates/VectorUDAFAvgDecimal.txt
@@ -61,23 +61,11 @@ public class <ClassName> extends VectorAggregateExpression {
 
       transient private final HiveDecimalWritable sum = new HiveDecimalWritable();
       transient private long count;
-      transient private boolean isNull;
 
       public void avgValue(HiveDecimalWritable writable) {
-        if (isNull) {
-          // Make a copy since we intend to mutate sum.
-          sum.set(writable);
-          count = 1;
-          isNull = false;
-        } else {
-          // Note that if sum is out of range, mutateAdd will ignore the call.
-          // At the end, sum.isSet() can be checked for null.
-          sum.mutateAdd(writable);
-          count++;
-        }
-      }
 
-      public void avgValueNoNullCheck(HiveDecimalWritable writable) {
+        // Note that if sum is out of range, mutateAdd will ignore the call.
+        // At the end, sum.isSet() can be checked for null.
         sum.mutateAdd(writable);
         count++;
       }
@@ -89,7 +77,6 @@ public class <ClassName> extends VectorAggregateExpression {
 
       @Override
       public void reset() {
-        isNull = true;
         sum.setFromLong(0L);
         count = 0;
       }
@@ -189,15 +176,9 @@ public class <ClassName> extends VectorAggregateExpression {
         }
       } else {
         if (inputVector.isRepeating) {
-          if (batch.selectedInUse) {
-            iterateHasNullsRepeatingSelectionWithAggregationSelection(
-              aggregationBufferSets, bufferIndex,
-              vector[0], batchSize, batch.selected, inputVector.isNull);
-          } else {
-            iterateHasNullsRepeatingWithAggregationSelection(
-              aggregationBufferSets, bufferIndex,
-              vector[0], batchSize, inputVector.isNull);
-          }
+          iterateHasNullsRepeatingWithAggregationSelection(
+            aggregationBufferSets, bufferIndex,
+            vector[0], batchSize, inputVector.isNull);
         } else {
           if (batch.selectedInUse) {
             iterateHasNullsSelectionWithAggregationSelection(
@@ -257,28 +238,6 @@ public class <ClassName> extends VectorAggregateExpression {
       }
     }
 
-    private void iterateHasNullsRepeatingSelectionWithAggregationSelection(
-      VectorAggregationBufferRow[] aggregationBufferSets,
-      int bufferIndex,
-      HiveDecimalWritable value,
-      int batchSize,
-      int[] selection,
-      boolean[] isNull) {
-
-      if (isNull[0]) {
-        return;
-      }
-
-      for (int i=0; i < batchSize; ++i) {
-        Aggregation myagg = getCurrentAggregationBuffer(
-          aggregationBufferSets,
-          bufferIndex,
-          i);
-        myagg.avgValue(value);
-      }
-
-    }
-
     private void iterateHasNullsRepeatingWithAggregationSelection(
       VectorAggregationBufferRow[] aggregationBufferSets,
       int bufferIndex,
@@ -360,11 +319,6 @@ public class <ClassName> extends VectorAggregateExpression {
 
         if (inputVector.isRepeating) {
           if (inputVector.noNulls || !inputVector.isNull[0]) {
-            if (myagg.isNull) {
-              myagg.isNull = false;
-              myagg.sum.setFromLong(0L);
-              myagg.count = 0;
-            }
             HiveDecimal value = vector[0].getHiveDecimal();
             HiveDecimal multiple = value.multiply(HiveDecimal.create(batchSize));
             myagg.sum.mutateAdd(multiple);
@@ -408,14 +362,8 @@ public class <ClassName> extends VectorAggregateExpression {
         int batchSize,
         int[] selected) {
 
-      if (myagg.isNull) {
-        myagg.isNull = false;
-        myagg.sum.setFromLong(0L);
-        myagg.count = 0;
-      }
-
       for (int i=0; i< batchSize; ++i) {
-        myagg.avgValueNoNullCheck(vector[selected[i]]);
+        myagg.avgValue(vector[selected[i]]);
       }
     }
 
@@ -436,14 +384,9 @@ public class <ClassName> extends VectorAggregateExpression {
         Aggregation myagg,
         HiveDecimalWritable[] vector,
         int batchSize) {
-      if (myagg.isNull) {
-        myagg.isNull = false;
-        myagg.sum.setFromLong(0L);
-        myagg.count = 0;
-      }
 
       for (int i=0;i<batchSize;++i) {
-        myagg.avgValueNoNullCheck(vector[i]);
+        myagg.avgValue(vector[i]);
       }
     }
 
@@ -509,12 +452,14 @@ public class <ClassName> extends VectorAggregateExpression {
 #ENDIF COMPLETE
 
     Aggregation myagg = (Aggregation) agg;
-    if (myagg.isNull || !myagg.sum.isSet()) {
+
+    // For AVG, we only mark NULL on actual overflow.
+    if (!myagg.sum.isSet()) {
       outputColVector.noNulls = false;
       outputColVector.isNull[batchIndex] = true;
       return;
     }
-    Preconditions.checkState(myagg.count > 0);
+
     outputColVector.isNull[batchIndex] = false;
 
 #IF PARTIAL1
@@ -532,6 +477,12 @@ public class <ClassName> extends VectorAggregateExpression {
 
 #ENDIF PARTIAL1
 #IF COMPLETE
+    // For AVG, we mark NULL on count 0 or on overflow.
+    if (myagg.count == 0 || !myagg.sum.isSet()) {
+      outputColVector.noNulls = false;
+      outputColVector.isNull[batchIndex] = true;
+      return;
+    }
     tempDecWritable.setFromLong (myagg.count);
     HiveDecimalWritable result = outputColVector.vector[batchIndex];
     result.set(myagg.sum);

http://git-wip-us.apache.org/repos/asf/hive/blob/0966a383/ql/src/gen/vectorization/UDAFTemplates/VectorUDAFAvgDecimal64ToDecimal.txt
----------------------------------------------------------------------
diff --git a/ql/src/gen/vectorization/UDAFTemplates/VectorUDAFAvgDecimal64ToDecimal.txt b/ql/src/gen/vectorization/UDAFTemplates/VectorUDAFAvgDecimal64ToDecimal.txt
index 53dceeb..39e0562 100644
--- a/ql/src/gen/vectorization/UDAFTemplates/VectorUDAFAvgDecimal64ToDecimal.txt
+++ b/ql/src/gen/vectorization/UDAFTemplates/VectorUDAFAvgDecimal64ToDecimal.txt
@@ -81,7 +81,6 @@ public class <ClassName> extends VectorAggregateExpression {
     /**
      * Value is explicitly (re)initialized in reset()
      */
-    private boolean isNull = true;
     private boolean usingRegularDecimal = false;
 
     public Aggregation(int inputScale, HiveDecimalWritable temp) {
@@ -90,26 +89,21 @@ public class <ClassName> extends VectorAggregateExpression {
     }
 
     public void avgValue(long value) {
-      if (isNull) {
-        sum = value;
-        count = 1;
-        isNull = false;
-      } else {
-        if (Math.abs(sum) > nearDecimal64Max) {
-          if (!usingRegularDecimal) {
-            usingRegularDecimal = true;
-            regularDecimalSum.deserialize64(sum, inputScale);
-          } else {
-            temp.deserialize64(sum, inputScale);
-            regularDecimalSum.mutateAdd(temp);
-          }
-          sum = value;
+
+      if (Math.abs(sum) > nearDecimal64Max) {
+        if (!usingRegularDecimal) {
+          usingRegularDecimal = true;
+          regularDecimalSum.deserialize64(sum, inputScale);
         } else {
-          sum += value;
+          temp.deserialize64(sum, inputScale);
+          regularDecimalSum.mutateAdd(temp);
         }
-
-        count++;
+        sum = value;
+      } else {
+        sum += value;
       }
+
+      count++;
     }
 
     @Override
@@ -119,7 +113,6 @@ public class <ClassName> extends VectorAggregateExpression {
 
     @Override
     public void reset () {
-      isNull = true;
       usingRegularDecimal = false;
       sum = 0;
       regularDecimalSum.setFromLong(0);
@@ -202,15 +195,9 @@ public class <ClassName> extends VectorAggregateExpression {
         }
       } else {
         if (inputVector.isRepeating) {
-          if (batch.selectedInUse) {
-            iterateHasNullsRepeatingSelectionWithAggregationSelection(
-              aggregationBufferSets, bufferIndex,
-              vector[0], batchSize, batch.selected, inputVector.isNull);
-          } else {
-            iterateHasNullsRepeatingWithAggregationSelection(
-              aggregationBufferSets, bufferIndex,
-              vector[0], batchSize, inputVector.isNull);
-          }
+          iterateHasNullsRepeatingWithAggregationSelection(
+            aggregationBufferSets, bufferIndex,
+            vector[0], batchSize, inputVector.isNull);
         } else {
           if (batch.selectedInUse) {
             iterateHasNullsSelectionWithAggregationSelection(
@@ -270,28 +257,6 @@ public class <ClassName> extends VectorAggregateExpression {
       }
     }
 
-    private void iterateHasNullsRepeatingSelectionWithAggregationSelection(
-      VectorAggregationBufferRow[] aggregationBufferSets,
-      int bufferIndex,
-      long value,
-      int batchSize,
-      int[] selection,
-      boolean[] isNull) {
-
-      if (isNull[0]) {
-        return;
-      }
-
-      for (int i=0; i < batchSize; ++i) {
-        Aggregation myagg = getCurrentAggregationBuffer(
-          aggregationBufferSets,
-          bufferIndex,
-          i);
-        myagg.avgValue(value);
-      }
-
-    }
-
     private void iterateHasNullsRepeatingWithAggregationSelection(
       VectorAggregationBufferRow[] aggregationBufferSets,
       int bufferIndex,
@@ -502,8 +467,10 @@ public class <ClassName> extends VectorAggregateExpression {
 #ENDIF COMPLETE
 
     Aggregation myagg = (Aggregation) agg;
-    final boolean isNull;
-    if (!myagg.isNull) {
+
+#IF PARTIAL1
+    if (myagg.count > 0) {
+
       if (!myagg.usingRegularDecimal) {
         myagg.regularDecimalSum.deserialize64(myagg.sum, inputScale);
       } else {
@@ -511,19 +478,15 @@ public class <ClassName> extends VectorAggregateExpression {
         myagg.regularDecimalSum.mutateAdd(myagg.temp);
       }
 
-      isNull = !myagg.regularDecimalSum.isSet();
-    } else {
-      isNull = true;
-    }
-    if (isNull) {
-      outputColVector.noNulls = false;
-      outputColVector.isNull[batchIndex] = true;
-      return;
+      // For AVG, we only mark NULL on actual overflow.
+      if (!myagg.regularDecimalSum.isSet()) {
+        outputColVector.noNulls = false;
+        outputColVector.isNull[batchIndex] = true;
+        return;
+      }
     }
-    Preconditions.checkState(myagg.count > 0);
-    outputColVector.isNull[batchIndex] = false;
 
-#IF PARTIAL1
+    outputColVector.isNull[batchIndex] = false;
     ColumnVector[] fields = outputColVector.fields;
     fields[AVERAGE_COUNT_FIELD_INDEX].isNull[batchIndex] = false;
     ((LongColumnVector) fields[AVERAGE_COUNT_FIELD_INDEX]).vector[batchIndex] = myagg.count;
@@ -539,6 +502,27 @@ public class <ClassName> extends VectorAggregateExpression {
 
 #ENDIF PARTIAL1
 #IF COMPLETE
+    final boolean isNull;
+    if (myagg.count > 0) {
+      if (!myagg.usingRegularDecimal) {
+        myagg.regularDecimalSum.deserialize64(myagg.sum, inputScale);
+      } else {
+        myagg.temp.deserialize64(myagg.sum, inputScale);
+        myagg.regularDecimalSum.mutateAdd(myagg.temp);
+      }
+
+      isNull = !myagg.regularDecimalSum.isSet();
+    } else {
+      isNull = true;
+    }
+    if (isNull) {
+      outputColVector.noNulls = false;
+      outputColVector.isNull[batchIndex] = true;
+      return;
+    }
+    Preconditions.checkState(myagg.count > 0);
+    outputColVector.isNull[batchIndex] = false;
+
     temp.setFromLong (myagg.count);
     HiveDecimalWritable result = outputColVector.vector[batchIndex];
     result.set(myagg.regularDecimalSum);

http://git-wip-us.apache.org/repos/asf/hive/blob/0966a383/ql/src/gen/vectorization/UDAFTemplates/VectorUDAFAvgDecimalMerge.txt
----------------------------------------------------------------------
diff --git a/ql/src/gen/vectorization/UDAFTemplates/VectorUDAFAvgDecimalMerge.txt b/ql/src/gen/vectorization/UDAFTemplates/VectorUDAFAvgDecimalMerge.txt
index 5fe9256..3691c05 100644
--- a/ql/src/gen/vectorization/UDAFTemplates/VectorUDAFAvgDecimalMerge.txt
+++ b/ql/src/gen/vectorization/UDAFTemplates/VectorUDAFAvgDecimalMerge.txt
@@ -188,15 +188,9 @@ public class <ClassName> extends VectorAggregateExpression {
         }
       } else {
         if (inputStructColVector.isRepeating) {
-          if (batch.selectedInUse) {
-            iterateHasNullsRepeatingSelectionWithAggregationSelection(
-              aggregationBufferSets, bufferIndex,
-              countVector[0], sumVector[0], batchSize, batch.selected, inputStructColVector.isNull);
-          } else {
-            iterateHasNullsRepeatingWithAggregationSelection(
-              aggregationBufferSets, bufferIndex,
-              countVector[0], sumVector[0], batchSize, inputStructColVector.isNull);
-          }
+          iterateHasNullsRepeatingWithAggregationSelection(
+            aggregationBufferSets, bufferIndex,
+            countVector[0], sumVector[0], batchSize, inputStructColVector.isNull);
         } else {
           if (batch.selectedInUse) {
             iterateHasNullsSelectionWithAggregationSelection(
@@ -260,29 +254,6 @@ public class <ClassName> extends VectorAggregateExpression {
       }
     }
 
-    private void iterateHasNullsRepeatingSelectionWithAggregationSelection(
-      VectorAggregationBufferRow[] aggregationBufferSets,
-      int bufferIndex,
-      long count,
-      HiveDecimalWritable sum,
-      int batchSize,
-      int[] selection,
-      boolean[] isNull) {
-
-      if (isNull[0]) {
-        return;
-      }
-
-      for (int i=0; i < batchSize; ++i) {
-        Aggregation myagg = getCurrentAggregationBuffer(
-          aggregationBufferSets,
-          bufferIndex,
-          i);
-        myagg.merge(count, sum);
-      }
-
-    }
-
     private void iterateHasNullsRepeatingWithAggregationSelection(
       VectorAggregationBufferRow[] aggregationBufferSets,
       int bufferIndex,

http://git-wip-us.apache.org/repos/asf/hive/blob/0966a383/ql/src/gen/vectorization/UDAFTemplates/VectorUDAFAvgMerge.txt
----------------------------------------------------------------------
diff --git a/ql/src/gen/vectorization/UDAFTemplates/VectorUDAFAvgMerge.txt b/ql/src/gen/vectorization/UDAFTemplates/VectorUDAFAvgMerge.txt
index 162d1ba..2e93efd 100644
--- a/ql/src/gen/vectorization/UDAFTemplates/VectorUDAFAvgMerge.txt
+++ b/ql/src/gen/vectorization/UDAFTemplates/VectorUDAFAvgMerge.txt
@@ -154,15 +154,9 @@ public class <ClassName> extends VectorAggregateExpression {
         }
       } else {
         if (inputStructColVector.isRepeating) {
-          if (batch.selectedInUse) {
-            iterateHasNullsRepeatingSelectionWithAggregationSelection(
-              aggregationBufferSets, bufferIndex,
-              countVector[0], sumVector[0], batchSize, batch.selected, inputStructColVector.isNull);
-          } else {
-            iterateHasNullsRepeatingWithAggregationSelection(
-              aggregationBufferSets, bufferIndex,
-              countVector[0], sumVector[0], batchSize, inputStructColVector.isNull);
-          }
+          iterateHasNullsRepeatingWithAggregationSelection(
+            aggregationBufferSets, bufferIndex,
+            countVector[0], sumVector[0], batchSize, inputStructColVector.isNull);
         } else {
           if (batch.selectedInUse) {
             iterateHasNullsSelectionWithAggregationSelection(
@@ -226,29 +220,6 @@ public class <ClassName> extends VectorAggregateExpression {
       }
     }
 
-    private void iterateHasNullsRepeatingSelectionWithAggregationSelection(
-      VectorAggregationBufferRow[] aggregationBufferSets,
-      int bufferIndex,
-      long count,
-      double sum,
-      int batchSize,
-      int[] selection,
-      boolean[] isNull) {
-
-      if (isNull[0]) {
-        return;
-      }
-
-      for (int i=0; i < batchSize; ++i) {
-        Aggregation myagg = getCurrentAggregationBuffer(
-          aggregationBufferSets,
-          bufferIndex,
-          i);
-        myagg.merge(count, sum);
-      }
-
-    }
-
     private void iterateHasNullsRepeatingWithAggregationSelection(
       VectorAggregationBufferRow[] aggregationBufferSets,
       int bufferIndex,

http://git-wip-us.apache.org/repos/asf/hive/blob/0966a383/ql/src/gen/vectorization/UDAFTemplates/VectorUDAFAvgTimestamp.txt
----------------------------------------------------------------------
diff --git a/ql/src/gen/vectorization/UDAFTemplates/VectorUDAFAvgTimestamp.txt b/ql/src/gen/vectorization/UDAFTemplates/VectorUDAFAvgTimestamp.txt
index 810f31f..358d108 100644
--- a/ql/src/gen/vectorization/UDAFTemplates/VectorUDAFAvgTimestamp.txt
+++ b/ql/src/gen/vectorization/UDAFTemplates/VectorUDAFAvgTimestamp.txt
@@ -59,20 +59,9 @@ public class <ClassName> extends VectorAggregateExpression {
       transient private double sum;
       transient private long count;
 
-      /**
-      * Value is explicitly (re)initialized in reset()
-      */
-      transient private boolean isNull = true;
-
-      public void sumValue(double value) {
-        if (isNull) {
-          sum = value;
-          count = 1;
-          isNull = false;
-        } else {
-          sum += value;
-          count++;
-        }
+      public void avgValue(double value) {
+        sum += value;
+        count++;
       }
 
       @Override
@@ -82,7 +71,6 @@ public class <ClassName> extends VectorAggregateExpression {
 
       @Override
       public void reset() {
-        isNull = true;
         sum = 0;
         count = 0L;
       }
@@ -153,15 +141,9 @@ public class <ClassName> extends VectorAggregateExpression {
         }
       } else {
         if (inputColVector.isRepeating) {
-          if (batch.selectedInUse) {
-            iterateHasNullsRepeatingSelectionWithAggregationSelection(
-              aggregationBufferSets, bufferIndex,
-              inputColVector.getDouble(0), batchSize, batch.selected, inputColVector.isNull);
-          } else {
-            iterateHasNullsRepeatingWithAggregationSelection(
-              aggregationBufferSets, bufferIndex,
-              inputColVector.getDouble(0), batchSize, inputColVector.isNull);
-          }
+          iterateHasNullsRepeatingWithAggregationSelection(
+            aggregationBufferSets, bufferIndex,
+            inputColVector.getDouble(0), batchSize, inputColVector.isNull);
         } else {
           if (batch.selectedInUse) {
             iterateHasNullsSelectionWithAggregationSelection(
@@ -187,7 +169,7 @@ public class <ClassName> extends VectorAggregateExpression {
           aggregationBufferSets,
           bufferIndex,
           i);
-        myagg.sumValue(value);
+        myagg.avgValue(value);
       }
     }
 
@@ -203,7 +185,7 @@ public class <ClassName> extends VectorAggregateExpression {
           aggregationBufferSets,
           bufferIndex,
           i);
-        myagg.sumValue(
+        myagg.avgValue(
             inputColVector.getDouble(selection[i]));
       }
     }
@@ -218,45 +200,27 @@ public class <ClassName> extends VectorAggregateExpression {
           aggregationBufferSets,
           bufferIndex,
           i);
-        myagg.sumValue(inputColVector.getDouble(i));
+        myagg.avgValue(inputColVector.getDouble(i));
       }
     }
 
-    private void iterateHasNullsRepeatingSelectionWithAggregationSelection(
+    private void iterateHasNullsRepeatingWithAggregationSelection(
       VectorAggregationBufferRow[] aggregationBufferSets,
       int bufferIndex,
       double value,
       int batchSize,
-      int[] selection,
       boolean[] isNull) {
 
-      for (int i=0; i < batchSize; ++i) {
-        if (!isNull[selection[i]]) {
-          Aggregation myagg = getCurrentAggregationBuffer(
-            aggregationBufferSets,
-            bufferIndex,
-            i);
-          myagg.sumValue(value);
-        }
+      if (isNull[0]) {
+        return;
       }
 
-    }
-
-    private void iterateHasNullsRepeatingWithAggregationSelection(
-      VectorAggregationBufferRow[] aggregationBufferSets,
-      int bufferIndex,
-      double value,
-      int batchSize,
-      boolean[] isNull) {
-
       for (int i=0; i < batchSize; ++i) {
-        if (!isNull[i]) {
-          Aggregation myagg = getCurrentAggregationBuffer(
-            aggregationBufferSets,
-            bufferIndex,
-            i);
-          myagg.sumValue(value);
-        }
+        Aggregation myagg = getCurrentAggregationBuffer(
+          aggregationBufferSets,
+          bufferIndex,
+          i);
+        myagg.avgValue(value);
       }
     }
 
@@ -275,7 +239,7 @@ public class <ClassName> extends VectorAggregateExpression {
             aggregationBufferSets,
             bufferIndex,
             j);
-          myagg.sumValue(inputColVector.getDouble(i));
+          myagg.avgValue(inputColVector.getDouble(i));
         }
       }
    }
@@ -293,7 +257,7 @@ public class <ClassName> extends VectorAggregateExpression {
             aggregationBufferSets,
             bufferIndex,
             i);
-          myagg.sumValue(inputColVector.getDouble(i));
+          myagg.avgValue(inputColVector.getDouble(i));
         }
       }
    }
@@ -318,11 +282,6 @@ public class <ClassName> extends VectorAggregateExpression {
 
         if (inputColVector.isRepeating) {
           if (inputColVector.noNulls || !inputColVector.isNull[0]) {
-            if (myagg.isNull) {
-              myagg.isNull = false;
-              myagg.sum = 0;
-              myagg.count = 0;
-            }
             myagg.sum += inputColVector.getDouble(0)*batchSize;
             myagg.count += batchSize;
           }
@@ -353,13 +312,7 @@ public class <ClassName> extends VectorAggregateExpression {
       for (int j=0; j< batchSize; ++j) {
         int i = selected[j];
         if (!isNull[i]) {
-          double value = inputColVector.getDouble(i);
-          if (myagg.isNull) {
-            myagg.isNull = false;
-            myagg.sum = 0;
-            myagg.count = 0;
-          }
-          myagg.sum += value;
+          myagg.sum += inputColVector.getDouble(i);
           myagg.count += 1;
         }
       }
@@ -371,15 +324,8 @@ public class <ClassName> extends VectorAggregateExpression {
         int batchSize,
         int[] selected) {
 
-      if (myagg.isNull) {
-        myagg.isNull = false;
-        myagg.sum = 0;
-        myagg.count = 0;
-      }
-
       for (int i=0; i< batchSize; ++i) {
-        double value = inputColVector.getDouble(selected[i]);
-        myagg.sum += value;
+        myagg.sum += inputColVector.getDouble(selected[i]);
         myagg.count += 1;
       }
     }
@@ -392,13 +338,7 @@ public class <ClassName> extends VectorAggregateExpression {
 
       for(int i=0;i<batchSize;++i) {
         if (!isNull[i]) {
-          double value = inputColVector.getDouble(i);
-          if (myagg.isNull) {
-            myagg.isNull = false;
-            myagg.sum = 0;
-            myagg.count = 0;
-          }
-          myagg.sum += value;
+          myagg.sum += inputColVector.getDouble(i);
           myagg.count += 1;
         }
       }
@@ -408,15 +348,9 @@ public class <ClassName> extends VectorAggregateExpression {
         Aggregation myagg,
         TimestampColumnVector inputColVector,
         int batchSize) {
-      if (myagg.isNull) {
-        myagg.isNull = false;
-        myagg.sum = 0;
-        myagg.count = 0;
-      }
 
       for (int i=0;i<batchSize;++i) {
-        double value = inputColVector.getDouble(i);
-        myagg.sum += value;
+        myagg.sum += inputColVector.getDouble(i);
         myagg.count += 1;
       }
     }
@@ -475,23 +409,14 @@ public class <ClassName> extends VectorAggregateExpression {
   public void assignRowColumn(VectorizedRowBatch batch, int batchIndex, int columnNum,
       AggregationBuffer agg) throws HiveException {
 
+    Aggregation myagg = (Aggregation) agg;
+
 #IF PARTIAL1
     StructColumnVector outputColVector = (StructColumnVector) batch.cols[columnNum];
-#ENDIF PARTIAL1
-#IF COMPLETE
-    DoubleColumnVector outputColVector = (DoubleColumnVector) batch.cols[columnNum];
-#ENDIF COMPLETE
 
-    Aggregation myagg = (Aggregation) agg;
-    if (myagg.isNull) {
-      outputColVector.noNulls = false;
-      outputColVector.isNull[batchIndex] = true;
-      return;
-    }
-    Preconditions.checkState(myagg.count > 0);
+    // For AVG, we do not mark NULL if all inputs were NULL.
     outputColVector.isNull[batchIndex] = false;
 
-#IF PARTIAL1
     ColumnVector[] fields = outputColVector.fields;
     fields[AVERAGE_COUNT_FIELD_INDEX].isNull[batchIndex] = false;
     ((LongColumnVector) fields[AVERAGE_COUNT_FIELD_INDEX]).vector[batchIndex] = myagg.count;
@@ -506,6 +431,15 @@ public class <ClassName> extends VectorAggregateExpression {
 
 #ENDIF PARTIAL1
 #IF COMPLETE
+    DoubleColumnVector outputColVector = (DoubleColumnVector) batch.cols[columnNum];
+
+    if (myagg.count == 0) {
+      outputColVector.noNulls = false;
+      outputColVector.isNull[batchIndex] = true;
+      return;
+    }
+    outputColVector.isNull[batchIndex] = false;
+
     outputColVector.vector[batchIndex] = myagg.sum / myagg.count;
 #ENDIF COMPLETE
   }

http://git-wip-us.apache.org/repos/asf/hive/blob/0966a383/ql/src/gen/vectorization/UDAFTemplates/VectorUDAFMinMax.txt
----------------------------------------------------------------------
diff --git a/ql/src/gen/vectorization/UDAFTemplates/VectorUDAFMinMax.txt b/ql/src/gen/vectorization/UDAFTemplates/VectorUDAFMinMax.txt
index 2df45bb..3569d51 100644
--- a/ql/src/gen/vectorization/UDAFTemplates/VectorUDAFMinMax.txt
+++ b/ql/src/gen/vectorization/UDAFTemplates/VectorUDAFMinMax.txt
@@ -145,15 +145,9 @@ public class <ClassName> extends VectorAggregateExpression {
         }
       } else {
         if (inputVector.isRepeating) {
-          if (batch.selectedInUse) {
-            iterateHasNullsRepeatingSelectionWithAggregationSelection(
-              aggregationBufferSets, aggregrateIndex,
-              vector[0], batchSize, batch.selected, inputVector.isNull);
-          } else {
-            iterateHasNullsRepeatingWithAggregationSelection(
-              aggregationBufferSets, aggregrateIndex,
-              vector[0], batchSize, inputVector.isNull);
-          }
+          iterateHasNullsRepeatingWithAggregationSelection(
+            aggregationBufferSets, aggregrateIndex,
+            vector[0], batchSize, inputVector.isNull);
         } else {
           if (batch.selectedInUse) {
             iterateHasNullsSelectionWithAggregationSelection(
@@ -213,28 +207,6 @@ public class <ClassName> extends VectorAggregateExpression {
       }
     }
 
-    private void iterateHasNullsRepeatingSelectionWithAggregationSelection(
-      VectorAggregationBufferRow[] aggregationBufferSets,
-      int aggregrateIndex,
-      <ValueType> value,
-      int batchSize,
-      int[] selection,
-      boolean[] isNull) {
-
-      if (isNull[0]) {
-        return;
-      }
-
-      for (int i=0; i < batchSize; ++i) {
-        Aggregation myagg = getCurrentAggregationBuffer(
-          aggregationBufferSets,
-          aggregrateIndex,
-          i);
-        myagg.minmaxValue(value);
-      }
-
-    }
-
     private void iterateHasNullsRepeatingWithAggregationSelection(
       VectorAggregationBufferRow[] aggregationBufferSets,
       int aggregrateIndex,
@@ -363,7 +335,7 @@ public class <ClassName> extends VectorAggregateExpression {
 
       for (int i=0; i< batchSize; ++i) {
         <ValueType> value = vector[selected[i]];
-        myagg.minmaxValueNoCheck(value);
+        myagg.minmaxValue(value);
       }
     }
 
@@ -437,7 +409,7 @@ public class <ClassName> extends VectorAggregateExpression {
 
     <InputColumnVectorType> outputColVector = (<InputColumnVectorType>) batch.cols[columnNum];
     Aggregation myagg = (Aggregation) agg;
-       if (myagg.isNull) {
+    if (myagg.isNull) {
       outputColVector.noNulls = false;
       outputColVector.isNull[batchIndex] = true;
       return;

http://git-wip-us.apache.org/repos/asf/hive/blob/0966a383/ql/src/gen/vectorization/UDAFTemplates/VectorUDAFMinMaxDecimal.txt
----------------------------------------------------------------------
diff --git a/ql/src/gen/vectorization/UDAFTemplates/VectorUDAFMinMaxDecimal.txt b/ql/src/gen/vectorization/UDAFTemplates/VectorUDAFMinMaxDecimal.txt
index 9c8ebcc..eb63301 100644
--- a/ql/src/gen/vectorization/UDAFTemplates/VectorUDAFMinMaxDecimal.txt
+++ b/ql/src/gen/vectorization/UDAFTemplates/VectorUDAFMinMaxDecimal.txt
@@ -63,7 +63,7 @@ public class <ClassName> extends VectorAggregateExpression {
         value = new HiveDecimalWritable();
       }
 
-      public void checkValue(HiveDecimalWritable writable, short scale) {
+      public void minmaxValue(HiveDecimalWritable writable, short scale) {
         if (isNull) {
           isNull = false;
           this.value.set(writable);
@@ -144,15 +144,9 @@ public class <ClassName> extends VectorAggregateExpression {
         }
       } else {
         if (inputVector.isRepeating) {
-          if (batch.selectedInUse) {
-            iterateHasNullsRepeatingSelectionWithAggregationSelection(
-              aggregationBufferSets, aggregrateIndex,
-              vector[0], inputVector.scale, batchSize, batch.selected, inputVector.isNull);
-          } else {
-            iterateHasNullsRepeatingWithAggregationSelection(
-              aggregationBufferSets, aggregrateIndex,
-              vector[0], inputVector.scale, batchSize, inputVector.isNull);
-          }
+          iterateHasNullsRepeatingWithAggregationSelection(
+            aggregationBufferSets, aggregrateIndex,
+            vector[0], inputVector.scale, batchSize, inputVector.isNull);
         } else {
           if (batch.selectedInUse) {
             iterateHasNullsSelectionWithAggregationSelection(
@@ -179,14 +173,14 @@ public class <ClassName> extends VectorAggregateExpression {
           aggregationBufferSets,
           aggregrateIndex,
           i);
-        myagg.checkValue(value, scale);
+        myagg.minmaxValue(value, scale);
       }
     }
 
     private void iterateNoNullsSelectionWithAggregationSelection(
       VectorAggregationBufferRow[] aggregationBufferSets,
       int aggregrateIndex,
-      HiveDecimalWritable[] values,
+      HiveDecimalWritable[] vector,
       short scale,
       int[] selection,
       int batchSize) {
@@ -196,14 +190,14 @@ public class <ClassName> extends VectorAggregateExpression {
           aggregationBufferSets,
           aggregrateIndex,
           i);
-        myagg.checkValue(values[selection[i]], scale);
+        myagg.minmaxValue(vector[selection[i]], scale);
       }
     }
 
     private void iterateNoNullsWithAggregationSelection(
       VectorAggregationBufferRow[] aggregationBufferSets,
       int aggregrateIndex,
-      HiveDecimalWritable[] values,
+      HiveDecimalWritable[] vector,
       short scale,
       int batchSize) {
       for (int i=0; i < batchSize; ++i) {
@@ -211,31 +205,10 @@ public class <ClassName> extends VectorAggregateExpression {
           aggregationBufferSets,
           aggregrateIndex,
           i);
-        myagg.checkValue(values[i], scale);
+        myagg.minmaxValue(vector[i], scale);
       }
     }
 
-    private void iterateHasNullsRepeatingSelectionWithAggregationSelection(
-      VectorAggregationBufferRow[] aggregationBufferSets,
-      int aggregrateIndex,
-      HiveDecimalWritable value,
-      short scale,
-      int batchSize,
-      int[] selection,
-      boolean[] isNull) {
-
-      for (int i=0; i < batchSize; ++i) {
-        if (!isNull[selection[i]]) {
-          Aggregation myagg = getCurrentAggregationBuffer(
-            aggregationBufferSets,
-            aggregrateIndex,
-            i);
-          myagg.checkValue(value, scale);
-        }
-      }
-
-    }
-
     private void iterateHasNullsRepeatingWithAggregationSelection(
       VectorAggregationBufferRow[] aggregationBufferSets,
       int aggregrateIndex,
@@ -253,14 +226,14 @@ public class <ClassName> extends VectorAggregateExpression {
           aggregationBufferSets,
           aggregrateIndex,
           i);
-        myagg.checkValue(value, scale);
+        myagg.minmaxValue(value, scale);
       }
     }
 
     private void iterateHasNullsSelectionWithAggregationSelection(
       VectorAggregationBufferRow[] aggregationBufferSets,
       int aggregrateIndex,
-      HiveDecimalWritable[] values,
+      HiveDecimalWritable[] vector,
       short scale,
       int batchSize,
       int[] selection,
@@ -273,7 +246,7 @@ public class <ClassName> extends VectorAggregateExpression {
             aggregationBufferSets,
             aggregrateIndex,
             j);
-          myagg.checkValue(values[i], scale);
+          myagg.minmaxValue(vector[i], scale);
         }
       }
    }
@@ -281,7 +254,7 @@ public class <ClassName> extends VectorAggregateExpression {
     private void iterateHasNullsWithAggregationSelection(
       VectorAggregationBufferRow[] aggregationBufferSets,
       int aggregrateIndex,
-      HiveDecimalWritable[] values,
+      HiveDecimalWritable[] vector,
       short scale,
       int batchSize,
       boolean[] isNull) {
@@ -292,7 +265,7 @@ public class <ClassName> extends VectorAggregateExpression {
             aggregationBufferSets,
             aggregrateIndex,
             i);
-          myagg.checkValue(values[i], scale);
+          myagg.minmaxValue(vector[i], scale);
         }
       }
    }
@@ -318,10 +291,8 @@ public class <ClassName> extends VectorAggregateExpression {
         HiveDecimalWritable[] vector = inputVector.vector;
 
         if (inputVector.isRepeating) {
-          if ((inputVector.noNulls || !inputVector.isNull[0]) &&
-            (myagg.isNull || (myagg.value.compareTo(vector[0]) <OperatorSymbol> 0))) {
-            myagg.isNull = false;
-            myagg.value.set(vector[0]);
+          if (inputVector.noNulls || !inputVector.isNull[0]) {
+            myagg.minmaxValue(vector[0], inputVector.scale);
           }
           return;
         }
@@ -353,14 +324,7 @@ public class <ClassName> extends VectorAggregateExpression {
       for (int j=0; j< batchSize; ++j) {
         int i = selected[j];
         if (!isNull[i]) {
-          HiveDecimalWritable writable = vector[i];
-          if (myagg.isNull) {
-            myagg.isNull = false;
-            myagg.value.set(writable);
-          }
-          else if (myagg.value.compareTo(writable) <OperatorSymbol> 0) {
-            myagg.value.set(writable);
-          }
+          myagg.minmaxValue(vector[i], scale);
         }
       }
     }
@@ -372,16 +336,8 @@ public class <ClassName> extends VectorAggregateExpression {
         int batchSize,
         int[] selected) {
 
-      if (myagg.isNull) {
-        myagg.value.set(vector[selected[0]]);
-        myagg.isNull = false;
-      }
-
       for (int i=0; i< batchSize; ++i) {
-        HiveDecimalWritable writable = vector[selected[i]];
-        if (myagg.value.compareTo(writable) <OperatorSymbol> 0) {
-          myagg.value.set(writable);
-        }
+        myagg.minmaxValue(vector[selected[i]], scale);
       }
     }
 
@@ -394,14 +350,7 @@ public class <ClassName> extends VectorAggregateExpression {
 
       for(int i=0;i<batchSize;++i) {
         if (!isNull[i]) {
-          HiveDecimalWritable writable = vector[i];
-          if (myagg.isNull) {
-            myagg.value.set(writable);
-            myagg.isNull = false;
-          }
-          else if (myagg.value.compareTo(writable) <OperatorSymbol> 0) {
-            myagg.value.set(writable);
-          }
+          myagg.minmaxValue(vector[i], scale);
         }
       }
     }
@@ -411,16 +360,9 @@ public class <ClassName> extends VectorAggregateExpression {
         HiveDecimalWritable[] vector,
         short scale,
         int batchSize) {
-      if (myagg.isNull) {
-        myagg.value.set(vector[0]);
-        myagg.isNull = false;
-      }
 
       for (int i=0;i<batchSize;++i) {
-        HiveDecimalWritable writable = vector[i];
-        if (myagg.value.compareTo(writable) <OperatorSymbol> 0) {
-          myagg.value.set(writable);
-        }
+        myagg.minmaxValue(vector[i], scale);
       }
     }
 

http://git-wip-us.apache.org/repos/asf/hive/blob/0966a383/ql/src/gen/vectorization/UDAFTemplates/VectorUDAFMinMaxIntervalDayTime.txt
----------------------------------------------------------------------
diff --git a/ql/src/gen/vectorization/UDAFTemplates/VectorUDAFMinMaxIntervalDayTime.txt b/ql/src/gen/vectorization/UDAFTemplates/VectorUDAFMinMaxIntervalDayTime.txt
index 9a0a6e7..9fdf77c 100644
--- a/ql/src/gen/vectorization/UDAFTemplates/VectorUDAFMinMaxIntervalDayTime.txt
+++ b/ql/src/gen/vectorization/UDAFTemplates/VectorUDAFMinMaxIntervalDayTime.txt
@@ -62,7 +62,7 @@ public class <ClassName> extends VectorAggregateExpression {
         value = new HiveIntervalDayTime();
       }
 
-      public void checkValue(IntervalDayTimeColumnVector colVector, int index) {
+      public void minmaxValue(IntervalDayTimeColumnVector colVector, int index) {
         if (isNull) {
           isNull = false;
           colVector.intervalDayTimeUpdate(this.value, index);
@@ -141,15 +141,9 @@ public class <ClassName> extends VectorAggregateExpression {
         }
       } else {
         if (inputColVector.isRepeating) {
-          if (batch.selectedInUse) {
-            iterateHasNullsRepeatingSelectionWithAggregationSelection(
-              aggregationBufferSets, aggregrateIndex,
-              inputColVector, batchSize, batch.selected, inputColVector.isNull);
-          } else {
-            iterateHasNullsRepeatingWithAggregationSelection(
-              aggregationBufferSets, aggregrateIndex,
-              inputColVector, batchSize, inputColVector.isNull);
-          }
+          iterateHasNullsRepeatingWithAggregationSelection(
+            aggregationBufferSets, aggregrateIndex,
+            inputColVector, batchSize, inputColVector.isNull);
         } else {
           if (batch.selectedInUse) {
             iterateHasNullsSelectionWithAggregationSelection(
@@ -176,7 +170,7 @@ public class <ClassName> extends VectorAggregateExpression {
           aggregrateIndex,
           i);
         // Repeating use index 0.
-        myagg.checkValue(inputColVector, 0);
+        myagg.minmaxValue(inputColVector, 0);
       }
     }
 
@@ -192,7 +186,7 @@ public class <ClassName> extends VectorAggregateExpression {
           aggregationBufferSets,
           aggregrateIndex,
           i);
-        myagg.checkValue(inputColVector, selection[i]);
+        myagg.minmaxValue(inputColVector, selection[i]);
       }
     }
 
@@ -206,47 +200,28 @@ public class <ClassName> extends VectorAggregateExpression {
           aggregationBufferSets,
           aggregrateIndex,
           i);
-        myagg.checkValue(inputColVector, i);
+        myagg.minmaxValue(inputColVector, i);
       }
     }
 
-    private void iterateHasNullsRepeatingSelectionWithAggregationSelection(
+    private void iterateHasNullsRepeatingWithAggregationSelection(
       VectorAggregationBufferRow[] aggregationBufferSets,
       int aggregrateIndex,
       IntervalDayTimeColumnVector inputColVector,
       int batchSize,
-      int[] selection,
       boolean[] isNull) {
 
-      for (int i=0; i < batchSize; ++i) {
-        if (!isNull[selection[i]]) {
-          Aggregation myagg = getCurrentAggregationBuffer(
-            aggregationBufferSets,
-            aggregrateIndex,
-            i);
-          // Repeating use index 0.
-          myagg.checkValue(inputColVector, 0);
-        }
+      if (isNull[0]) {
+        return;
       }
 
-    }
-
-    private void iterateHasNullsRepeatingWithAggregationSelection(
-      VectorAggregationBufferRow[] aggregationBufferSets,
-      int aggregrateIndex,
-      IntervalDayTimeColumnVector inputColVector,
-      int batchSize,
-      boolean[] isNull) {
-
       for (int i=0; i < batchSize; ++i) {
-        if (!isNull[i]) {
-          Aggregation myagg = getCurrentAggregationBuffer(
-            aggregationBufferSets,
-            aggregrateIndex,
-            i);
-          // Repeating use index 0.
-          myagg.checkValue(inputColVector, 0);
-        }
+        Aggregation myagg = getCurrentAggregationBuffer(
+          aggregationBufferSets,
+          aggregrateIndex,
+          i);
+        // Repeating use index 0.
+        myagg.minmaxValue(inputColVector, 0);
       }
     }
 
@@ -265,7 +240,7 @@ public class <ClassName> extends VectorAggregateExpression {
             aggregationBufferSets,
             aggregrateIndex,
             j);
-          myagg.checkValue(inputColVector, i);
+          myagg.minmaxValue(inputColVector, i);
         }
       }
    }
@@ -283,7 +258,7 @@ public class <ClassName> extends VectorAggregateExpression {
             aggregationBufferSets,
             aggregrateIndex,
             i);
-          myagg.checkValue(inputColVector, i);
+          myagg.minmaxValue(inputColVector, i);
         }
       }
    }
@@ -307,10 +282,8 @@ public class <ClassName> extends VectorAggregateExpression {
         Aggregation myagg = (Aggregation)agg;
 
         if (inputColVector.isRepeating) {
-          if ((inputColVector.noNulls || !inputColVector.isNull[0]) &&
-            (myagg.isNull || (inputColVector.compareTo(myagg.value, 0) <OperatorSymbol> 0))) {
-            myagg.isNull = false;
-            inputColVector.intervalDayTimeUpdate(myagg.value, 0);
+          if (inputColVector.noNulls || !inputColVector.isNull[0]) {
+            myagg.minmaxValue(inputColVector, 0);
           }
           return;
         }
@@ -341,13 +314,7 @@ public class <ClassName> extends VectorAggregateExpression {
       for (int j=0; j< batchSize; ++j) {
         int i = selected[j];
         if (!isNull[i]) {
-          if (myagg.isNull) {
-            myagg.isNull = false;
-            inputColVector.intervalDayTimeUpdate(myagg.value, i);
-          }
-          else if (inputColVector.compareTo(myagg.value, i) <OperatorSymbol> 0) {
-            inputColVector.intervalDayTimeUpdate(myagg.value, i);
-          }
+          myagg.minmaxValue(inputColVector, i);
         }
       }
     }
@@ -358,16 +325,9 @@ public class <ClassName> extends VectorAggregateExpression {
         int batchSize,
         int[] selected) {
 
-      if (myagg.isNull) {
-        inputColVector.intervalDayTimeUpdate(myagg.value, selected[0]);
-        myagg.isNull = false;
-      }
-
       for (int i=0; i< batchSize; ++i) {
         int sel = selected[i];
-        if (inputColVector.compareTo(myagg.value, sel) <OperatorSymbol> 0) {
-          inputColVector.intervalDayTimeUpdate(myagg.value, sel);
-        }
+        myagg.minmaxValue(inputColVector, sel);
       }
     }
 
@@ -379,13 +339,7 @@ public class <ClassName> extends VectorAggregateExpression {
 
       for(int i=0;i<batchSize;++i) {
         if (!isNull[i]) {
-          if (myagg.isNull) {
-            inputColVector.intervalDayTimeUpdate(myagg.value, i);
-            myagg.isNull = false;
-          }
-          else if (inputColVector.compareTo(myagg.value, i) <OperatorSymbol> 0) {
-            inputColVector.intervalDayTimeUpdate(myagg.value, i);
-          }
+          myagg.minmaxValue(inputColVector, i);
         }
       }
     }
@@ -394,15 +348,9 @@ public class <ClassName> extends VectorAggregateExpression {
         Aggregation myagg,
         IntervalDayTimeColumnVector inputColVector,
         int batchSize) {
-      if (myagg.isNull) {
-        inputColVector.intervalDayTimeUpdate(myagg.value, 0);
-        myagg.isNull = false;
-      }
 
       for (int i=0;i<batchSize;++i) {
-        if (inputColVector.compareTo(myagg.value, i) <OperatorSymbol> 0) {
-          inputColVector.intervalDayTimeUpdate(myagg.value, i);
-        }
+        myagg.minmaxValue(inputColVector, i);
       }
     }
 
@@ -447,7 +395,7 @@ public class <ClassName> extends VectorAggregateExpression {
 
     IntervalDayTimeColumnVector outputColVector = (IntervalDayTimeColumnVector) batch.cols[columnNum];
     Aggregation myagg = (Aggregation) agg;
-       if (myagg.isNull) {
+    if (myagg.isNull) {
       outputColVector.noNulls = false;
       outputColVector.isNull[batchIndex] = true;
       return;

http://git-wip-us.apache.org/repos/asf/hive/blob/0966a383/ql/src/gen/vectorization/UDAFTemplates/VectorUDAFMinMaxString.txt
----------------------------------------------------------------------
diff --git a/ql/src/gen/vectorization/UDAFTemplates/VectorUDAFMinMaxString.txt b/ql/src/gen/vectorization/UDAFTemplates/VectorUDAFMinMaxString.txt
index 4f0b5a5..3387c0d 100644
--- a/ql/src/gen/vectorization/UDAFTemplates/VectorUDAFMinMaxString.txt
+++ b/ql/src/gen/vectorization/UDAFTemplates/VectorUDAFMinMaxString.txt
@@ -60,7 +60,7 @@ public class <ClassName> extends VectorAggregateExpression {
       */
       transient private boolean isNull = true;
 
-      public void checkValue(byte[] bytes, int start, int length) {
+      public void minmaxValue(byte[] bytes, int start, int length) {
         if (isNull) {
           isNull = false;
           assign(bytes, start, length);
@@ -151,7 +151,9 @@ public class <ClassName> extends VectorAggregateExpression {
         }
       } else {
         if (inputColumn.isRepeating) {
-          // All nulls, no-op for min/max
+          iterateHasNullsRepeatingWithAggregationSelection(
+            aggregationBufferSets, aggregrateIndex,
+            inputColumn, batchSize, inputColumn.isNull);
         } else {
           if (batch.selectedInUse) {
             iterateHasNullsSelectionWithAggregationSelection(
@@ -180,7 +182,7 @@ public class <ClassName> extends VectorAggregateExpression {
           aggregationBufferSets,
           aggregrateIndex,
           i);
-        myagg.checkValue(bytes, start, length);
+        myagg.minmaxValue(bytes, start, length);
       }
     }
 
@@ -197,7 +199,7 @@ public class <ClassName> extends VectorAggregateExpression {
           aggregationBufferSets,
           aggregrateIndex,
           i);
-        myagg.checkValue(inputColumn.vector[row],
+        myagg.minmaxValue(inputColumn.vector[row],
           inputColumn.start[row],
           inputColumn.length[row]);
       }
@@ -213,12 +215,36 @@ public class <ClassName> extends VectorAggregateExpression {
           aggregationBufferSets,
           aggregrateIndex,
           i);
-        myagg.checkValue(inputColumn.vector[i],
+        myagg.minmaxValue(inputColumn.vector[i],
           inputColumn.start[i],
           inputColumn.length[i]);
       }
     }
 
+    private void iterateHasNullsRepeatingWithAggregationSelection(
+      VectorAggregationBufferRow[] aggregationBufferSets,
+      int aggregrateIndex,
+      BytesColumnVector inputColumn,
+      int batchSize,
+      boolean[] isNull) {
+
+      if (isNull[0]) {
+        return;
+      }
+
+      for (int i=0; i < batchSize; ++i) {
+        Aggregation myagg = getCurrentAggregationBuffer(
+          aggregationBufferSets,
+          aggregrateIndex,
+          i);
+        // Repeating use index 0.
+        myagg.minmaxValue(inputColumn.vector[0],
+            inputColumn.start[0],
+            inputColumn.length[0]);
+      }
+
+    }
+
     private void iterateHasNullsSelectionWithAggregationSelection(
       VectorAggregationBufferRow[] aggregationBufferSets,
       int aggregrateIndex,
@@ -233,7 +259,7 @@ public class <ClassName> extends VectorAggregateExpression {
             aggregationBufferSets,
             aggregrateIndex,
             i);
-          myagg.checkValue(inputColumn.vector[row],
+          myagg.minmaxValue(inputColumn.vector[row],
             inputColumn.start[row],
             inputColumn.length[row]);
         }
@@ -252,7 +278,7 @@ public class <ClassName> extends VectorAggregateExpression {
             aggregationBufferSets,
             aggregrateIndex,
             i);
-          myagg.checkValue(inputColumn.vector[i],
+          myagg.minmaxValue(inputColumn.vector[i],
             inputColumn.start[i],
             inputColumn.length[i]);
         }
@@ -279,7 +305,7 @@ public class <ClassName> extends VectorAggregateExpression {
 
         if (inputColumn.isRepeating) {
           if (inputColumn.noNulls || !inputColumn.isNull[0]) {
-            myagg.checkValue(inputColumn.vector[0],
+            myagg.minmaxValue(inputColumn.vector[0],
               inputColumn.start[0],
               inputColumn.length[0]);
           }
@@ -309,7 +335,7 @@ public class <ClassName> extends VectorAggregateExpression {
       for (int j=0; j< batchSize; ++j) {
         int i = selected[j];
         if (!inputColumn.isNull[i]) {
-          myagg.checkValue(inputColumn.vector[i],
+          myagg.minmaxValue(inputColumn.vector[i],
             inputColumn.start[i],
             inputColumn.length[i]);
         }
@@ -324,7 +350,7 @@ public class <ClassName> extends VectorAggregateExpression {
 
       for (int j=0; j< batchSize; ++j) {
         int i = selected[j];
-        myagg.checkValue(inputColumn.vector[i],
+        myagg.minmaxValue(inputColumn.vector[i],
           inputColumn.start[i],
           inputColumn.length[i]);
       }
@@ -337,7 +363,7 @@ public class <ClassName> extends VectorAggregateExpression {
 
       for (int i=0; i< batchSize; ++i) {
         if (!inputColumn.isNull[i]) {
-          myagg.checkValue(inputColumn.vector[i],
+          myagg.minmaxValue(inputColumn.vector[i],
             inputColumn.start[i],
             inputColumn.length[i]);
         }
@@ -349,7 +375,7 @@ public class <ClassName> extends VectorAggregateExpression {
         BytesColumnVector inputColumn,
         int batchSize) {
       for (int i=0; i< batchSize; ++i) {
-        myagg.checkValue(inputColumn.vector[i],
+        myagg.minmaxValue(inputColumn.vector[i],
           inputColumn.start[i],
           inputColumn.length[i]);
       }

http://git-wip-us.apache.org/repos/asf/hive/blob/0966a383/ql/src/gen/vectorization/UDAFTemplates/VectorUDAFMinMaxTimestamp.txt
----------------------------------------------------------------------
diff --git a/ql/src/gen/vectorization/UDAFTemplates/VectorUDAFMinMaxTimestamp.txt b/ql/src/gen/vectorization/UDAFTemplates/VectorUDAFMinMaxTimestamp.txt
index 5114cda..b8d71d6 100644
--- a/ql/src/gen/vectorization/UDAFTemplates/VectorUDAFMinMaxTimestamp.txt
+++ b/ql/src/gen/vectorization/UDAFTemplates/VectorUDAFMinMaxTimestamp.txt
@@ -64,7 +64,7 @@ public class <ClassName> extends VectorAggregateExpression {
         value = new Timestamp(0);
       }
 
-      public void checkValue(TimestampColumnVector colVector, int index) {
+      public void minmaxValue(TimestampColumnVector colVector, int index) {
         if (isNull) {
           isNull = false;
           colVector.timestampUpdate(this.value, index);
@@ -143,15 +143,9 @@ public class <ClassName> extends VectorAggregateExpression {
         }
       } else {
         if (inputColVector.isRepeating) {
-          if (batch.selectedInUse) {
-            iterateHasNullsRepeatingSelectionWithAggregationSelection(
-              aggregationBufferSets, aggregrateIndex,
-              inputColVector, batchSize, batch.selected, inputColVector.isNull);
-          } else {
-            iterateHasNullsRepeatingWithAggregationSelection(
-              aggregationBufferSets, aggregrateIndex,
-              inputColVector, batchSize, inputColVector.isNull);
-          }
+          iterateHasNullsRepeatingWithAggregationSelection(
+            aggregationBufferSets, aggregrateIndex,
+            inputColVector, batchSize, inputColVector.isNull);
         } else {
           if (batch.selectedInUse) {
             iterateHasNullsSelectionWithAggregationSelection(
@@ -178,7 +172,7 @@ public class <ClassName> extends VectorAggregateExpression {
           aggregrateIndex,
           i);
         // Repeating use index 0.
-        myagg.checkValue(inputColVector, 0);
+        myagg.minmaxValue(inputColVector, 0);
       }
     }
 
@@ -194,7 +188,7 @@ public class <ClassName> extends VectorAggregateExpression {
           aggregationBufferSets,
           aggregrateIndex,
           i);
-        myagg.checkValue(inputColVector, selection[i]);
+        myagg.minmaxValue(inputColVector, selection[i]);
       }
     }
 
@@ -208,47 +202,28 @@ public class <ClassName> extends VectorAggregateExpression {
           aggregationBufferSets,
           aggregrateIndex,
           i);
-        myagg.checkValue(inputColVector, i);
+        myagg.minmaxValue(inputColVector, i);
       }
     }
 
-    private void iterateHasNullsRepeatingSelectionWithAggregationSelection(
+    private void iterateHasNullsRepeatingWithAggregationSelection(
       VectorAggregationBufferRow[] aggregationBufferSets,
       int aggregrateIndex,
       TimestampColumnVector inputColVector,
       int batchSize,
-      int[] selection,
       boolean[] isNull) {
 
-      for (int i=0; i < batchSize; ++i) {
-        if (!isNull[selection[i]]) {
-          Aggregation myagg = getCurrentAggregationBuffer(
-            aggregationBufferSets,
-            aggregrateIndex,
-            i);
-          // Repeating use index 0.
-          myagg.checkValue(inputColVector, 0);
-        }
+      if (isNull[0]) {
+        return;
       }
 
-    }
-
-    private void iterateHasNullsRepeatingWithAggregationSelection(
-      VectorAggregationBufferRow[] aggregationBufferSets,
-      int aggregrateIndex,
-      TimestampColumnVector inputColVector,
-      int batchSize,
-      boolean[] isNull) {
-
       for (int i=0; i < batchSize; ++i) {
-        if (!isNull[i]) {
-          Aggregation myagg = getCurrentAggregationBuffer(
-            aggregationBufferSets,
-            aggregrateIndex,
-            i);
-          // Repeating use index 0.
-          myagg.checkValue(inputColVector, 0);
-        }
+        Aggregation myagg = getCurrentAggregationBuffer(
+          aggregationBufferSets,
+          aggregrateIndex,
+          i);
+        // Repeating use index 0.
+        myagg.minmaxValue(inputColVector, 0);
       }
     }
 
@@ -267,7 +242,7 @@ public class <ClassName> extends VectorAggregateExpression {
             aggregationBufferSets,
             aggregrateIndex,
             j);
-          myagg.checkValue(inputColVector, i);
+          myagg.minmaxValue(inputColVector, i);
         }
       }
    }
@@ -285,7 +260,7 @@ public class <ClassName> extends VectorAggregateExpression {
             aggregationBufferSets,
             aggregrateIndex,
             i);
-          myagg.checkValue(inputColVector, i);
+          myagg.minmaxValue(inputColVector, i);
         }
       }
    }
@@ -309,10 +284,8 @@ public class <ClassName> extends VectorAggregateExpression {
         Aggregation myagg = (Aggregation)agg;
 
         if (inputColVector.isRepeating) {
-          if ((inputColVector.noNulls || !inputColVector.isNull[0]) &&
-            (myagg.isNull || (inputColVector.compareTo(myagg.value, 0) <OperatorSymbol> 0))) {
-            myagg.isNull = false;
-            inputColVector.timestampUpdate(myagg.value, 0);
+          if (inputColVector.noNulls || !inputColVector.isNull[0]) {
+            myagg.minmaxValue(inputColVector, 0);
           }
           return;
         }
@@ -343,13 +316,7 @@ public class <ClassName> extends VectorAggregateExpression {
       for (int j=0; j< batchSize; ++j) {
         int i = selected[j];
         if (!isNull[i]) {
-          if (myagg.isNull) {
-            myagg.isNull = false;
-            inputColVector.timestampUpdate(myagg.value, i);
-          }
-          else if (inputColVector.compareTo(myagg.value, i) <OperatorSymbol> 0) {
-            inputColVector.timestampUpdate(myagg.value, i);
-          }
+          myagg.minmaxValue(inputColVector, i);
         }
       }
     }
@@ -360,16 +327,9 @@ public class <ClassName> extends VectorAggregateExpression {
         int batchSize,
         int[] selected) {
 
-      if (myagg.isNull) {
-        inputColVector.timestampUpdate(myagg.value, selected[0]);
-        myagg.isNull = false;
-      }
-
-      for (int i=0; i< batchSize; ++i) {
+     for (int i=0; i< batchSize; ++i) {
         int sel = selected[i];
-        if (inputColVector.compareTo(myagg.value, sel) <OperatorSymbol> 0) {
-          inputColVector.timestampUpdate(myagg.value, sel);
-        }
+        myagg.minmaxValue(inputColVector, sel);
       }
     }
 
@@ -381,13 +341,7 @@ public class <ClassName> extends VectorAggregateExpression {
 
       for(int i=0;i<batchSize;++i) {
         if (!isNull[i]) {
-          if (myagg.isNull) {
-            inputColVector.timestampUpdate(myagg.value, i);
-            myagg.isNull = false;
-          }
-          else if (inputColVector.compareTo(myagg.value, i) <OperatorSymbol> 0) {
-            inputColVector.timestampUpdate(myagg.value, i);
-          }
+          myagg.minmaxValue(inputColVector, i);
         }
       }
     }
@@ -396,15 +350,9 @@ public class <ClassName> extends VectorAggregateExpression {
         Aggregation myagg,
         TimestampColumnVector inputColVector,
         int batchSize) {
-      if (myagg.isNull) {
-        inputColVector.timestampUpdate(myagg.value, 0);
-        myagg.isNull = false;
-      }
 
       for (int i=0;i<batchSize;++i) {
-        if (inputColVector.compareTo(myagg.value, i) <OperatorSymbol> 0) {
-          inputColVector.timestampUpdate(myagg.value, i);
-        }
+        myagg.minmaxValue(inputColVector, i);
       }
     }
 
@@ -449,7 +397,7 @@ public class <ClassName> extends VectorAggregateExpression {
 
     TimestampColumnVector outputColVector = (TimestampColumnVector) batch.cols[columnNum];
     Aggregation myagg = (Aggregation) agg;
-       if (myagg.isNull) {
+    if (myagg.isNull) {
       outputColVector.noNulls = false;
       outputColVector.isNull[batchIndex] = true;
       return;

http://git-wip-us.apache.org/repos/asf/hive/blob/0966a383/ql/src/gen/vectorization/UDAFTemplates/VectorUDAFSum.txt
----------------------------------------------------------------------
diff --git a/ql/src/gen/vectorization/UDAFTemplates/VectorUDAFSum.txt b/ql/src/gen/vectorization/UDAFTemplates/VectorUDAFSum.txt
index c731869..548125e 100644
--- a/ql/src/gen/vectorization/UDAFTemplates/VectorUDAFSum.txt
+++ b/ql/src/gen/vectorization/UDAFTemplates/VectorUDAFSum.txt
@@ -142,15 +142,9 @@ public class <ClassName> extends VectorAggregateExpression {
         }
       } else {
         if (inputVector.isRepeating) {
-          if (batch.selectedInUse) {
-            iterateHasNullsRepeatingSelectionWithAggregationSelection(
-              aggregationBufferSets, aggregateIndex,
-              vector[0], batchSize, batch.selected, inputVector.isNull);
-          } else {
-            iterateHasNullsRepeatingWithAggregationSelection(
-              aggregationBufferSets, aggregateIndex,
-              vector[0], batchSize, inputVector.isNull);
-          }
+          iterateHasNullsRepeatingWithAggregationSelection(
+            aggregationBufferSets, aggregateIndex,
+            vector[0], batchSize, inputVector.isNull);
         } else {
           if (batch.selectedInUse) {
             iterateHasNullsSelectionWithAggregationSelection(
@@ -210,28 +204,6 @@ public class <ClassName> extends VectorAggregateExpression {
       }
     }
 
-    private void iterateHasNullsRepeatingSelectionWithAggregationSelection(
-      VectorAggregationBufferRow[] aggregationBufferSets,
-      int aggregateIndex,
-      <ValueType> value,
-      int batchSize,
-      int[] selection,
-      boolean[] isNull) {
-
-      if (isNull[0]) {
-        return;
-      }
-
-      for (int i=0; i < batchSize; ++i) {
-        Aggregation myagg = getCurrentAggregationBuffer(
-          aggregationBufferSets,
-          aggregateIndex,
-          i);
-        myagg.sumValue(value);
-      }
-
-    }
-
     private void iterateHasNullsRepeatingWithAggregationSelection(
       VectorAggregationBufferRow[] aggregationBufferSets,
       int aggregateIndex,

http://git-wip-us.apache.org/repos/asf/hive/blob/0966a383/ql/src/gen/vectorization/UDAFTemplates/VectorUDAFVar.txt
----------------------------------------------------------------------
diff --git a/ql/src/gen/vectorization/UDAFTemplates/VectorUDAFVar.txt b/ql/src/gen/vectorization/UDAFTemplates/VectorUDAFVar.txt
index 876ead5..995190f 100644
--- a/ql/src/gen/vectorization/UDAFTemplates/VectorUDAFVar.txt
+++ b/ql/src/gen/vectorization/UDAFTemplates/VectorUDAFVar.txt
@@ -127,7 +127,7 @@ public class <ClassName> extends VectorAggregateExpression {
 
     private void init() {
 #IF COMPLETE
-      String aggregateName = vecAggrDesc.getAggrDesc().getGenericUDAFName();
+      String aggregateName = vecAggrDesc.getAggregationName();
       varianceKind = VarianceKind.nameMap.get(aggregateName);
 #ENDIF COMPLETE
     }
@@ -490,11 +490,8 @@ public class <ClassName> extends VectorAggregateExpression {
     StructColumnVector outputColVector = (StructColumnVector) batch.cols[columnNum];
 
     Aggregation myagg = (Aggregation) agg;
-    if (myagg.isNull) {
-      outputColVector.noNulls = false;
-      outputColVector.isNull[batchIndex] = true;
-      return;
-    }
+
+    // For Variance Family, we do not mark NULL if all inputs were NULL.
     outputColVector.isNull[batchIndex] = false;
 
     ColumnVector[] fields = outputColVector.fields;

http://git-wip-us.apache.org/repos/asf/hive/blob/0966a383/ql/src/gen/vectorization/UDAFTemplates/VectorUDAFVarDecimal.txt
----------------------------------------------------------------------
diff --git a/ql/src/gen/vectorization/UDAFTemplates/VectorUDAFVarDecimal.txt b/ql/src/gen/vectorization/UDAFTemplates/VectorUDAFVarDecimal.txt
index cf19b14..a831610 100644
--- a/ql/src/gen/vectorization/UDAFTemplates/VectorUDAFVarDecimal.txt
+++ b/ql/src/gen/vectorization/UDAFTemplates/VectorUDAFVarDecimal.txt
@@ -138,7 +138,7 @@ public class <ClassName> extends VectorAggregateExpression {
 
     private void init() {
 #IF COMPLETE
-      String aggregateName = vecAggrDesc.getAggrDesc().getGenericUDAFName();
+      String aggregateName = vecAggrDesc.getAggregationName();
       varianceKind = VarianceKind.nameMap.get(aggregateName);
 #ENDIF COMPLETE
     }
@@ -450,15 +450,12 @@ public class <ClassName> extends VectorAggregateExpression {
   public void assignRowColumn(VectorizedRowBatch batch, int batchIndex, int columnNum,
       AggregationBuffer agg) throws HiveException {
 
+    Aggregation myagg = (Aggregation) agg;
+
 #IF PARTIAL1
     StructColumnVector outputColVector = (StructColumnVector) batch.cols[columnNum];
 
-    Aggregation myagg = (Aggregation) agg;
-    if (myagg.isNull) {
-      outputColVector.noNulls = false;
-      outputColVector.isNull[batchIndex] = true;
-      return;
-    }
+    // For Variance Family, we do not mark NULL if all inputs were NULL.
     outputColVector.isNull[batchIndex] = false;
 
     ColumnVector[] fields = outputColVector.fields;
@@ -469,7 +466,13 @@ public class <ClassName> extends VectorAggregateExpression {
 #IF COMPLETE
     DoubleColumnVector outputColVector = (DoubleColumnVector) batch.cols[columnNum];
 
-    Aggregation myagg = (Aggregation) agg;
+    if (myagg.isNull) {
+      outputColVector.noNulls = false;
+      outputColVector.isNull[batchIndex] = true;
+      return;
+    }
+    outputColVector.isNull[batchIndex] = false;
+
     if (GenericUDAFVariance.isVarianceNull(myagg.count, varianceKind)) {
 
       // SQL standard - return null for zero (or 1 for sample) elements

http://git-wip-us.apache.org/repos/asf/hive/blob/0966a383/ql/src/gen/vectorization/UDAFTemplates/VectorUDAFVarMerge.txt
----------------------------------------------------------------------
diff --git a/ql/src/gen/vectorization/UDAFTemplates/VectorUDAFVarMerge.txt b/ql/src/gen/vectorization/UDAFTemplates/VectorUDAFVarMerge.txt
index ccc5a22..dfc73a1 100644
--- a/ql/src/gen/vectorization/UDAFTemplates/VectorUDAFVarMerge.txt
+++ b/ql/src/gen/vectorization/UDAFTemplates/VectorUDAFVarMerge.txt
@@ -60,27 +60,21 @@ public class <ClassName> extends VectorAggregateExpression {
       transient private double mergeSum;
       transient private double mergeVariance;
 
-      /**
-      * Value is explicitly (re)initialized in reset()
-      */
-      transient private boolean isNull = true;
-
       public void merge(long partialCount, double partialSum, double partialVariance) {
 
-        if (isNull || mergeCount == 0) {
+        if (mergeCount == 0) {
 
           // Just copy the information since there is nothing so far.
           mergeCount = partialCount;
           mergeSum = partialSum;
           mergeVariance = partialVariance;
-          isNull = false;
           return;
         }
 
         if (partialCount > 0 && mergeCount > 0) {
 
           // Merge the two partials.
-          mergeVariance +=
+          mergeVariance =
               GenericUDAFVariance.calculateMerge(
                   partialCount, mergeCount, partialSum, mergeSum,
                   partialVariance, mergeVariance);
@@ -98,7 +92,6 @@ public class <ClassName> extends VectorAggregateExpression {
 
       @Override
       public void reset () {
-        isNull = true;
         mergeCount = 0L;
         mergeSum = 0;
         mergeVariance = 0;
@@ -127,7 +120,7 @@ public class <ClassName> extends VectorAggregateExpression {
 
     private void init() {
 #IF FINAL
-      String aggregateName = vecAggrDesc.getAggrDesc().getGenericUDAFName();
+      String aggregateName = vecAggrDesc.getAggregationName();
       varianceKind = VarianceKind.nameMap.get(aggregateName);
 #ENDIF FINAL
     }
@@ -183,15 +176,9 @@ public class <ClassName> extends VectorAggregateExpression {
         }
       } else {
         if (inputStructColVector.isRepeating) {
-          if (batch.selectedInUse) {
-            iterateHasNullsRepeatingSelectionWithAggregationSelection(
-              aggregationBufferSets, bufferIndex,
-              countVector[0], sumVector[0], varianceVector[0], batchSize, batch.selected, inputStructColVector.isNull);
-          } else {
-            iterateHasNullsRepeatingWithAggregationSelection(
-              aggregationBufferSets, bufferIndex,
-              countVector[0], sumVector[0], varianceVector[0], batchSize, inputStructColVector.isNull);
-          }
+          iterateHasNullsRepeatingWithAggregationSelection(
+            aggregationBufferSets, bufferIndex,
+            countVector[0], sumVector[0], varianceVector[0], batchSize, inputStructColVector.isNull);
         } else {
           if (batch.selectedInUse) {
             iterateHasNullsSelectionWithAggregationSelection(
@@ -258,30 +245,6 @@ public class <ClassName> extends VectorAggregateExpression {
       }
     }
 
-    private void iterateHasNullsRepeatingSelectionWithAggregationSelection(
-      VectorAggregationBufferRow[] aggregationBufferSets,
-      int bufferIndex,
-      long count,
-      double sum,
-      double variance,
-      int batchSize,
-      int[] selection,
-      boolean[] isNull) {
-
-      if (isNull[0]) {
-        return;
-      }
-
-      for (int i=0; i < batchSize; ++i) {
-        Aggregation myagg = getCurrentAggregationBuffer(
-          aggregationBufferSets,
-          bufferIndex,
-          i);
-        myagg.merge(count, sum, variance);
-      }
-
-    }
-
     private void iterateHasNullsRepeatingWithAggregationSelection(
       VectorAggregationBufferRow[] aggregationBufferSets,
       int bufferIndex,
@@ -488,8 +451,6 @@ public class <ClassName> extends VectorAggregateExpression {
 #ENDIF FINAL
      */
 
-/*
-    There seems to be a Wrong Results bug in VectorUDAFVarFinal -- disabling vectorization for now...
     return
         GenericUDAFVariance.isVarianceFamilyName(name) &&
         inputColVectorType == ColumnVector.Type.STRUCT &&
@@ -501,8 +462,6 @@ public class <ClassName> extends VectorAggregateExpression {
         outputColVectorType == ColumnVector.Type.DOUBLE &&
         mode == Mode.FINAL;
 #ENDIF FINAL
-*/
-    return false;
   }
 
   @Override
@@ -513,11 +472,8 @@ public class <ClassName> extends VectorAggregateExpression {
     StructColumnVector outputColVector = (StructColumnVector) batch.cols[columnNum];
 
     Aggregation myagg = (Aggregation) agg;
-    if (myagg.isNull) {
-      outputColVector.noNulls = false;
-      outputColVector.isNull[batchIndex] = true;
-      return;
-    }
+
+    // For Variance Family, we do not mark NULL if all inputs were NULL.
     outputColVector.isNull[batchIndex] = false;
 
     ColumnVector[] fields = outputColVector.fields;

http://git-wip-us.apache.org/repos/asf/hive/blob/0966a383/ql/src/gen/vectorization/UDAFTemplates/VectorUDAFVarTimestamp.txt
----------------------------------------------------------------------
diff --git a/ql/src/gen/vectorization/UDAFTemplates/VectorUDAFVarTimestamp.txt b/ql/src/gen/vectorization/UDAFTemplates/VectorUDAFVarTimestamp.txt
index 1dd5ab4..4e79f22 100644
--- a/ql/src/gen/vectorization/UDAFTemplates/VectorUDAFVarTimestamp.txt
+++ b/ql/src/gen/vectorization/UDAFTemplates/VectorUDAFVarTimestamp.txt
@@ -128,7 +128,7 @@ public class <ClassName> extends VectorAggregateExpression {
 
     private void init() {
 #IF COMPLETE
-      String aggregateName = vecAggrDesc.getAggrDesc().getGenericUDAFName();
+      String aggregateName = vecAggrDesc.getAggregationName();
       varianceKind = VarianceKind.nameMap.get(aggregateName);
 #ENDIF COMPLETE
     }
@@ -422,15 +422,12 @@ public class <ClassName> extends VectorAggregateExpression {
   public void assignRowColumn(VectorizedRowBatch batch, int batchIndex, int columnNum,
       AggregationBuffer agg) throws HiveException {
 
+   Aggregation myagg = (Aggregation) agg;
+
 #IF PARTIAL1
     StructColumnVector outputColVector = (StructColumnVector) batch.cols[columnNum];
 
-    Aggregation myagg = (Aggregation) agg;
-    if (myagg.isNull) {
-      outputColVector.noNulls = false;
-      outputColVector.isNull[batchIndex] = true;
-      return;
-    }
+    // For Variance Family, we do not mark NULL if all inputs were NULL.
     outputColVector.isNull[batchIndex] = false;
 
     ColumnVector[] fields = outputColVector.fields;
@@ -441,7 +438,13 @@ public class <ClassName> extends VectorAggregateExpression {
 #IF COMPLETE
     DoubleColumnVector outputColVector = (DoubleColumnVector) batch.cols[columnNum];
 
-    Aggregation myagg = (Aggregation) agg;
+    if (myagg.isNull) {
+      outputColVector.noNulls = false;
+      outputColVector.isNull[batchIndex] = true;
+      return;
+    }
+    outputColVector.isNull[batchIndex] = false;
+
     if (GenericUDAFVariance.isVarianceNull(myagg.count, varianceKind)) {
 
       // SQL standard - return null for zero (or 1 for sample) elements

http://git-wip-us.apache.org/repos/asf/hive/blob/0966a383/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/VectorAggregationDesc.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/VectorAggregationDesc.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/VectorAggregationDesc.java
index 5736399..417beec 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/VectorAggregationDesc.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/VectorAggregationDesc.java
@@ -86,7 +86,7 @@ public class VectorAggregationDesc implements java.io.Serializable {
 
   private static final long serialVersionUID = 1L;
 
-  private final AggregationDesc aggrDesc;
+  private final String aggregationName;
 
   private final TypeInfo inputTypeInfo;
   private final ColumnVector.Type inputColVectorType;
@@ -99,15 +99,19 @@ public class VectorAggregationDesc implements java.io.Serializable {
   private final Class<? extends VectorAggregateExpression> vecAggrClass;
 
   private GenericUDAFEvaluator evaluator;
+  private GenericUDAFEvaluator.Mode udafEvaluatorMode;
 
-  public VectorAggregationDesc(AggregationDesc aggrDesc, GenericUDAFEvaluator evaluator,
+  public VectorAggregationDesc(String aggregationName, GenericUDAFEvaluator evaluator,
+      GenericUDAFEvaluator.Mode udafEvaluatorMode,
       TypeInfo inputTypeInfo, ColumnVector.Type inputColVectorType,
       VectorExpression inputExpression, TypeInfo outputTypeInfo,
       ColumnVector.Type outputColVectorType,
       Class<? extends VectorAggregateExpression> vecAggrClass) {
 
-    this.aggrDesc = aggrDesc;
+    this.aggregationName = aggregationName;
+
     this.evaluator = evaluator;
+    this.udafEvaluatorMode = udafEvaluatorMode;
 
     this.inputTypeInfo = inputTypeInfo;
     this.inputColVectorType = inputColVectorType;
@@ -122,8 +126,12 @@ public class VectorAggregationDesc implements java.io.Serializable {
     this.vecAggrClass = vecAggrClass;
   }
 
-  public AggregationDesc getAggrDesc() {
-    return aggrDesc;
+  public String getAggregationName() {
+    return aggregationName;
+  }
+
+  public GenericUDAFEvaluator.Mode getUdafEvaluatorMode() {
+    return udafEvaluatorMode;
   }
 
   public TypeInfo getInputTypeInfo() {
@@ -174,7 +182,6 @@ public class VectorAggregationDesc implements java.io.Serializable {
       sb.append("/");
       sb.append(outputDataTypePhysicalVariation);
     }
-    String aggregationName = aggrDesc.getGenericUDAFName();
     if (GenericUDAFVariance.isVarianceFamilyName(aggregationName)) {
       sb.append(" aggregation: ");
       sb.append(aggregationName);

http://git-wip-us.apache.org/repos/asf/hive/blob/0966a383/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/aggregates/VectorAggregateExpression.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/aggregates/VectorAggregateExpression.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/aggregates/VectorAggregateExpression.java
index 3224557..2499f09 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/aggregates/VectorAggregateExpression.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/aggregates/VectorAggregateExpression.java
@@ -84,7 +84,7 @@ public abstract class VectorAggregateExpression  implements Serializable {
     outputTypeInfo =  vecAggrDesc.getOutputTypeInfo();
     outputDataTypePhysicalVariation = vecAggrDesc.getOutputDataTypePhysicalVariation();
 
-    mode = vecAggrDesc.getAggrDesc().getMode();
+    mode = vecAggrDesc.getUdafEvaluatorMode();
   }
 
   public VectorExpression getInputExpression() {

http://git-wip-us.apache.org/repos/asf/hive/blob/0966a383/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/aggregates/VectorUDAFCountMerge.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/aggregates/VectorUDAFCountMerge.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/aggregates/VectorUDAFCountMerge.java
index 0463de5..bd781af 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/aggregates/VectorUDAFCountMerge.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/aggregates/VectorUDAFCountMerge.java
@@ -117,15 +117,9 @@ public class VectorUDAFCountMerge extends VectorAggregateExpression {
       }
     } else {
       if (inputVector.isRepeating) {
-        if (batch.selectedInUse) {
-          iterateHasNullsRepeatingSelectionWithAggregationSelection(
-              aggregationBufferSets, aggregateIndex,
-              vector[0], batchSize, batch.selected, inputVector.isNull);
-        } else {
-          iterateHasNullsRepeatingWithAggregationSelection(
-              aggregationBufferSets, aggregateIndex,
-              vector[0], batchSize, inputVector.isNull);
-        }
+        iterateHasNullsRepeatingWithAggregationSelection(
+            aggregationBufferSets, aggregateIndex,
+            vector[0], batchSize, inputVector.isNull);
       } else {
         if (batch.selectedInUse) {
           iterateHasNullsSelectionWithAggregationSelection(
@@ -185,28 +179,6 @@ public class VectorUDAFCountMerge extends VectorAggregateExpression {
     }
   }
 
-  private void iterateHasNullsRepeatingSelectionWithAggregationSelection(
-      VectorAggregationBufferRow[] aggregationBufferSets,
-      int aggregateIndex,
-      long value,
-      int batchSize,
-      int[] selection,
-      boolean[] isNull) {
-
-    if (isNull[0]) {
-      return;
-    }
-
-    for (int i=0; i < batchSize; ++i) {
-      Aggregation myagg = getCurrentAggregationBuffer(
-          aggregationBufferSets,
-          aggregateIndex,
-          i);
-      myagg.value += value;
-    }
-    
-  }
-
   private void iterateHasNullsRepeatingWithAggregationSelection(
       VectorAggregationBufferRow[] aggregationBufferSets,
       int aggregateIndex,

http://git-wip-us.apache.org/repos/asf/hive/blob/0966a383/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/aggregates/VectorUDAFSumDecimal.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/aggregates/VectorUDAFSumDecimal.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/aggregates/VectorUDAFSumDecimal.java
index 315b72b..469f610 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/aggregates/VectorUDAFSumDecimal.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/aggregates/VectorUDAFSumDecimal.java
@@ -139,17 +139,10 @@ public class VectorUDAFSumDecimal extends VectorAggregateExpression {
       }
     } else {
       if (inputVector.isRepeating) {
-        if (batch.selectedInUse) {
-          iterateHasNullsRepeatingSelectionWithAggregationSelection(
-            aggregationBufferSets, aggregateIndex,
-            vector[0],
-            batchSize, batch.selected, inputVector.isNull);
-        } else {
-          iterateHasNullsRepeatingWithAggregationSelection(
-            aggregationBufferSets, aggregateIndex,
-            vector[0],
-            batchSize, inputVector.isNull);
-        }
+        iterateHasNullsRepeatingWithAggregationSelection(
+          aggregationBufferSets, aggregateIndex,
+          vector[0],
+          batchSize, inputVector.isNull);
       } else {
         if (batch.selectedInUse) {
           iterateHasNullsSelectionWithAggregationSelection(
@@ -211,28 +204,6 @@ public class VectorUDAFSumDecimal extends VectorAggregateExpression {
     }
   }
 
-  private void iterateHasNullsRepeatingSelectionWithAggregationSelection(
-    VectorAggregationBufferRow[] aggregationBufferSets,
-    int aggregateIndex,
-    HiveDecimalWritable value,
-    int batchSize,
-    int[] selection,
-    boolean[] isNull) {
-
-    if (isNull[0]) {
-      return;
-    }
-
-    for (int i=0; i < batchSize; ++i) {
-      Aggregation myagg = getCurrentAggregationBuffer(
-        aggregationBufferSets,
-        aggregateIndex,
-        i);
-      myagg.sumValue(value);
-    }
-
-  }
-
   private void iterateHasNullsRepeatingWithAggregationSelection(
     VectorAggregationBufferRow[] aggregationBufferSets,
     int aggregateIndex,