HIVE-20321: Vectorization: Cut down memory size of 1 col VectorHashKeyWrapper 
to <1 CacheLine (Matt McCline, reviewed by Gopal Vijayaraghavan)


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

Branch: refs/heads/master
Commit: ccdcc5e2eb39211ff3a5510bd7866eb5f5df7eb4
Parents: 59cf159
Author: Matt McCline <mmccl...@hortonworks.com>
Authored: Fri Aug 17 08:08:48 2018 -0500
Committer: Matt McCline <mmccl...@hortonworks.com>
Committed: Fri Aug 17 08:08:48 2018 -0500

----------------------------------------------------------------------
 .../ql/exec/persistence/HashMapWrapper.java     |    6 +-
 .../persistence/HybridHashTableContainer.java   |    6 +-
 .../persistence/MapJoinBytesTableContainer.java |    6 +-
 .../hive/ql/exec/persistence/MapJoinKey.java    |    6 +-
 .../ql/exec/persistence/MapJoinKeyObject.java   |    6 +-
 .../exec/persistence/MapJoinTableContainer.java |    6 +-
 .../ql/exec/vector/VectorColumnSetInfo.java     |   20 +-
 .../ql/exec/vector/VectorGroupByOperator.java   |   24 +-
 .../ql/exec/vector/VectorHashKeyWrapper.java    |  682 -----------
 .../exec/vector/VectorHashKeyWrapperBatch.java  | 1067 -----------------
 .../ql/exec/vector/VectorMapJoinOperator.java   |    4 +-
 .../exec/vector/VectorSMBMapJoinOperator.java   |    8 +-
 .../wrapper/VectorHashKeyWrapperBase.java       |  223 ++++
 .../wrapper/VectorHashKeyWrapperBatch.java      | 1076 ++++++++++++++++++
 .../wrapper/VectorHashKeyWrapperEmpty.java      |   81 ++
 .../wrapper/VectorHashKeyWrapperFactory.java    |   55 +
 .../wrapper/VectorHashKeyWrapperGeneral.java    |  649 +++++++++++
 .../wrapper/VectorHashKeyWrapperSingleBase.java |   53 +
 .../wrapper/VectorHashKeyWrapperSingleLong.java |  131 +++
 .../wrapper/VectorHashKeyWrapperTwoBase.java    |   63 +
 .../wrapper/VectorHashKeyWrapperTwoLong.java    |  170 +++
 .../vector/TestVectorHashKeyWrapperBatch.java   |    6 +-
 22 files changed, 2554 insertions(+), 1794 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hive/blob/ccdcc5e2/ql/src/java/org/apache/hadoop/hive/ql/exec/persistence/HashMapWrapper.java
----------------------------------------------------------------------
diff --git 
a/ql/src/java/org/apache/hadoop/hive/ql/exec/persistence/HashMapWrapper.java 
b/ql/src/java/org/apache/hadoop/hive/ql/exec/persistence/HashMapWrapper.java
index 9d35805..765a647 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/persistence/HashMapWrapper.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/persistence/HashMapWrapper.java
@@ -32,9 +32,9 @@ import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hive.conf.HiveConf;
 import org.apache.hadoop.hive.ql.exec.ExprNodeEvaluator;
 import org.apache.hadoop.hive.ql.exec.JoinUtil;
-import org.apache.hadoop.hive.ql.exec.vector.VectorHashKeyWrapper;
-import org.apache.hadoop.hive.ql.exec.vector.VectorHashKeyWrapperBatch;
 import 
org.apache.hadoop.hive.ql.exec.vector.expressions.VectorExpressionWriter;
+import org.apache.hadoop.hive.ql.exec.vector.wrapper.VectorHashKeyWrapperBase;
+import org.apache.hadoop.hive.ql.exec.vector.wrapper.VectorHashKeyWrapperBatch;
 import org.apache.hadoop.hive.ql.metadata.HiveException;
 import org.apache.hadoop.hive.serde2.SerDeException;
 import org.apache.hadoop.hive.serde2.ByteStream.Output;
@@ -163,7 +163,7 @@ public class HashMapWrapper extends 
AbstractMapJoinTableContainer implements Ser
     }
 
     @Override
-    public JoinUtil.JoinResult setFromVector(VectorHashKeyWrapper kw,
+    public JoinUtil.JoinResult setFromVector(VectorHashKeyWrapperBase kw,
         VectorExpressionWriter[] keyOutputWriters, VectorHashKeyWrapperBatch 
keyWrapperBatch)
         throws HiveException {
       if (currentKey == null) {

http://git-wip-us.apache.org/repos/asf/hive/blob/ccdcc5e2/ql/src/java/org/apache/hadoop/hive/ql/exec/persistence/HybridHashTableContainer.java
----------------------------------------------------------------------
diff --git 
a/ql/src/java/org/apache/hadoop/hive/ql/exec/persistence/HybridHashTableContainer.java
 
b/ql/src/java/org/apache/hadoop/hive/ql/exec/persistence/HybridHashTableContainer.java
index 027e39a..13f1702 100644
--- 
a/ql/src/java/org/apache/hadoop/hive/ql/exec/persistence/HybridHashTableContainer.java
+++ 
b/ql/src/java/org/apache/hadoop/hive/ql/exec/persistence/HybridHashTableContainer.java
@@ -39,10 +39,10 @@ import org.apache.hadoop.hive.ql.exec.JoinUtil;
 import org.apache.hadoop.hive.ql.exec.JoinUtil.JoinResult;
 import org.apache.hadoop.hive.ql.exec.SerializationUtilities;
 import 
org.apache.hadoop.hive.ql.exec.persistence.MapJoinBytesTableContainer.KeyValueHelper;
-import org.apache.hadoop.hive.ql.exec.vector.VectorHashKeyWrapper;
-import org.apache.hadoop.hive.ql.exec.vector.VectorHashKeyWrapperBatch;
 import 
org.apache.hadoop.hive.ql.exec.vector.expressions.VectorExpressionWriter;
 import 
org.apache.hadoop.hive.ql.exec.vector.rowbytescontainer.VectorRowBytesContainer;
+import org.apache.hadoop.hive.ql.exec.vector.wrapper.VectorHashKeyWrapperBase;
+import org.apache.hadoop.hive.ql.exec.vector.wrapper.VectorHashKeyWrapperBatch;
 import org.apache.hadoop.hive.ql.io.HiveKey;
 import org.apache.hadoop.hive.ql.metadata.HiveException;
 import org.apache.hadoop.hive.ql.metadata.HiveUtils;
@@ -812,7 +812,7 @@ public class HybridHashTableContainer
     }
 
     @Override
-    public JoinUtil.JoinResult setFromVector(VectorHashKeyWrapper kw,
+    public JoinUtil.JoinResult setFromVector(VectorHashKeyWrapperBase kw,
         VectorExpressionWriter[] keyOutputWriters, VectorHashKeyWrapperBatch 
keyWrapperBatch)
         throws HiveException {
       if (nulls == null) {

http://git-wip-us.apache.org/repos/asf/hive/blob/ccdcc5e2/ql/src/java/org/apache/hadoop/hive/ql/exec/persistence/MapJoinBytesTableContainer.java
----------------------------------------------------------------------
diff --git 
a/ql/src/java/org/apache/hadoop/hive/ql/exec/persistence/MapJoinBytesTableContainer.java
 
b/ql/src/java/org/apache/hadoop/hive/ql/exec/persistence/MapJoinBytesTableContainer.java
index 033bbdb..b632e1d 100644
--- 
a/ql/src/java/org/apache/hadoop/hive/ql/exec/persistence/MapJoinBytesTableContainer.java
+++ 
b/ql/src/java/org/apache/hadoop/hive/ql/exec/persistence/MapJoinBytesTableContainer.java
@@ -30,9 +30,9 @@ import org.apache.hadoop.hive.common.MemoryEstimate;
 import org.apache.hadoop.hive.conf.HiveConf;
 import org.apache.hadoop.hive.ql.exec.ExprNodeEvaluator;
 import org.apache.hadoop.hive.ql.exec.JoinUtil;
-import org.apache.hadoop.hive.ql.exec.vector.VectorHashKeyWrapper;
-import org.apache.hadoop.hive.ql.exec.vector.VectorHashKeyWrapperBatch;
 import 
org.apache.hadoop.hive.ql.exec.vector.expressions.VectorExpressionWriter;
+import org.apache.hadoop.hive.ql.exec.vector.wrapper.VectorHashKeyWrapperBase;
+import org.apache.hadoop.hive.ql.exec.vector.wrapper.VectorHashKeyWrapperBatch;
 import org.apache.hadoop.hive.ql.metadata.HiveException;
 import org.apache.hadoop.hive.ql.util.JavaDataModel;
 import org.apache.hadoop.hive.serde2.ByteStream.Output;
@@ -519,7 +519,7 @@ public class MapJoinBytesTableContainer
     }
 
     @Override
-    public JoinUtil.JoinResult setFromVector(VectorHashKeyWrapper kw,
+    public JoinUtil.JoinResult setFromVector(VectorHashKeyWrapperBase kw,
         VectorExpressionWriter[] keyOutputWriters, VectorHashKeyWrapperBatch 
keyWrapperBatch)
         throws HiveException {
       if (nulls == null) {

http://git-wip-us.apache.org/repos/asf/hive/blob/ccdcc5e2/ql/src/java/org/apache/hadoop/hive/ql/exec/persistence/MapJoinKey.java
----------------------------------------------------------------------
diff --git 
a/ql/src/java/org/apache/hadoop/hive/ql/exec/persistence/MapJoinKey.java 
b/ql/src/java/org/apache/hadoop/hive/ql/exec/persistence/MapJoinKey.java
index 6504a5f..2e3716c 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/persistence/MapJoinKey.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/persistence/MapJoinKey.java
@@ -24,9 +24,9 @@ import java.util.ArrayList;
 import java.util.HashSet;
 import java.util.List;
 
-import org.apache.hadoop.hive.ql.exec.vector.VectorHashKeyWrapper;
-import org.apache.hadoop.hive.ql.exec.vector.VectorHashKeyWrapperBatch;
 import 
org.apache.hadoop.hive.ql.exec.vector.expressions.VectorExpressionWriter;
+import org.apache.hadoop.hive.ql.exec.vector.wrapper.VectorHashKeyWrapperBase;
+import org.apache.hadoop.hive.ql.exec.vector.wrapper.VectorHashKeyWrapperBatch;
 import org.apache.hadoop.hive.ql.metadata.HiveException;
 import org.apache.hadoop.hive.serde2.ByteStream.Output;
 import org.apache.hadoop.hive.serde2.AbstractSerDe;
@@ -118,7 +118,7 @@ public abstract class MapJoinKey {
    * Serializes row to output for vectorized path.
    * @param byteStream Output to reuse. Can be null, in that case a new one 
would be created.
    */
-  public static Output serializeVector(Output byteStream, VectorHashKeyWrapper 
kw,
+  public static Output serializeVector(Output byteStream, 
VectorHashKeyWrapperBase kw,
       VectorExpressionWriter[] keyOutputWriters, VectorHashKeyWrapperBatch 
keyWrapperBatch,
       boolean[] nulls, boolean[] sortableSortOrders, byte[] nullMarkers, 
byte[] notNullMarkers)
               throws HiveException, SerDeException {

http://git-wip-us.apache.org/repos/asf/hive/blob/ccdcc5e2/ql/src/java/org/apache/hadoop/hive/ql/exec/persistence/MapJoinKeyObject.java
----------------------------------------------------------------------
diff --git 
a/ql/src/java/org/apache/hadoop/hive/ql/exec/persistence/MapJoinKeyObject.java 
b/ql/src/java/org/apache/hadoop/hive/ql/exec/persistence/MapJoinKeyObject.java
index 5c750a3..555ccdf 100644
--- 
a/ql/src/java/org/apache/hadoop/hive/ql/exec/persistence/MapJoinKeyObject.java
+++ 
b/ql/src/java/org/apache/hadoop/hive/ql/exec/persistence/MapJoinKeyObject.java
@@ -25,10 +25,10 @@ import java.util.Arrays;
 import java.util.List;
 
 import org.apache.hadoop.hive.ql.exec.ExprNodeEvaluator;
-import org.apache.hadoop.hive.ql.exec.vector.VectorHashKeyWrapper;
-import org.apache.hadoop.hive.ql.exec.vector.VectorHashKeyWrapperBatch;
 import org.apache.hadoop.hive.ql.exec.vector.expressions.VectorExpression;
 import 
org.apache.hadoop.hive.ql.exec.vector.expressions.VectorExpressionWriter;
+import org.apache.hadoop.hive.ql.exec.vector.wrapper.VectorHashKeyWrapperBase;
+import org.apache.hadoop.hive.ql.exec.vector.wrapper.VectorHashKeyWrapperBatch;
 import org.apache.hadoop.hive.ql.metadata.HiveException;
 import org.apache.hadoop.hive.serde2.AbstractSerDe;
 import org.apache.hadoop.hive.serde2.SerDeException;
@@ -149,7 +149,7 @@ public class MapJoinKeyObject extends MapJoinKey {
     return nulls;
   }
 
-  public void readFromVector(VectorHashKeyWrapper kw, VectorExpressionWriter[] 
keyOutputWriters,
+  public void readFromVector(VectorHashKeyWrapperBase kw, 
VectorExpressionWriter[] keyOutputWriters,
       VectorHashKeyWrapperBatch keyWrapperBatch) throws HiveException {
     if (key == null || key.length != keyOutputWriters.length) {
       key = new Object[keyOutputWriters.length];

http://git-wip-us.apache.org/repos/asf/hive/blob/ccdcc5e2/ql/src/java/org/apache/hadoop/hive/ql/exec/persistence/MapJoinTableContainer.java
----------------------------------------------------------------------
diff --git 
a/ql/src/java/org/apache/hadoop/hive/ql/exec/persistence/MapJoinTableContainer.java
 
b/ql/src/java/org/apache/hadoop/hive/ql/exec/persistence/MapJoinTableContainer.java
index b0c7574..2c4229f 100644
--- 
a/ql/src/java/org/apache/hadoop/hive/ql/exec/persistence/MapJoinTableContainer.java
+++ 
b/ql/src/java/org/apache/hadoop/hive/ql/exec/persistence/MapJoinTableContainer.java
@@ -24,9 +24,9 @@ import java.util.List;
 import org.apache.hadoop.hive.ql.exec.ExprNodeEvaluator;
 import org.apache.hadoop.hive.ql.exec.JoinUtil;
 import org.apache.hadoop.hive.common.MemoryEstimate;
-import org.apache.hadoop.hive.ql.exec.vector.VectorHashKeyWrapper;
-import org.apache.hadoop.hive.ql.exec.vector.VectorHashKeyWrapperBatch;
 import 
org.apache.hadoop.hive.ql.exec.vector.expressions.VectorExpressionWriter;
+import org.apache.hadoop.hive.ql.exec.vector.wrapper.VectorHashKeyWrapperBase;
+import org.apache.hadoop.hive.ql.exec.vector.wrapper.VectorHashKeyWrapperBatch;
 import org.apache.hadoop.hive.ql.metadata.HiveException;
 import org.apache.hadoop.hive.serde2.SerDeException;
 import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspector;
@@ -43,7 +43,7 @@ public interface MapJoinTableContainer extends MemoryEstimate 
{
      * Changes current rows to which adaptor is referring to the rows 
corresponding to
      * the key represented by a VHKW object, and writers and batch used to 
interpret it.
      */
-    JoinUtil.JoinResult setFromVector(VectorHashKeyWrapper kw, 
VectorExpressionWriter[] keyOutputWriters,
+    JoinUtil.JoinResult setFromVector(VectorHashKeyWrapperBase kw, 
VectorExpressionWriter[] keyOutputWriters,
         VectorHashKeyWrapperBatch keyWrapperBatch) throws HiveException;
 
     /**

http://git-wip-us.apache.org/repos/asf/hive/blob/ccdcc5e2/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/VectorColumnSetInfo.java
----------------------------------------------------------------------
diff --git 
a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/VectorColumnSetInfo.java 
b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/VectorColumnSetInfo.java
index 7758ac4..7ada2bf 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/VectorColumnSetInfo.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/VectorColumnSetInfo.java
@@ -37,34 +37,34 @@ public class VectorColumnSetInfo {
   /**
    * indices of LONG primitive keys.
    */
-  protected int[] longIndices;
+  public int[] longIndices;
 
   /**
    * indices of DOUBLE primitive keys.
    */
-  protected int[] doubleIndices;
+  public int[] doubleIndices;
 
   /**
    * indices of string (byte[]) primitive keys.
    */
-  protected int[] stringIndices;
+  public int[] stringIndices;
 
   /**
    * indices of decimal primitive keys.
    */
-  protected int[] decimalIndices;
+  public int[] decimalIndices;
 
   /**
    * indices of TIMESTAMP primitive keys.
    */
-  protected int[] timestampIndices;
+  public int[] timestampIndices;
 
   /**
    * indices of INTERVAL_DAY_TIME primitive keys.
    */
-  protected int[] intervalDayTimeIndices;
+  public int[] intervalDayTimeIndices;
 
-  final protected int keyCount;
+  final public int keyCount;
   private int addKeyIndex;
 
   private int addLongIndex;
@@ -77,9 +77,9 @@ public class VectorColumnSetInfo {
   // Given the keyIndex these arrays return:
   //   The ColumnVector.Type,
   //   The type specific index into longIndices, doubleIndices, etc...
-  protected TypeInfo[] typeInfos;
-  protected ColumnVector.Type[] columnVectorTypes;
-  protected int[] columnTypeSpecificIndices;
+  public TypeInfo[] typeInfos;
+  public ColumnVector.Type[] columnVectorTypes;
+  public int[] columnTypeSpecificIndices;
 
   protected VectorColumnSetInfo(int keyCount) {
     this.keyCount = keyCount;

http://git-wip-us.apache.org/repos/asf/hive/blob/ccdcc5e2/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/VectorGroupByOperator.java
----------------------------------------------------------------------
diff --git 
a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/VectorGroupByOperator.java 
b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/VectorGroupByOperator.java
index 43f1162..7816cbb 100644
--- 
a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/VectorGroupByOperator.java
+++ 
b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/VectorGroupByOperator.java
@@ -44,6 +44,8 @@ import 
org.apache.hadoop.hive.ql.exec.vector.expressions.VectorExpression;
 import 
org.apache.hadoop.hive.ql.exec.vector.expressions.VectorExpressionWriter;
 import 
org.apache.hadoop.hive.ql.exec.vector.expressions.VectorExpressionWriterFactory;
 import 
org.apache.hadoop.hive.ql.exec.vector.expressions.aggregates.VectorAggregateExpression;
+import org.apache.hadoop.hive.ql.exec.vector.wrapper.VectorHashKeyWrapperBase;
+import org.apache.hadoop.hive.ql.exec.vector.wrapper.VectorHashKeyWrapperBatch;
 import org.apache.hadoop.hive.ql.metadata.HiveException;
 import org.apache.hadoop.hive.ql.plan.ExprNodeDesc;
 import org.apache.hadoop.hive.ql.plan.GroupByDesc;
@@ -453,7 +455,7 @@ public class VectorGroupByOperator extends 
Operator<GroupByDesc>
       if (!aborted && sumBatchSize == 0 && 
GroupByOperator.shouldEmitSummaryRow(conf)) {
         // in case the empty grouping set is preset; but no output has done
         // the "summary row" still needs to be emitted
-        VectorHashKeyWrapper kw = 
keyWrappersBatch.getVectorHashKeyWrappers()[0];
+        VectorHashKeyWrapperBase kw = 
keyWrappersBatch.getVectorHashKeyWrappers()[0];
         kw.setNull();
         int pos = conf.getGroupingSetPosition();
         if (pos >= 0) {
@@ -481,13 +483,13 @@ public class VectorGroupByOperator extends 
Operator<GroupByDesc>
 
       // We now have to probe the global hash and find-or-allocate
       // the aggregation buffers to use for each key present in the batch
-      VectorHashKeyWrapper[] keyWrappers = 
keyWrappersBatch.getVectorHashKeyWrappers();
+      VectorHashKeyWrapperBase[] keyWrappers = 
keyWrappersBatch.getVectorHashKeyWrappers();
 
       final int n = keyExpressions.length == 0 ? 1 : batch.size;
       // note - the row mapping is not relevant when 
aggregationBatchInfo::getDistinctBufferSetCount() == 1
 
       for (int i=0; i < n; ++i) {
-        VectorHashKeyWrapper kw = keyWrappers[i];
+        VectorHashKeyWrapperBase kw = keyWrappers[i];
         VectorAggregationBufferRow aggregationBuffer = 
mapKeysAggregationBuffers.get(kw);
         if (null == aggregationBuffer) {
           // the probe failed, we must allocate a set of aggregation buffers
@@ -564,7 +566,7 @@ public class VectorGroupByOperator extends 
Operator<GroupByDesc>
       while(iter.hasNext()) {
         Map.Entry<KeyWrapper, VectorAggregationBufferRow> pair = iter.next();
 
-        writeSingleRow((VectorHashKeyWrapper) pair.getKey(), pair.getValue());
+        writeSingleRow((VectorHashKeyWrapperBase) pair.getKey(), 
pair.getValue());
 
         if (!all) {
           iter.remove();
@@ -659,13 +661,13 @@ public class VectorGroupByOperator extends 
Operator<GroupByDesc>
     /**
      * The current key, used in streaming mode
      */
-    private VectorHashKeyWrapper streamingKey;
+    private VectorHashKeyWrapperBase streamingKey;
 
     /**
      * The keys that needs to be flushed at the end of the current batch
      */
-    private final VectorHashKeyWrapper[] keysToFlush =
-        new VectorHashKeyWrapper[VectorizedRowBatch.DEFAULT_SIZE];
+    private final VectorHashKeyWrapperBase[] keysToFlush =
+        new VectorHashKeyWrapperBase[VectorizedRowBatch.DEFAULT_SIZE];
 
     /**
      * The aggregates that needs to be flushed at the end of the current batch
@@ -723,9 +725,9 @@ public class VectorGroupByOperator extends 
Operator<GroupByDesc>
         keyWrappersBatch.evaluateBatchGroupingSets(batch, 
currentGroupingSetsOverrideIsNulls);
       }
 
-      VectorHashKeyWrapper[] batchKeys = 
keyWrappersBatch.getVectorHashKeyWrappers();
+      VectorHashKeyWrapperBase[] batchKeys = 
keyWrappersBatch.getVectorHashKeyWrappers();
 
-      final VectorHashKeyWrapper prevKey = streamingKey;
+      final VectorHashKeyWrapperBase prevKey = streamingKey;
       if (streamingKey == null) {
         // This is the first batch we process after switching from hash mode
         currentStreamingAggregators = 
streamAggregationBufferRowPool.getFromPool();
@@ -760,7 +762,7 @@ public class VectorGroupByOperator extends 
Operator<GroupByDesc>
       }
 
       if (streamingKey != prevKey) {
-        streamingKey = (VectorHashKeyWrapper) streamingKey.copyKey();
+        streamingKey = (VectorHashKeyWrapperBase) streamingKey.copyKey();
       }
     }
 
@@ -1127,7 +1129,7 @@ public class VectorGroupByOperator extends 
Operator<GroupByDesc>
    * @param agg
    * @throws HiveException
    */
-  private void writeSingleRow(VectorHashKeyWrapper kw, 
VectorAggregationBufferRow agg)
+  private void writeSingleRow(VectorHashKeyWrapperBase kw, 
VectorAggregationBufferRow agg)
       throws HiveException {
 
     int colNum = 0;

http://git-wip-us.apache.org/repos/asf/hive/blob/ccdcc5e2/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/VectorHashKeyWrapper.java
----------------------------------------------------------------------
diff --git 
a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/VectorHashKeyWrapper.java 
b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/VectorHashKeyWrapper.java
deleted file mode 100644
index 38c31a5..0000000
--- 
a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/VectorHashKeyWrapper.java
+++ /dev/null
@@ -1,682 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.hadoop.hive.ql.exec.vector;
-
-import org.apache.hadoop.hive.serde2.io.DateWritableV2;
-import org.apache.hive.common.util.Murmur3;
-
-import java.sql.Date;
-import java.sql.Timestamp;
-import java.util.Arrays;
-
-import org.apache.hadoop.hive.common.type.HiveDecimal;
-import org.apache.hadoop.hive.common.type.HiveIntervalDayTime;
-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;
-import org.apache.hadoop.hive.ql.util.JavaDataModel;
-import org.apache.hadoop.hive.serde2.io.HiveDecimalWritable;
-import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspector;
-import org.apache.hadoop.hive.serde2.typeinfo.PrimitiveTypeInfo;
-
-import com.google.common.base.Preconditions;
-
-/**
- * A hash map key wrapper for vectorized processing.
- * It stores the key values as primitives in arrays for each supported 
primitive type.
- * This works in conjunction with
- * {@link org.apache.hadoop.hive.ql.exec.VectorHashKeyWrapperBatch 
VectorHashKeyWrapperBatch}
- * to hash vectorized processing units (batches).
- */
-public class VectorHashKeyWrapper extends KeyWrapper {
-
-  public static final class HashContext {
-    private final Murmur3.IncrementalHash32 bytesHash = new 
Murmur3.IncrementalHash32();
-
-    public static Murmur3.IncrementalHash32 getBytesHash(HashContext ctx) {
-      if (ctx == null) {
-        return new Murmur3.IncrementalHash32();
-      }
-      return ctx.bytesHash;
-    }
-  }
-
-  private static final int[] EMPTY_INT_ARRAY = new int[0];
-  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 HiveDecimalWritable[] EMPTY_DECIMAL_ARRAY = new 
HiveDecimalWritable[0];
-  private static final Timestamp[] EMPTY_TIMESTAMP_ARRAY = new Timestamp[0];
-  private static final HiveIntervalDayTime[] EMPTY_INTERVAL_DAY_TIME_ARRAY = 
new HiveIntervalDayTime[0];
-
-  public static final VectorHashKeyWrapper EMPTY_KEY_WRAPPER = new 
EmptyVectorHashKeyWrapper();
-
-  private long[] longValues;
-  private double[] doubleValues;
-
-  private byte[][] byteValues;
-  private int[] byteStarts;
-  private int[] byteLengths;
-
-  private HiveDecimalWritable[] decimalValues;
-
-  private Timestamp[] timestampValues;
-  private static Timestamp ZERO_TIMESTAMP = new Timestamp(0);
-
-  private HiveIntervalDayTime[] intervalDayTimeValues;
-  private static HiveIntervalDayTime ZERO_INTERVALDAYTIME= new 
HiveIntervalDayTime(0, 0);
-
-  // NOTE: The null array is indexed by keyIndex, which is not available 
internally.  The mapping
-  //       from a long, double, etc index to key index is kept once in the 
separate
-  //       VectorColumnSetInfo object.
-  private boolean[] isNull;
-
-  private int hashcode;
-
-  private HashContext hashCtx;
-
-  private VectorHashKeyWrapper(HashContext ctx, int longValuesCount, int 
doubleValuesCount,
-          int byteValuesCount, int decimalValuesCount, int 
timestampValuesCount,
-          int intervalDayTimeValuesCount,
-          int keyCount) {
-    hashCtx = ctx;
-    longValues = longValuesCount > 0 ? new long[longValuesCount] : 
EMPTY_LONG_ARRAY;
-    doubleValues = doubleValuesCount > 0 ? new double[doubleValuesCount] : 
EMPTY_DOUBLE_ARRAY;
-    decimalValues = decimalValuesCount > 0 ? new 
HiveDecimalWritable[decimalValuesCount] : EMPTY_DECIMAL_ARRAY;
-    timestampValues = timestampValuesCount > 0 ? new 
Timestamp[timestampValuesCount] : EMPTY_TIMESTAMP_ARRAY;
-    intervalDayTimeValues = intervalDayTimeValuesCount > 0 ? new 
HiveIntervalDayTime[intervalDayTimeValuesCount] : EMPTY_INTERVAL_DAY_TIME_ARRAY;
-    for(int i = 0; i < decimalValuesCount; ++i) {
-      decimalValues[i] = new HiveDecimalWritable(HiveDecimal.ZERO);
-    }
-    if (byteValuesCount > 0) {
-      byteValues = new byte[byteValuesCount][];
-      byteStarts = new int[byteValuesCount];
-      byteLengths = new int[byteValuesCount];
-    } else {
-      byteValues = EMPTY_BYTES_ARRAY;
-      byteStarts = EMPTY_INT_ARRAY;
-      byteLengths = EMPTY_INT_ARRAY;
-    }
-    for(int i = 0; i < timestampValuesCount; ++i) {
-      timestampValues[i] = new Timestamp(0);
-    }
-    for(int i = 0; i < intervalDayTimeValuesCount; ++i) {
-      intervalDayTimeValues[i] = new HiveIntervalDayTime();
-    }
-    isNull = new boolean[keyCount];
-    hashcode = 0;
-  }
-
-  private VectorHashKeyWrapper() {
-  }
-
-  public static VectorHashKeyWrapper allocate(HashContext ctx, int 
longValuesCount, int doubleValuesCount,
-      int byteValuesCount, int decimalValuesCount, int timestampValuesCount,
-      int intervalDayTimeValuesCount, int keyCount) {
-    if ((longValuesCount + doubleValuesCount + byteValuesCount + 
decimalValuesCount
-        + timestampValuesCount + intervalDayTimeValuesCount) == 0) {
-      return EMPTY_KEY_WRAPPER;
-    }
-    return new VectorHashKeyWrapper(ctx, longValuesCount, doubleValuesCount, 
byteValuesCount,
-        decimalValuesCount, timestampValuesCount, intervalDayTimeValuesCount,
-        keyCount);
-  }
-
-  @Override
-  public void getNewKey(Object row, ObjectInspector rowInspector) throws 
HiveException {
-    throw new HiveException("Should not be called");
-  }
-
-  @Override
-  public void setHashKey() {
-    // compute locally and assign
-    int hash = Arrays.hashCode(longValues) ^
-        Arrays.hashCode(doubleValues) ^
-        Arrays.hashCode(isNull);
-
-    for (int i = 0; i < decimalValues.length; i++) {
-      // Use the new faster hash code since we are hashing memory objects.
-      hash ^= decimalValues[i].newFasterHashCode();
-    }
-
-    for (int i = 0; i < timestampValues.length; i++) {
-      hash ^= timestampValues[i].hashCode();
-    }
-
-    for (int i = 0; i < intervalDayTimeValues.length; i++) {
-      hash ^= intervalDayTimeValues[i].hashCode();
-    }
-
-    // This code, with branches and all, is not executed if there are no 
string keys
-    Murmur3.IncrementalHash32 bytesHash = null;
-    for (int i = 0; i < byteValues.length; ++i) {
-      /*
-       *  Hashing the string is potentially expensive so is better to branch.
-       *  Additionally not looking at values for nulls allows us not reset the 
values.
-       */
-      if (byteLengths[i] == -1) {
-        continue;
-      }
-      if (bytesHash == null) {
-        bytesHash = HashContext.getBytesHash(hashCtx);
-        bytesHash.start(hash);
-      }
-      bytesHash.add(byteValues[i], byteStarts[i], byteLengths[i]);
-    }
-    if (bytesHash != null) {
-      hash = bytesHash.end();
-    }
-    this.hashcode = hash;
-  }
-
-  @Override
-  public int hashCode() {
-    return hashcode;
-  }
-
-  @Override
-  public boolean equals(Object that) {
-    if (that instanceof VectorHashKeyWrapper) {
-      VectorHashKeyWrapper keyThat = (VectorHashKeyWrapper)that;
-      // not comparing hashCtx - irrelevant
-      return hashcode == keyThat.hashcode &&
-          Arrays.equals(longValues, keyThat.longValues) &&
-          Arrays.equals(doubleValues, keyThat.doubleValues) &&
-          Arrays.equals(decimalValues,  keyThat.decimalValues) &&
-          Arrays.equals(timestampValues,  keyThat.timestampValues) &&
-          Arrays.equals(intervalDayTimeValues,  keyThat.intervalDayTimeValues) 
&&
-          Arrays.equals(isNull, keyThat.isNull) &&
-          byteValues.length == keyThat.byteValues.length &&
-          (0 == byteValues.length || bytesEquals(keyThat));
-    }
-    return false;
-  }
-
-  private boolean bytesEquals(VectorHashKeyWrapper keyThat) {
-    //By the time we enter here the byteValues.lentgh and isNull must have 
already been compared
-    for (int i = 0; i < byteValues.length; ++i) {
-      // the byte comparison is potentially expensive so is better to branch 
on null
-      if (byteLengths[i] != -1) {
-        if (!StringExpr.equal(
-            byteValues[i],
-            byteStarts[i],
-            byteLengths[i],
-            keyThat.byteValues[i],
-            keyThat.byteStarts[i],
-            keyThat.byteLengths[i])) {
-          return false;
-        }
-      }
-    }
-    return true;
-  }
-
-  @Override
-  protected Object clone() {
-    VectorHashKeyWrapper clone = new VectorHashKeyWrapper();
-    duplicateTo(clone);
-    return clone;
-  }
-
-  public void duplicateTo(VectorHashKeyWrapper clone) {
-    clone.hashCtx = hashCtx;
-    clone.longValues = (longValues.length > 0) ? longValues.clone() : 
EMPTY_LONG_ARRAY;
-    clone.doubleValues = (doubleValues.length > 0) ? doubleValues.clone() : 
EMPTY_DOUBLE_ARRAY;
-    clone.isNull = isNull.clone();
-
-    if (decimalValues.length > 0) {
-      // 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;
-    }
-
-    if (byteLengths.length > 0) {
-      clone.byteValues = new byte[byteValues.length][];
-      clone.byteStarts = new int[byteValues.length];
-      clone.byteLengths = byteLengths.clone();
-      for (int i = 0; i < byteValues.length; ++i) {
-        // avoid allocation/copy of nulls, because it potentially expensive.
-        // branch instead.
-        if (byteLengths[i] != -1) {
-          clone.byteValues[i] = Arrays.copyOfRange(byteValues[i],
-              byteStarts[i], byteStarts[i] + byteLengths[i]);
-        }
-      }
-    } else {
-      clone.byteValues = EMPTY_BYTES_ARRAY;
-      clone.byteStarts = EMPTY_INT_ARRAY;
-      clone.byteLengths = EMPTY_INT_ARRAY;
-    }
-    if (timestampValues.length > 0) {
-      clone.timestampValues = new Timestamp[timestampValues.length];
-      for(int i = 0; i < timestampValues.length; ++i) {
-        clone.timestampValues[i] = (Timestamp) timestampValues[i].clone();
-      }
-    } else {
-      clone.timestampValues = EMPTY_TIMESTAMP_ARRAY;
-    }
-    if (intervalDayTimeValues.length > 0) {
-      clone.intervalDayTimeValues = new 
HiveIntervalDayTime[intervalDayTimeValues.length];
-      for(int i = 0; i < intervalDayTimeValues.length; ++i) {
-        clone.intervalDayTimeValues[i] = (HiveIntervalDayTime) 
intervalDayTimeValues[i].clone();
-      }
-    } else {
-      clone.intervalDayTimeValues = EMPTY_INTERVAL_DAY_TIME_ARRAY;
-    }
-
-    clone.hashcode = hashcode;
-    assert clone.equals(this);
-  }
-
-  @Override
-  public KeyWrapper copyKey() {
-    return (KeyWrapper) clone();
-  }
-
-  @Override
-  public void copyKey(KeyWrapper oldWrapper) {
-    throw new UnsupportedOperationException();
-  }
-
-  @Override
-  public Object[] getKeyArray() {
-    throw new UnsupportedOperationException();
-  }
-
-  public void assignLong(int keyIndex, int index, long v) {
-    isNull[keyIndex] = false;
-    longValues[index] = v;
-  }
-
-  // FIXME: isNull is not updated; which might cause problems
-  @Deprecated
-  public void assignLong(int index, long v) {
-    longValues[index] = v;
-  }
-
-  public void assignNullLong(int keyIndex, int index) {
-    isNull[keyIndex] = true;
-    longValues[index] = 0; // assign 0 to simplify hashcode
-  }
-
-  public void assignDouble(int index, double d) {
-    doubleValues[index] = d;
-  }
-
-  public void assignNullDouble(int keyIndex, int index) {
-    isNull[keyIndex] = true;
-    doubleValues[index] = 0; // assign 0 to simplify hashcode
-  }
-
-  public void assignString(int index, byte[] bytes, int start, int length) {
-    Preconditions.checkState(bytes != null);
-    byteValues[index] = bytes;
-    byteStarts[index] = start;
-    byteLengths[index] = length;
-  }
-
-  public void assignNullString(int keyIndex, int index) {
-    isNull[keyIndex] = true;
-    byteValues[index] = null;
-    byteStarts[index] = 0;
-    // We need some value that indicates NULL.
-    byteLengths[index] = -1;
-  }
-
-  public void assignDecimal(int index, HiveDecimalWritable value) {
-    decimalValues[index].set(value);
-  }
-
-  public void assignNullDecimal(int keyIndex, int index) {
-    isNull[keyIndex] = true;
-    decimalValues[index].set(HiveDecimal.ZERO); // assign 0 to simplify 
hashcode
-  }
-
-  public void assignTimestamp(int index, Timestamp value) {
-    // Do not assign the input value object to the timestampValues array 
element.
-    // Always copy value using set* methods.
-    timestampValues[index].setTime(value.getTime());
-    timestampValues[index].setNanos(value.getNanos());
-  }
-
-  public void assignTimestamp(int index, TimestampColumnVector colVector, int 
elementNum) {
-    colVector.timestampUpdate(timestampValues[index], elementNum);
-  }
-
-  public void assignNullTimestamp(int keyIndex, int index) {
-    isNull[keyIndex] = true;
-    // assign 0 to simplify hashcode
-    timestampValues[index].setTime(ZERO_TIMESTAMP.getTime());
-    timestampValues[index].setNanos(ZERO_TIMESTAMP.getNanos());
-  }
-
-  public void assignIntervalDayTime(int index, HiveIntervalDayTime value) {
-    intervalDayTimeValues[index].set(value);
-  }
-
-  public void assignIntervalDayTime(int index, IntervalDayTimeColumnVector 
colVector, int elementNum) {
-    
intervalDayTimeValues[index].set(colVector.asScratchIntervalDayTime(elementNum));
-  }
-
-  public void assignNullIntervalDayTime(int keyIndex, int index) {
-    isNull[keyIndex] = true;
-    intervalDayTimeValues[index].set(ZERO_INTERVALDAYTIME); // assign 0 to 
simplify hashcode
-  }
-
-  /*
-   * This method is mainly intended for debug display purposes.
-   */
-  public String stringifyKeys(VectorColumnSetInfo columnSetInfo)
-  {
-    StringBuilder sb = new StringBuilder();
-    boolean isFirstKey = true;
-
-    if (longValues.length > 0) {
-      isFirstKey = false;
-      sb.append("longs ");
-      boolean isFirstValue = true;
-      for (int i = 0; i < columnSetInfo.longIndices.length; i++) {
-        if (isFirstValue) {
-          isFirstValue = false;
-        } else {
-          sb.append(", ");
-        }
-        int keyIndex = columnSetInfo.longIndices[i];
-        if (isNull[keyIndex]) {
-          sb.append("null");
-        } else {
-          sb.append(longValues[i]);
-          PrimitiveTypeInfo primitiveTypeInfo = (PrimitiveTypeInfo) 
columnSetInfo.typeInfos[keyIndex];
-          // FUTURE: Add INTERVAL_YEAR_MONTH, etc, as desired.
-          switch (primitiveTypeInfo.getPrimitiveCategory()) {
-          case DATE:
-            {
-              Date dt = new Date(0);
-              dt.setTime(DateWritableV2.daysToMillis((int) longValues[i]));
-              sb.append(" date ");
-              sb.append(dt.toString());
-            }
-            break;
-          default:
-            // Add nothing more.
-            break;
-          }
-        }
-      }
-    }
-    if (doubleValues.length > 0) {
-      if (isFirstKey) {
-        isFirstKey = false;
-      } else {
-        sb.append(", ");
-      }
-      sb.append("doubles ");
-      boolean isFirstValue = true;
-      for (int i = 0; i < columnSetInfo.doubleIndices.length; i++) {
-        if (isFirstValue) {
-          isFirstValue = false;
-        } else {
-          sb.append(", ");
-        }
-        int keyIndex = columnSetInfo.doubleIndices[i];
-        if (isNull[keyIndex]) {
-          sb.append("null");
-        } else {
-          sb.append(doubleValues[i]);
-        }
-      }
-    }
-    if (byteValues.length > 0) {
-      if (isFirstKey) {
-        isFirstKey = false;
-      } else {
-        sb.append(", ");
-      }
-      sb.append("byte lengths ");
-      boolean isFirstValue = true;
-      for (int i = 0; i < columnSetInfo.stringIndices.length; i++) {
-        if (isFirstValue) {
-          isFirstValue = false;
-        } else {
-          sb.append(", ");
-        }
-        int keyIndex = columnSetInfo.stringIndices[i];
-        if (isNull[keyIndex]) {
-          sb.append("null");
-        } else {
-          sb.append(byteLengths[i]);
-        }
-      }
-    }
-    if (decimalValues.length > 0) {
-      if (isFirstKey) {
-        isFirstKey = true;
-      } else {
-        sb.append(", ");
-      }
-      sb.append("decimals ");
-      boolean isFirstValue = true;
-      for (int i = 0; i < columnSetInfo.decimalIndices.length; i++) {
-        if (isFirstValue) {
-          isFirstValue = false;
-        } else {
-          sb.append(", ");
-        }
-        int keyIndex = columnSetInfo.decimalIndices[i];
-        if (isNull[keyIndex]) {
-          sb.append("null");
-        } else {
-          sb.append(decimalValues[i]);
-        }
-      }
-    }
-    if (timestampValues.length > 0) {
-      if (isFirstKey) {
-        isFirstKey = false;
-      } else {
-        sb.append(", ");
-      }
-      sb.append("timestamps ");
-      boolean isFirstValue = true;
-      for (int i = 0; i < columnSetInfo.timestampIndices.length; i++) {
-        if (isFirstValue) {
-          isFirstValue = false;
-        } else {
-          sb.append(", ");
-        }
-        int keyIndex = columnSetInfo.timestampIndices[i];
-        if (isNull[keyIndex]) {
-          sb.append("null");
-        } else {
-          sb.append(timestampValues[i]);
-        }
-      }
-    }
-    if (intervalDayTimeValues.length > 0) {
-      if (isFirstKey) {
-        isFirstKey = false;
-      } else {
-        sb.append(", ");
-      }
-      sb.append("interval day times ");
-      boolean isFirstValue = true;
-      for (int i = 0; i < columnSetInfo.intervalDayTimeIndices.length; i++) {
-        if (isFirstValue) {
-          isFirstValue = false;
-        } else {
-          sb.append(", ");
-        }
-        int keyIndex = columnSetInfo.intervalDayTimeIndices[i];
-        if (isNull[keyIndex]) {
-          sb.append("null");
-        } else {
-          sb.append(intervalDayTimeValues[i]);
-        }
-      }
-    }
-
-    return sb.toString();
-  }
-
-  @Override
-  public String toString()
-  {
-    StringBuilder sb = new StringBuilder();
-    boolean isFirst = true;
-    if (longValues.length > 0) {
-      isFirst = false;
-      sb.append("longs ");
-      sb.append(Arrays.toString(longValues));
-    }
-    if (doubleValues.length > 0) {
-      if (isFirst) {
-        isFirst = false;
-      } else {
-        sb.append(", ");
-      }
-      sb.append("doubles ");
-      sb.append(Arrays.toString(doubleValues));
-    }
-    if (byteValues.length > 0) {
-      if (isFirst) {
-        isFirst = false;
-      } else {
-        sb.append(", ");
-      }
-      sb.append("byte lengths ");
-      sb.append(Arrays.toString(byteLengths));
-    }
-    if (decimalValues.length > 0) {
-      if (isFirst) {
-        isFirst = false;
-      } else {
-        sb.append(", ");
-      }
-      sb.append("decimals ");
-      sb.append(Arrays.toString(decimalValues));
-    }
-    if (timestampValues.length > 0) {
-      if (isFirst) {
-        isFirst = false;
-      } else {
-        sb.append(", ");
-      }
-      sb.append("timestamps ");
-      sb.append(Arrays.toString(timestampValues));
-    }
-    if (intervalDayTimeValues.length > 0) {
-      if (isFirst) {
-        isFirst = false;
-      } else {
-        sb.append(", ");
-      }
-      sb.append("interval day times ");
-      sb.append(Arrays.toString(intervalDayTimeValues));
-    }
-
-    if (isFirst) {
-      isFirst = false;
-    } else {
-      sb.append(", ");
-    }
-    sb.append("nulls ");
-    sb.append(Arrays.toString(isNull));
-
-    return sb.toString();
-  }
-
-  public long getLongValue(int i) {
-    return longValues[i];
-  }
-
-  public double getDoubleValue(int i) {
-    return doubleValues[i];
-  }
-
-  public byte[] getBytes(int i) {
-    return byteValues[i];
-  }
-
-  public int getByteStart(int i) {
-    return byteStarts[i];
-  }
-
-  public int getByteLength(int i) {
-    return byteLengths[i];
-  }
-
-  public int getVariableSize() {
-    int variableSize = 0;
-    for (int i=0; i<byteLengths.length; ++i) {
-      JavaDataModel model = JavaDataModel.get();
-      variableSize += model.lengthForByteArrayOfSize(byteLengths[i]);
-    }
-    return variableSize;
-  }
-
-  public HiveDecimalWritable getDecimal(int i) {
-    return decimalValues[i];
-  }
-
-  public Timestamp getTimestamp(int i) {
-    return timestampValues[i];
-  }
-
-  public HiveIntervalDayTime getIntervalDayTime(int i) {
-    return intervalDayTimeValues[i];
-  }
-
-  public void clearIsNull() {
-    Arrays.fill(isNull, false);
-  }
-
-  public void setNull() {
-    Arrays.fill(isNull, true);
-  }
-
-  public boolean isNull(int keyIndex) {
-    return isNull[keyIndex];
-  }
-
-  public static final class EmptyVectorHashKeyWrapper extends 
VectorHashKeyWrapper {
-    private EmptyVectorHashKeyWrapper() {
-      super(null, 0, 0, 0, 0, 0, 0, /* keyCount */ 0);
-      // no need to override assigns - all assign ops will fail due to 0 size
-    }
-
-    @Override
-    protected Object clone() {
-      // immutable
-      return this;
-    }
-
-    @Override
-    public boolean equals(Object that) {
-      if (that == this) {
-        // should only be one object
-        return true;
-      }
-      return super.equals(that);
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/hive/blob/ccdcc5e2/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/VectorHashKeyWrapperBatch.java
----------------------------------------------------------------------
diff --git 
a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/VectorHashKeyWrapperBatch.java
 
b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/VectorHashKeyWrapperBatch.java
deleted file mode 100644
index 689d3c3..0000000
--- 
a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/VectorHashKeyWrapperBatch.java
+++ /dev/null
@@ -1,1067 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.hadoop.hive.ql.exec.vector;
-
-import org.apache.hadoop.hive.ql.exec.vector.expressions.VectorExpression;
-import 
org.apache.hadoop.hive.ql.exec.vector.expressions.VectorExpressionWriter;
-import org.apache.hadoop.hive.ql.metadata.HiveException;
-import org.apache.hadoop.hive.ql.util.JavaDataModel;
-import org.apache.hadoop.hive.ql.exec.vector.ColumnVector.Type;
-import org.apache.hadoop.hive.serde2.typeinfo.TypeInfo;
-
-/**
- * Class for handling vectorized hash map key wrappers. It evaluates the key 
columns in a
- * row batch in a vectorized fashion.
- * This class stores additional information about keys needed to evaluate and 
output the key values.
- *
- */
-public class VectorHashKeyWrapperBatch extends VectorColumnSetInfo {
-
-  public VectorHashKeyWrapperBatch(int keyCount) {
-    super(keyCount);
-  }
-
-  /**
-   * Number of object references in 'this' (for size computation)
-   */
-  private static final int MODEL_REFERENCES_COUNT = 7;
-
-  /**
-   * The key expressions that require evaluation and output the primitive 
values for each key.
-   */
-  private VectorExpression[] keyExpressions;
-
-  /**
-   * Pre-allocated batch size vector of keys wrappers.
-   * N.B. these keys are **mutable** and should never be used in a HashMap.
-   * Always clone the key wrapper to obtain an immutable keywrapper suitable
-   * to use a key in a HashMap.
-   */
-  private VectorHashKeyWrapper[] vectorHashKeyWrappers;
-
-  /**
-   * The fixed size of the key wrappers.
-   */
-  private int keysFixedSize;
-
-  /**
-   * Shared hashcontext for all keys in this batch
-   */
-  private final VectorHashKeyWrapper.HashContext hashCtx = new 
VectorHashKeyWrapper.HashContext();
-
-   /**
-   * Returns the compiled fixed size for the key wrappers.
-   * @return
-   */
-  public int getKeysFixedSize() {
-    return keysFixedSize;
-  }
-
-  /**
-   * Accessor for the batch-sized array of key wrappers.
-   */
-  public VectorHashKeyWrapper[] getVectorHashKeyWrappers() {
-    return vectorHashKeyWrappers;
-  }
-
-  /**
-   * Processes a batch:
-   * <ul>
-   * <li>Evaluates each key vector expression.</li>
-   * <li>Copies out each key's primitive values into the key wrappers</li>
-   * <li>computes the hashcode of the key wrappers</li>
-   * </ul>
-   * @param batch
-   * @throws HiveException
-   */
-  public void evaluateBatch(VectorizedRowBatch batch) throws HiveException {
-
-    if (keyCount == 0) {
-      // all keywrappers must be EmptyVectorHashKeyWrapper
-      return;
-    }
-
-    for(int i=0;i<batch.size;++i) {
-      vectorHashKeyWrappers[i].clearIsNull();
-    }
-
-    int keyIndex;
-    int columnIndex;
-    for(int i = 0; i< longIndices.length; ++i) {
-      keyIndex = longIndices[i];
-      columnIndex = keyExpressions[keyIndex].getOutputColumnNum();
-      LongColumnVector columnVector = (LongColumnVector) 
batch.cols[columnIndex];
-
-      evaluateLongColumnVector(batch, columnVector, keyIndex, i);
-    }
-
-    for(int i=0;i<doubleIndices.length; ++i) {
-      keyIndex = doubleIndices[i];
-      columnIndex = keyExpressions[keyIndex].getOutputColumnNum();
-      DoubleColumnVector columnVector = (DoubleColumnVector) 
batch.cols[columnIndex];
-
-      evaluateDoubleColumnVector(batch, columnVector, keyIndex, i);
-    }
-
-    for(int i=0;i<stringIndices.length; ++i) {
-      keyIndex = stringIndices[i];
-      columnIndex = keyExpressions[keyIndex].getOutputColumnNum();
-      BytesColumnVector columnVector = (BytesColumnVector) 
batch.cols[columnIndex];
-
-      evaluateStringColumnVector(batch, columnVector, keyIndex, i);
-    }
-
-    for(int i=0;i<decimalIndices.length; ++i) {
-      keyIndex = decimalIndices[i];
-      columnIndex = keyExpressions[keyIndex].getOutputColumnNum();
-      DecimalColumnVector columnVector = (DecimalColumnVector) 
batch.cols[columnIndex];
-
-      evaluateDecimalColumnVector(batch, columnVector, keyIndex, i);
-    }
-
-    for(int i=0;i<timestampIndices.length; ++i) {
-      keyIndex = timestampIndices[i];
-      columnIndex = keyExpressions[keyIndex].getOutputColumnNum();
-      TimestampColumnVector columnVector = (TimestampColumnVector) 
batch.cols[columnIndex];
-
-      evaluateTimestampColumnVector(batch, columnVector, keyIndex, i);
-    }
-
-    for(int i=0;i<intervalDayTimeIndices.length; ++i) {
-      keyIndex = intervalDayTimeIndices[i];
-      columnIndex = keyExpressions[keyIndex].getOutputColumnNum();
-      IntervalDayTimeColumnVector columnVector = (IntervalDayTimeColumnVector) 
batch.cols[columnIndex];
-
-      evaluateIntervalDayTimeColumnVector(batch, columnVector, keyIndex, i);
-    }
-    for(int i=0;i<batch.size;++i) {
-      vectorHashKeyWrappers[i].setHashKey();
-    }
-  }
-
-  public void evaluateBatchGroupingSets(VectorizedRowBatch batch,
-      boolean[] groupingSetsOverrideIsNulls) throws HiveException {
-
-    for(int i=0;i<batch.size;++i) {
-      vectorHashKeyWrappers[i].clearIsNull();
-    }
-    int keyIndex;
-    int columnIndex;
-    for(int i = 0; i< longIndices.length; ++i) {
-      keyIndex = longIndices[i];
-      if (groupingSetsOverrideIsNulls[keyIndex]) {
-        final int batchSize = batch.size;
-        for(int r = 0; r < batchSize; ++r) {
-          vectorHashKeyWrappers[r].assignNullLong(keyIndex, i);
-        }
-        continue;
-      }
-      columnIndex = keyExpressions[keyIndex].getOutputColumnNum();
-      LongColumnVector columnVector = (LongColumnVector) 
batch.cols[columnIndex];
-
-      evaluateLongColumnVector(batch, columnVector, keyIndex, i);
-    }
-
-    for(int i=0;i<doubleIndices.length; ++i) {
-      keyIndex = doubleIndices[i];
-      if (groupingSetsOverrideIsNulls[keyIndex]) {
-        final int batchSize = batch.size;
-        for(int r = 0; r < batchSize; ++r) {
-          vectorHashKeyWrappers[r].assignNullDouble(keyIndex, i);
-        }
-        continue;
-      }
-      columnIndex = keyExpressions[keyIndex].getOutputColumnNum();
-      DoubleColumnVector columnVector = (DoubleColumnVector) 
batch.cols[columnIndex];
-
-      evaluateDoubleColumnVector(batch, columnVector, keyIndex, i);
-    }
-
-    for(int i=0;i<stringIndices.length; ++i) {
-      keyIndex = stringIndices[i];
-      if (groupingSetsOverrideIsNulls[keyIndex]) {
-        final int batchSize = batch.size;
-        for(int r = 0; r < batchSize; ++r) {
-          vectorHashKeyWrappers[r].assignNullString(keyIndex, i);
-        }
-        continue;
-      }
-      columnIndex = keyExpressions[keyIndex].getOutputColumnNum();
-      BytesColumnVector columnVector = (BytesColumnVector) 
batch.cols[columnIndex];
-
-      evaluateStringColumnVector(batch, columnVector, keyIndex, i);
-    }
-
-    for(int i=0;i<decimalIndices.length; ++i) {
-      keyIndex = decimalIndices[i];
-      if (groupingSetsOverrideIsNulls[keyIndex]) {
-        final int batchSize = batch.size;
-        for(int r = 0; r < batchSize; ++r) {
-          vectorHashKeyWrappers[r].assignNullDecimal(keyIndex, i);
-        }
-        continue;
-      }
-      columnIndex = keyExpressions[keyIndex].getOutputColumnNum();
-      DecimalColumnVector columnVector = (DecimalColumnVector) 
batch.cols[columnIndex];
-
-      evaluateDecimalColumnVector(batch, columnVector, keyIndex, i);
-    }
-
-    for(int i=0;i<timestampIndices.length; ++i) {
-      keyIndex = timestampIndices[i];
-      if (groupingSetsOverrideIsNulls[keyIndex]) {
-        final int batchSize = batch.size;
-        for(int r = 0; r < batchSize; ++r) {
-          vectorHashKeyWrappers[r].assignNullTimestamp(keyIndex, i);
-        }
-        continue;
-      }
-      columnIndex = keyExpressions[keyIndex].getOutputColumnNum();
-      TimestampColumnVector columnVector = (TimestampColumnVector) 
batch.cols[columnIndex];
-
-      evaluateTimestampColumnVector(batch, columnVector, keyIndex, i);
-    }
-
-    for(int i=0;i<intervalDayTimeIndices.length; ++i) {
-      keyIndex = intervalDayTimeIndices[i];
-      if (groupingSetsOverrideIsNulls[keyIndex]) {
-        final int batchSize = batch.size;
-        for(int r = 0; r < batchSize; ++r) {
-          vectorHashKeyWrappers[r].assignNullIntervalDayTime(keyIndex, i);
-        }
-        continue;
-      }
-      columnIndex = keyExpressions[keyIndex].getOutputColumnNum();
-      IntervalDayTimeColumnVector columnVector = (IntervalDayTimeColumnVector) 
batch.cols[columnIndex];
-
-      evaluateIntervalDayTimeColumnVector(batch, columnVector, keyIndex, i);
-    }
-
-    for(int i=0;i<batch.size;++i) {
-      vectorHashKeyWrappers[i].setHashKey();
-    }
-  }
-
-  private void evaluateLongColumnVector(VectorizedRowBatch batch, 
LongColumnVector columnVector,
-      int keyIndex, int index) {
-    if (columnVector.isRepeating) {
-      if (columnVector.noNulls || !columnVector.isNull[0]) {
-        assignLongNoNullsRepeating(index, batch.size, columnVector);
-      } else {
-        assignLongNullsRepeating(keyIndex, index, batch.size, columnVector);
-      }
-    } else if (columnVector.noNulls) {
-      if (batch.selectedInUse) {
-        assignLongNoNullsNoRepeatingSelection(index, batch.size, columnVector, 
batch.selected);
-      } else {
-        assignLongNoNullsNoRepeatingNoSelection(index, batch.size, 
columnVector);
-      }
-    } else {
-      if (batch.selectedInUse) {
-        assignLongNullsNoRepeatingSelection (keyIndex, index, batch.size, 
columnVector, batch.selected);
-      } else {
-        assignLongNullsNoRepeatingNoSelection(keyIndex, index, batch.size, 
columnVector);
-      }
-    }
-  }
-
-  private void evaluateDoubleColumnVector(VectorizedRowBatch batch, 
DoubleColumnVector columnVector,
-      int keyIndex, int index) {
-    if (columnVector.isRepeating) {
-      if (columnVector.noNulls || !columnVector.isNull[0]) {
-        assignDoubleNoNullsRepeating(index, batch.size, columnVector);
-      } else {
-        assignDoubleNullsRepeating(keyIndex, index, batch.size, columnVector);
-      }
-    } else if (columnVector.noNulls) {
-      if (batch.selectedInUse) {
-        assignDoubleNoNullsNoRepeatingSelection(index, batch.size, 
columnVector, batch.selected);
-      } else {
-        assignDoubleNoNullsNoRepeatingNoSelection(index, batch.size, 
columnVector);
-      }
-    } else {
-      if (batch.selectedInUse) {
-        assignDoubleNullsNoRepeatingSelection (keyIndex, index, batch.size, 
columnVector, batch.selected);
-      } else {
-        assignDoubleNullsNoRepeatingNoSelection(keyIndex, index, batch.size, 
columnVector);
-      }
-    }
-  }
-
-  private void evaluateStringColumnVector(VectorizedRowBatch batch, 
BytesColumnVector columnVector,
-      int keyIndex, int index) {
-    if (columnVector.isRepeating) {
-      if (columnVector.noNulls || !columnVector.isNull[0]) {
-        assignStringNoNullsRepeating(index, batch.size, columnVector);
-      } else {
-        assignStringNullsRepeating(keyIndex, index, batch.size, columnVector);
-      }
-    } else if (columnVector.noNulls) {
-      if (batch.selectedInUse) {
-        assignStringNoNullsNoRepeatingSelection(index, batch.size, 
columnVector, batch.selected);
-      } else {
-        assignStringNoNullsNoRepeatingNoSelection(index, batch.size, 
columnVector);
-      }
-    } else {
-      if (batch.selectedInUse) {
-        assignStringNullsNoRepeatingSelection (keyIndex, index, batch.size, 
columnVector, batch.selected);
-      } else {
-        assignStringNullsNoRepeatingNoSelection(keyIndex, index, batch.size, 
columnVector);
-      }
-    }
-  }
-
-  private void evaluateDecimalColumnVector(VectorizedRowBatch batch, 
DecimalColumnVector columnVector,
-      int keyIndex, int index) {
-    if (columnVector.isRepeating) {
-      if (columnVector.noNulls || !columnVector.isNull[0]) {
-        assignDecimalNoNullsRepeating(index, batch.size, columnVector);
-      } else {
-        assignDecimalNullsRepeating(keyIndex, index, batch.size, columnVector);
-      }
-    } else if (columnVector.noNulls) {
-      if (batch.selectedInUse) {
-        assignDecimalNoNullsNoRepeatingSelection(index, batch.size, 
columnVector, batch.selected);
-      } else {
-        assignDecimalNoNullsNoRepeatingNoSelection(index, batch.size, 
columnVector);
-      }
-    } else {
-      if (batch.selectedInUse) {
-        assignDecimalNullsNoRepeatingSelection (keyIndex, index, batch.size, 
columnVector, batch.selected);
-      } else {
-        assignDecimalNullsNoRepeatingNoSelection(keyIndex, index, batch.size, 
columnVector);
-      }
-    }
-  }
-
-  private void evaluateTimestampColumnVector(VectorizedRowBatch batch, 
TimestampColumnVector columnVector,
-      int keyIndex, int index) {
-    if (columnVector.isRepeating) {
-      if (columnVector.noNulls || !columnVector.isNull[0]) {
-        assignTimestampNoNullsRepeating(index, batch.size, columnVector);
-      } else {
-        assignTimestampNullsRepeating(keyIndex, index, batch.size, 
columnVector);
-      }
-    } else if (columnVector.noNulls) {
-      if (batch.selectedInUse) {
-        assignTimestampNoNullsNoRepeatingSelection(index, batch.size, 
columnVector, batch.selected);
-      } else {
-        assignTimestampNoNullsNoRepeatingNoSelection(index, batch.size, 
columnVector);
-      }
-    } else {
-      if (batch.selectedInUse) {
-        assignTimestampNullsNoRepeatingSelection (keyIndex, index, batch.size, 
columnVector, batch.selected);
-      } else {
-        assignTimestampNullsNoRepeatingNoSelection(keyIndex, index, 
batch.size, columnVector);
-      }
-    }
-  }
-
-  private void evaluateIntervalDayTimeColumnVector(VectorizedRowBatch batch, 
IntervalDayTimeColumnVector columnVector,
-      int keyIndex, int index) {
-    if (columnVector.isRepeating) {
-      if (columnVector.noNulls || !columnVector.isNull[0]) {
-        assignIntervalDayTimeNoNullsRepeating(index, batch.size, columnVector);
-      } else {
-        assignIntervalDayTimeNullsRepeating(keyIndex, index, batch.size, 
columnVector);
-      }
-    } else if (columnVector.noNulls) {
-      if (batch.selectedInUse) {
-        assignIntervalDayTimeNoNullsNoRepeatingSelection(index, batch.size, 
columnVector, batch.selected);
-      } else {
-        assignIntervalDayTimeNoNullsNoRepeatingNoSelection(index, batch.size, 
columnVector);
-      }
-    } else {
-      if (batch.selectedInUse) {
-        assignIntervalDayTimeNullsNoRepeatingSelection (keyIndex, index, 
batch.size, columnVector, batch.selected);
-      } else {
-        assignIntervalDayTimeNullsNoRepeatingNoSelection(keyIndex, index, 
batch.size, columnVector);
-      }
-    }
-  }
-
-  /**
-   * Helper method to assign values from a vector column into the key wrapper.
-   * Optimized for string type, possible nulls, no repeat values, batch 
selection vector.
-   */
-  private void assignStringNullsNoRepeatingSelection(int keyIndex, int index, 
int size,
-      BytesColumnVector columnVector, int[] selected) {
-    for(int i=0; i<size; ++i) {
-      int row = selected[i];
-      if (columnVector.isNull[row]) {
-        vectorHashKeyWrappers[i].assignNullString(keyIndex, index);
-      } else {
-        vectorHashKeyWrappers[i].assignString(
-            index,
-            columnVector.vector[row],
-            columnVector.start[row],
-            columnVector.length[row]);
-      }
-    }
-  }
-
-  /**
-   * Helper method to assign values from a vector column into the key wrapper.
-   * Optimized for double type, possible nulls, repeat values.
-   */
-  private void assignStringNullsRepeating(int keyIndex, int index, int size, 
BytesColumnVector columnVector) {
-    if (columnVector.isNull[0]) {
-      for(int i = 0; i < size; ++i) {
-        vectorHashKeyWrappers[i].assignNullString(keyIndex, index);
-      }
-    } else {
-      for(int i = 0; i < size; ++i) {
-        vectorHashKeyWrappers[i].assignString(
-            index,
-            columnVector.vector[0],
-            columnVector.start[0],
-            columnVector.length[0]);
-      }
-    }
-  }
-
-  /**
-   * Helper method to assign values from a vector column into the key wrapper.
-   * Optimized for string type, possible nulls, no repeat values, no selection 
vector.
-   */
-  private void assignStringNullsNoRepeatingNoSelection(int keyIndex, int 
index, int size,
-      BytesColumnVector columnVector) {
-    for(int i=0; i<size; ++i) {
-      if (columnVector.isNull[i]) {
-        vectorHashKeyWrappers[i].assignNullString(keyIndex, index);
-      } else {
-        vectorHashKeyWrappers[i].assignString(
-            index,
-            columnVector.vector[i],
-            columnVector.start[i],
-            columnVector.length[i]);
-      }
-    }
-  }
-
-  /**
-   * Helper method to assign values from a vector column into the key wrapper.
-   * Optimized for double type, no nulls, repeat values, no selection vector.
-   */
-  private void assignStringNoNullsRepeating(int index, int size,
-      BytesColumnVector columnVector) {
-    for(int i = 0; i < size; ++i) {
-      vectorHashKeyWrappers[i].assignString(
-          index,
-          columnVector.vector[0],
-          columnVector.start[0],
-          columnVector.length[0]);
-    }
-  }
-
-  /**
-   * Helper method to assign values from a vector column into the key wrapper.
-   * Optimized for double type, no nulls, no repeat values, batch selection 
vector.
-   */
-  private void assignStringNoNullsNoRepeatingSelection(int index, int size,
-      BytesColumnVector columnVector, int[] selected) {
-    for(int i=0; i<size; ++i) {
-      int row = selected[i];
-      vectorHashKeyWrappers[i].assignString(
-          index,
-          columnVector.vector[row],
-          columnVector.start[row],
-          columnVector.length[row]);
-    }
-  }
-
-  /**
-   * Helper method to assign values from a vector column into the key wrapper.
-   * Optimized for double type, no nulls, no repeat values, no selection 
vector.
-   */
-  private void assignStringNoNullsNoRepeatingNoSelection(int index, int size,
-      BytesColumnVector columnVector) {
-    for(int i=0; i<size; ++i) {
-      vectorHashKeyWrappers[i].assignString(
-          index,
-          columnVector.vector[i],
-          columnVector.start[i],
-          columnVector.length[i]);
-    }
-  }
-
-  /**
-   * Helper method to assign values from a vector column into the key wrapper.
-   * Optimized for double type, possible nulls, no repeat values, batch 
selection vector.
-   */
-  private void assignDoubleNullsNoRepeatingSelection(int keyIndex, int index, 
int size,
-      DoubleColumnVector columnVector, int[] selected) {
-    for(int i = 0; i < size; ++i) {
-      int row = selected[i];
-      if (!columnVector.isNull[row]) {
-        vectorHashKeyWrappers[i].assignDouble(index, columnVector.vector[row]);
-      } else {
-        vectorHashKeyWrappers[i].assignNullDouble(keyIndex, index);
-      }
-    }
-  }
-
-  /**
-   * Helper method to assign values from a vector column into the key wrapper.
-   * Optimized for Double type, repeat null values.
-   */
-  private void assignDoubleNullsRepeating(int keyIndex, int index, int size,
-      DoubleColumnVector columnVector) {
-    for(int r = 0; r < size; ++r) {
-      vectorHashKeyWrappers[r].assignNullDouble(keyIndex, index);
-    }
-  }
-
-  /**
-   * Helper method to assign values from a vector column into the key wrapper.
-   * Optimized for Double type, possible nulls, repeat values.
-   */
-  private void assignDoubleNullsNoRepeatingNoSelection(int keyIndex, int 
index, int size,
-      DoubleColumnVector columnVector) {
-    for(int r = 0; r < size; ++r) {
-      if (!columnVector.isNull[r]) {
-        vectorHashKeyWrappers[r].assignDouble(index, columnVector.vector[r]);
-      } else {
-        vectorHashKeyWrappers[r].assignNullDouble(keyIndex, index);
-      }
-    }
-  }
-
-  /**
-   * Helper method to assign values from a vector column into the key wrapper.
-   * Optimized for double type, no nulls, repeat values, no selection vector.
-   */
-  private void assignDoubleNoNullsRepeating(int index, int size, 
DoubleColumnVector columnVector) {
-    for(int r = 0; r < size; ++r) {
-      vectorHashKeyWrappers[r].assignDouble(index, columnVector.vector[0]);
-    }
-  }
-
-  /**
-   * Helper method to assign values from a vector column into the key wrapper.
-   * Optimized for double type, no nulls, no repeat values, batch selection 
vector.
-   */
-  private void assignDoubleNoNullsNoRepeatingSelection(int index, int size,
-      DoubleColumnVector columnVector, int[] selected) {
-    for(int r = 0; r < size; ++r) {
-      vectorHashKeyWrappers[r].assignDouble(index, 
columnVector.vector[selected[r]]);
-    }
-  }
-
-  /**
-   * Helper method to assign values from a vector column into the key wrapper.
-   * Optimized for double type, no nulls, no repeat values, no selection 
vector.
-   */
-  private void assignDoubleNoNullsNoRepeatingNoSelection(int index, int size,
-      DoubleColumnVector columnVector) {
-    for(int r = 0; r < size; ++r) {
-      vectorHashKeyWrappers[r].assignDouble(index, columnVector.vector[r]);
-    }
-  }
-
-  /**
-   * Helper method to assign values from a vector column into the key wrapper.
-   * Optimized for double type, possible nulls, no repeat values, batch 
selection vector.
-   */
-  private void assignLongNullsNoRepeatingSelection(int keyIndex, int index, 
int size,
-      LongColumnVector columnVector, int[] selected) {
-    for(int i = 0; i < size; ++i) {
-      int row = selected[i];
-      if (!columnVector.isNull[row]) {
-        vectorHashKeyWrappers[i].assignLong(index, columnVector.vector[row]);
-      } else {
-        vectorHashKeyWrappers[i].assignNullLong(keyIndex, index);
-      }
-    }
-  }
-
-  /**
-   * Helper method to assign values from a vector column into the key wrapper.
-   * Optimized for double type, repeating nulls.
-   */
-  private void assignLongNullsRepeating(int keyIndex, int index, int size,
-      LongColumnVector columnVector) {
-    for(int r = 0; r < size; ++r) {
-      vectorHashKeyWrappers[r].assignNullLong(keyIndex, index);
-    }
-  }
-
-  /**
-   * Helper method to assign values from a vector column into the key wrapper.
-   * Optimized for double type, possible nulls, no repeat values, no selection 
vector.
-   */
-  private void assignLongNullsNoRepeatingNoSelection(int keyIndex, int index, 
int size,
-      LongColumnVector columnVector) {
-    for(int r = 0; r < size; ++r) {
-      if (!columnVector.isNull[r]) {
-        vectorHashKeyWrappers[r].assignLong(index, columnVector.vector[r]);
-      } else {
-        vectorHashKeyWrappers[r].assignNullLong(keyIndex, index);
-      }
-    }
-  }
-
-  /**
-   * Helper method to assign values from a vector column into the key wrapper.
-   * Optimized for double type, no nulls, repeat values, no selection vector.
-   */
-  private void assignLongNoNullsRepeating(int index, int size, 
LongColumnVector columnVector) {
-    for(int r = 0; r < size; ++r) {
-      vectorHashKeyWrappers[r].assignLong(index, columnVector.vector[0]);
-    }
-  }
-
-  /**
-   * Helper method to assign values from a vector column into the key wrapper.
-   * Optimized for double type, no nulls, no repeat values, batch selection 
vector.
-   */
-  private void assignLongNoNullsNoRepeatingSelection(int index, int size,
-      LongColumnVector columnVector, int[] selected) {
-    for(int r = 0; r < size; ++r) {
-      vectorHashKeyWrappers[r].assignLong(index, 
columnVector.vector[selected[r]]);
-    }
-  }
-
-  /**
-   * Helper method to assign values from a vector column into the key wrapper.
-   * Optimized for double type, no nulls, no repeat values, no selection 
vector.
-   */
-  private void assignLongNoNullsNoRepeatingNoSelection(int index, int size,
-      LongColumnVector columnVector) {
-    for(int r = 0; r < size; ++r) {
-      vectorHashKeyWrappers[r].assignLong(index, columnVector.vector[r]);
-    }
-  }
-
-  /**
-   * Helper method to assign values from a vector column into the key wrapper.
-   * Optimized for Decimal type, possible nulls, no repeat values, batch 
selection vector.
-   */
-  private void assignDecimalNullsNoRepeatingSelection(int keyIndex, int index, 
int size,
-      DecimalColumnVector columnVector, int[] selected) {
-    for(int i = 0; i < size; ++i) {
-      int row = selected[i];
-      if (!columnVector.isNull[row]) {
-        vectorHashKeyWrappers[i].assignDecimal(index, 
columnVector.vector[row]);
-      } else {
-        vectorHashKeyWrappers[i].assignNullDecimal(keyIndex, index);
-      }
-    }
-  }
-
-  /**
-   * Helper method to assign values from a vector column into the key wrapper.
-   * Optimized for Decimal type, repeat null values.
-   */
-  private void assignDecimalNullsRepeating(int keyIndex, int index, int size,
-      DecimalColumnVector columnVector) {
-    for(int r = 0; r < size; ++r) {
-      vectorHashKeyWrappers[r].assignNullDecimal(keyIndex, index);
-    }
-  }
-
-  /**
-   * Helper method to assign values from a vector column into the key wrapper.
-   * Optimized for Decimal type, possible nulls, repeat values.
-   */
-  private void assignDecimalNullsNoRepeatingNoSelection(int keyIndex, int 
index, int size,
-      DecimalColumnVector columnVector) {
-    for(int r = 0; r < size; ++r) {
-      if (!columnVector.isNull[r]) {
-        vectorHashKeyWrappers[r].assignDecimal(index, columnVector.vector[r]);
-      } else {
-        vectorHashKeyWrappers[r].assignNullDecimal(keyIndex, index);
-      }
-    }
-  }
-
-  /**
-   * Helper method to assign values from a vector column into the key wrapper.
-   * Optimized for Decimal type, no nulls, repeat values, no selection vector.
-   */
-  private void assignDecimalNoNullsRepeating(int index, int size, 
DecimalColumnVector columnVector) {
-    for(int r = 0; r < size; ++r) {
-      vectorHashKeyWrappers[r].assignDecimal(index, columnVector.vector[0]);
-    }
-  }
-
-  /**
-   * Helper method to assign values from a vector column into the key wrapper.
-   * Optimized for Decimal type, no nulls, no repeat values, batch selection 
vector.
-   */
-  private void assignDecimalNoNullsNoRepeatingSelection(int index, int size,
-      DecimalColumnVector columnVector, int[] selected) {
-    for(int r = 0; r < size; ++r) {
-      vectorHashKeyWrappers[r].assignDecimal(index, 
columnVector.vector[selected[r]]);
-    }
-  }
-
-  /**
-   * Helper method to assign values from a vector column into the key wrapper.
-   * Optimized for Decimal type, no nulls, no repeat values, no selection 
vector.
-   */
-  private void assignDecimalNoNullsNoRepeatingNoSelection(int index, int size,
-      DecimalColumnVector columnVector) {
-    for(int r = 0; r < size; ++r) {
-      vectorHashKeyWrappers[r].assignDecimal(index, columnVector.vector[r]);
-    }
-  }
-
-  /**
-   * Helper method to assign values from a vector column into the key wrapper.
-   * Optimized for Timestamp type, possible nulls, no repeat values, batch 
selection vector.
-   */
-  private void assignTimestampNullsNoRepeatingSelection(int keyIndex, int 
index, int size,
-      TimestampColumnVector columnVector, int[] selected) {
-    for(int i = 0; i < size; ++i) {
-      int row = selected[i];
-      if (!columnVector.isNull[row]) {
-        vectorHashKeyWrappers[i].assignTimestamp(index, columnVector, row);
-      } else {
-        vectorHashKeyWrappers[i].assignNullTimestamp(keyIndex, index);
-      }
-    }
-  }
-
-  /**
-   * Helper method to assign values from a vector column into the key wrapper.
-   * Optimized for Timestamp type, repeat null values.
-   */
-  private void assignTimestampNullsRepeating(int keyIndex, int index, int size,
-      TimestampColumnVector columnVector) {
-    for(int r = 0; r < size; ++r) {
-      vectorHashKeyWrappers[r].assignNullTimestamp(keyIndex, index);
-    }
-  }
-
-  /**
-   * Helper method to assign values from a vector column into the key wrapper.
-   * Optimized for Timestamp type, possible nulls, repeat values.
-   */
-  private void assignTimestampNullsNoRepeatingNoSelection(int keyIndex, int 
index, int size,
-      TimestampColumnVector columnVector) {
-    for(int r = 0; r < size; ++r) {
-      if (!columnVector.isNull[r]) {
-        vectorHashKeyWrappers[r].assignTimestamp(index, columnVector, r);
-      } else {
-        vectorHashKeyWrappers[r].assignNullTimestamp(keyIndex, index);
-      }
-    }
-  }
-
-  /**
-   * Helper method to assign values from a vector column into the key wrapper.
-   * Optimized for Timestamp type, no nulls, repeat values, no selection 
vector.
-   */
-  private void assignTimestampNoNullsRepeating(int index, int size, 
TimestampColumnVector columnVector) {
-    for(int r = 0; r < size; ++r) {
-      vectorHashKeyWrappers[r].assignTimestamp(index, columnVector, 0);
-    }
-  }
-
-  /**
-   * Helper method to assign values from a vector column into the key wrapper.
-   * Optimized for Timestamp type, no nulls, no repeat values, batch selection 
vector.
-   */
-  private void assignTimestampNoNullsNoRepeatingSelection(int index, int size,
-      TimestampColumnVector columnVector, int[] selected) {
-    for(int r = 0; r < size; ++r) {
-      vectorHashKeyWrappers[r].assignTimestamp(index, columnVector, 
selected[r]);
-    }
-  }
-
-  /**
-   * Helper method to assign values from a vector column into the key wrapper.
-   * Optimized for Timestamp type, no nulls, no repeat values, no selection 
vector.
-   */
-  private void assignTimestampNoNullsNoRepeatingNoSelection(int index, int 
size,
-      TimestampColumnVector columnVector) {
-    for(int r = 0; r < size; ++r) {
-      vectorHashKeyWrappers[r].assignTimestamp(index, columnVector, r);
-    }
-  }
-
-  /**
-   * Helper method to assign values from a vector column into the key wrapper.
-   * Optimized for IntervalDayTime type, possible nulls, no repeat values, 
batch selection vector.
-   */
-  private void assignIntervalDayTimeNullsNoRepeatingSelection(int keyIndex, 
int index, int size,
-      IntervalDayTimeColumnVector columnVector, int[] selected) {
-    for(int i = 0; i < size; ++i) {
-      int row = selected[i];
-      if (!columnVector.isNull[row]) {
-        vectorHashKeyWrappers[i].assignIntervalDayTime(index, columnVector, 
row);
-      } else {
-        vectorHashKeyWrappers[i].assignNullIntervalDayTime(keyIndex, index);
-      }
-    }
-  }
-
-  /**
-   * Helper method to assign values from a vector column into the key wrapper.
-   * Optimized for IntervalDayTime type, repeat null values.
-   */
-  private void assignIntervalDayTimeNullsRepeating(int keyIndex, int index, 
int size,
-      IntervalDayTimeColumnVector columnVector) {
-    for(int r = 0; r < size; ++r) {
-      vectorHashKeyWrappers[r].assignNullIntervalDayTime(keyIndex, index);
-    }
-  }
-
-  /**
-   * Helper method to assign values from a vector column into the key wrapper.
-   * Optimized for IntervalDayTime type, possible nulls, repeat values.
-   */
-  private void assignIntervalDayTimeNullsNoRepeatingNoSelection(int keyIndex, 
int index, int size,
-      IntervalDayTimeColumnVector columnVector) {
-    for(int r = 0; r < size; ++r) {
-      if (!columnVector.isNull[r]) {
-        vectorHashKeyWrappers[r].assignIntervalDayTime(index, columnVector, r);
-      } else {
-        vectorHashKeyWrappers[r].assignNullIntervalDayTime(keyIndex, index);
-      }
-    }
-  }
-
-  /**
-   * Helper method to assign values from a vector column into the key wrapper.
-   * Optimized for IntervalDayTime type, no nulls, repeat values, no selection 
vector.
-   */
-  private void assignIntervalDayTimeNoNullsRepeating(int index, int size, 
IntervalDayTimeColumnVector columnVector) {
-    for(int r = 0; r < size; ++r) {
-      vectorHashKeyWrappers[r].assignIntervalDayTime(index, columnVector, 0);
-    }
-  }
-
-  /**
-   * Helper method to assign values from a vector column into the key wrapper.
-   * Optimized for IntervalDayTime type, no nulls, no repeat values, batch 
selection vector.
-   */
-  private void assignIntervalDayTimeNoNullsNoRepeatingSelection(int index, int 
size,
-      IntervalDayTimeColumnVector columnVector, int[] selected) {
-    for(int r = 0; r < size; ++r) {
-      vectorHashKeyWrappers[r].assignIntervalDayTime(index, columnVector, 
selected[r]);
-    }
-  }
-
-  /**
-   * Helper method to assign values from a vector column into the key wrapper.
-   * Optimized for IntervalDayTime type, no nulls, no repeat values, no 
selection vector.
-   */
-  private void assignIntervalDayTimeNoNullsNoRepeatingNoSelection(int index, 
int size,
-      IntervalDayTimeColumnVector columnVector) {
-    for(int r = 0; r < size; ++r) {
-      vectorHashKeyWrappers[r].assignIntervalDayTime(index, columnVector, r);
-    }
-  }
-
-  public static VectorHashKeyWrapperBatch 
compileKeyWrapperBatch(VectorExpression[] keyExpressions)
-      throws HiveException
-  {
-
-    final int size = keyExpressions.length;
-    TypeInfo[] typeInfos = new TypeInfo[size];
-    for (int i = 0; i < size; i++) {
-      typeInfos[i] = keyExpressions[i].getOutputTypeInfo();
-    }
-    return compileKeyWrapperBatch(keyExpressions, typeInfos);
-  }
-
-  /**
-   * Prepares a VectorHashKeyWrapperBatch to work for a specific set of keys.
-   * Computes the fast access lookup indices, preallocates all needed internal 
arrays.
-   * This step is done only once per query, not once per batch. The 
information computed now
-   * will be used to generate proper individual VectorKeyHashWrapper objects.
-   */
-  public static VectorHashKeyWrapperBatch 
compileKeyWrapperBatch(VectorExpression[] keyExpressions,
-      TypeInfo[] typeInfos)
-    throws HiveException {
-    VectorHashKeyWrapperBatch compiledKeyWrapperBatch = new 
VectorHashKeyWrapperBatch(keyExpressions.length);
-    compiledKeyWrapperBatch.keyExpressions = keyExpressions;
-
-    compiledKeyWrapperBatch.keysFixedSize = 0;
-
-    // Inspect the output type of each key expression.
-    for(int i=0; i < typeInfos.length; ++i) {
-      compiledKeyWrapperBatch.addKey(typeInfos[i]);
-    }
-    compiledKeyWrapperBatch.finishAdding();
-
-    compiledKeyWrapperBatch.vectorHashKeyWrappers =
-        new VectorHashKeyWrapper[VectorizedRowBatch.DEFAULT_SIZE];
-    for(int i=0;i<VectorizedRowBatch.DEFAULT_SIZE; ++i) {
-      compiledKeyWrapperBatch.vectorHashKeyWrappers[i] =
-          compiledKeyWrapperBatch.allocateKeyWrapper();
-    }
-
-    JavaDataModel model = JavaDataModel.get();
-
-    // Compute the fixed size overhead for the keys
-    // start with the keywrapper itself
-    compiledKeyWrapperBatch.keysFixedSize += JavaDataModel.alignUp(
-        model.object() +
-        model.ref() * MODEL_REFERENCES_COUNT +
-        model.primitive1(),
-        model.memoryAlign());
-
-    // Now add the key wrapper arrays
-    compiledKeyWrapperBatch.keysFixedSize += 
model.lengthForLongArrayOfSize(compiledKeyWrapperBatch.longIndices.length);
-    compiledKeyWrapperBatch.keysFixedSize += 
model.lengthForDoubleArrayOfSize(compiledKeyWrapperBatch.doubleIndices.length);
-    compiledKeyWrapperBatch.keysFixedSize += 
model.lengthForObjectArrayOfSize(compiledKeyWrapperBatch.stringIndices.length);
-    compiledKeyWrapperBatch.keysFixedSize += 
model.lengthForObjectArrayOfSize(compiledKeyWrapperBatch.decimalIndices.length);
-    compiledKeyWrapperBatch.keysFixedSize += 
model.lengthForObjectArrayOfSize(compiledKeyWrapperBatch.timestampIndices.length);
-    compiledKeyWrapperBatch.keysFixedSize += 
model.lengthForObjectArrayOfSize(compiledKeyWrapperBatch.intervalDayTimeIndices.length);
-    compiledKeyWrapperBatch.keysFixedSize += 
model.lengthForIntArrayOfSize(compiledKeyWrapperBatch.longIndices.length) * 2;
-    compiledKeyWrapperBatch.keysFixedSize +=
-        model.lengthForBooleanArrayOfSize(keyExpressions.length);
-
-    return compiledKeyWrapperBatch;
-  }
-
-  public VectorHashKeyWrapper allocateKeyWrapper() {
-    return VectorHashKeyWrapper.allocate(hashCtx,
-        longIndices.length,
-        doubleIndices.length,
-        stringIndices.length,
-        decimalIndices.length,
-        timestampIndices.length,
-        intervalDayTimeIndices.length,
-        keyCount);
-  }
-
-  /**
-   * Get the row-mode writable object value of a key from a key wrapper
-   * @param keyOutputWriter
-   */
-  public Object getWritableKeyValue(VectorHashKeyWrapper kw, int keyIndex,
-      VectorExpressionWriter keyOutputWriter)
-    throws HiveException {
-
-    if (kw.isNull(keyIndex)) {
-      return null;
-    }
-
-    ColumnVector.Type columnVectorType = columnVectorTypes[keyIndex];
-    int columnTypeSpecificIndex = columnTypeSpecificIndices[keyIndex];
-
-    switch (columnVectorType) {
-    case LONG:
-      return keyOutputWriter.writeValue(
-          kw.getLongValue(columnTypeSpecificIndex));
-    case DOUBLE:
-      return keyOutputWriter.writeValue(
-          kw.getDoubleValue(columnTypeSpecificIndex));
-    case BYTES:
-      return keyOutputWriter.writeValue(
-          kw.getBytes(columnTypeSpecificIndex),
-          kw.getByteStart(columnTypeSpecificIndex),
-          kw.getByteLength(columnTypeSpecificIndex));
-    case DECIMAL:
-      return keyOutputWriter.writeValue(
-          kw.getDecimal(columnTypeSpecificIndex));
-    case DECIMAL_64:
-      throw new RuntimeException("Getting writable for DECIMAL_64 not 
supported");
-    case TIMESTAMP:
-      return keyOutputWriter.writeValue(
-          kw.getTimestamp(columnTypeSpecificIndex));
-    case INTERVAL_DAY_TIME:
-      return keyOutputWriter.writeValue(
-          kw.getIntervalDayTime(columnTypeSpecificIndex));
-    default:
-      throw new HiveException("Unexpected column vector type " + 
columnVectorType);
-    }
-  }
-
-  public void setLongValue(VectorHashKeyWrapper kw, int keyIndex, Long value)
-    throws HiveException {
-
-    if (columnVectorTypes[keyIndex] != Type.LONG) {
-      throw new HiveException("Consistency error: expected LONG type; found: " 
+ columnVectorTypes[keyIndex]);
-    }
-    int columnTypeSpecificIndex = columnTypeSpecificIndices[keyIndex];
-
-    if (value == null) {
-      kw.assignNullLong(keyIndex, columnTypeSpecificIndex);
-      return;
-    }
-    kw.assignLong(keyIndex, columnTypeSpecificIndex, value);
-  }
-
-  public void assignRowColumn(VectorizedRowBatch batch, int batchIndex, int 
keyIndex,
-      VectorHashKeyWrapper kw)
-    throws HiveException {
-
-    ColumnVector colVector = batch.cols[keyIndex];
-
-    if (kw.isNull(keyIndex)) {
-      colVector.noNulls = false;
-      colVector.isNull[batchIndex] = true;
-      return;
-    }
-    colVector.isNull[batchIndex] = false;
-
-    ColumnVector.Type columnVectorType = columnVectorTypes[keyIndex];
-    int columnTypeSpecificIndex = columnTypeSpecificIndices[keyIndex];
-
-    switch (columnVectorType) {
-    case LONG:
-    case DECIMAL_64:
-      ((LongColumnVector) colVector).vector[batchIndex] =
-          kw.getLongValue(columnTypeSpecificIndex);
-      break;
-    case DOUBLE:
-      ((DoubleColumnVector) colVector).vector[batchIndex] =
-          kw.getDoubleValue(columnTypeSpecificIndex);
-      break;
-    case BYTES:
-      ((BytesColumnVector) colVector).setVal(
-          batchIndex,
-          kw.getBytes(columnTypeSpecificIndex),
-          kw.getByteStart(columnTypeSpecificIndex),
-          kw.getByteLength(columnTypeSpecificIndex));
-      break;
-    case DECIMAL:
-      ((DecimalColumnVector) colVector).set(batchIndex,
-          kw.getDecimal(columnTypeSpecificIndex));
-      break;
-    case TIMESTAMP:
-      ((TimestampColumnVector) colVector).set(
-          batchIndex, kw.getTimestamp(columnTypeSpecificIndex));
-      break;
-    case INTERVAL_DAY_TIME:
-      ((IntervalDayTimeColumnVector) colVector).set(
-          batchIndex, kw.getIntervalDayTime(columnTypeSpecificIndex));
-      break;
-    default:
-      throw new HiveException("Unexpected column vector type " + 
columnVectorType);
-    }
-  }
-
-  public int getVariableSize(int batchSize) {
-    int variableSize = 0;
-    if ( 0 < stringIndices.length) {
-      for (int k=0; k<batchSize; ++k) {
-        VectorHashKeyWrapper hkw = vectorHashKeyWrappers[k];
-        variableSize += hkw.getVariableSize();
-      }
-    }
-    return variableSize;
-  }
-}
-

http://git-wip-us.apache.org/repos/asf/hive/blob/ccdcc5e2/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/VectorMapJoinOperator.java
----------------------------------------------------------------------
diff --git 
a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/VectorMapJoinOperator.java 
b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/VectorMapJoinOperator.java
index a84bd72..2d8e1d7 100644
--- 
a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/VectorMapJoinOperator.java
+++ 
b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/VectorMapJoinOperator.java
@@ -31,6 +31,8 @@ import 
org.apache.hadoop.hive.ql.exec.persistence.MapJoinTableContainer.Reusable
 import org.apache.hadoop.hive.ql.exec.vector.expressions.VectorExpression;
 import 
org.apache.hadoop.hive.ql.exec.vector.expressions.VectorExpressionWriter;
 import 
org.apache.hadoop.hive.ql.exec.vector.expressions.VectorExpressionWriterFactory;
+import org.apache.hadoop.hive.ql.exec.vector.wrapper.VectorHashKeyWrapperBase;
+import org.apache.hadoop.hive.ql.exec.vector.wrapper.VectorHashKeyWrapperBatch;
 import org.apache.hadoop.hive.ql.metadata.HiveException;
 import org.apache.hadoop.hive.ql.plan.ExprNodeDesc;
 import org.apache.hadoop.hive.ql.plan.MapJoinDesc;
@@ -69,7 +71,7 @@ public class VectorMapJoinOperator extends 
VectorMapJoinBaseOperator {
   // for the inner-loop supper.processOp callbacks
   //
   private transient int batchIndex;
-  private transient VectorHashKeyWrapper[] keyValues;
+  private transient VectorHashKeyWrapperBase[] keyValues;
   private transient VectorHashKeyWrapperBatch keyWrapperBatch;
   private transient VectorExpressionWriter[] keyOutputWriters;
 

http://git-wip-us.apache.org/repos/asf/hive/blob/ccdcc5e2/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/VectorSMBMapJoinOperator.java
----------------------------------------------------------------------
diff --git 
a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/VectorSMBMapJoinOperator.java
 
b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/VectorSMBMapJoinOperator.java
index 35f810f..c13510e 100644
--- 
a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/VectorSMBMapJoinOperator.java
+++ 
b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/VectorSMBMapJoinOperator.java
@@ -30,6 +30,8 @@ import org.apache.hadoop.hive.ql.exec.SMBMapJoinOperator;
 import org.apache.hadoop.hive.ql.exec.vector.expressions.VectorExpression;
 import 
org.apache.hadoop.hive.ql.exec.vector.expressions.VectorExpressionWriter;
 import 
org.apache.hadoop.hive.ql.exec.vector.expressions.VectorExpressionWriterFactory;
+import org.apache.hadoop.hive.ql.exec.vector.wrapper.VectorHashKeyWrapperBase;
+import org.apache.hadoop.hive.ql.exec.vector.wrapper.VectorHashKeyWrapperBatch;
 import org.apache.hadoop.hive.ql.metadata.HiveException;
 import org.apache.hadoop.hive.ql.plan.ExprNodeDesc;
 import org.apache.hadoop.hive.ql.plan.OperatorDesc;
@@ -87,14 +89,14 @@ public class VectorSMBMapJoinOperator extends 
SMBMapJoinOperator
 
   private transient int batchIndex = -1;
 
-  private transient VectorHashKeyWrapper[] keyValues;
+  private transient VectorHashKeyWrapperBase[] keyValues;
 
   private transient SMBJoinKeyEvaluator keyEvaluator;
 
   private transient VectorExpressionWriter[] valueWriters;
 
   private interface SMBJoinKeyEvaluator {
-    List<Object> evaluate(VectorHashKeyWrapper kw) throws HiveException;
+    List<Object> evaluate(VectorHashKeyWrapperBase kw) throws HiveException;
 }
 
   /** Kryo ctor. */
@@ -193,7 +195,7 @@ public class VectorSMBMapJoinOperator extends 
SMBMapJoinOperator
       }
 
       @Override
-      public List<Object> evaluate(VectorHashKeyWrapper kw) throws 
HiveException {
+      public List<Object> evaluate(VectorHashKeyWrapperBase kw) throws 
HiveException {
         for(int i = 0; i < keyExpressions.length; ++i) {
           key.set(i, keyWrapperBatch.getWritableKeyValue(kw, i, 
keyOutputWriters[i]));
         }

Reply via email to