http://git-wip-us.apache.org/repos/asf/hive/blob/f923db0b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/mapjoin/VectorMapJoinCommonOperator.java
----------------------------------------------------------------------
diff --git 
a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/mapjoin/VectorMapJoinCommonOperator.java
 
b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/mapjoin/VectorMapJoinCommonOperator.java
index c288731..77b44fb 100644
--- 
a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/mapjoin/VectorMapJoinCommonOperator.java
+++ 
b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/mapjoin/VectorMapJoinCommonOperator.java
@@ -22,6 +22,7 @@ import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.List;
 import java.util.Map;
+
 import org.apache.commons.lang.ArrayUtils;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -56,6 +57,7 @@ import org.apache.hadoop.hive.ql.plan.MapJoinDesc;
 import org.apache.hadoop.hive.ql.plan.OperatorDesc;
 import org.apache.hadoop.hive.ql.plan.VectorMapJoinDesc;
 import 
org.apache.hadoop.hive.ql.plan.VectorMapJoinDesc.HashTableImplementationType;
+import org.apache.hadoop.hive.ql.plan.VectorMapJoinInfo;
 import org.apache.hadoop.hive.ql.plan.api.OperatorType;
 import org.apache.hadoop.hive.serde2.lazybinary.fast.LazyBinaryDeserializeRead;
 import org.apache.hadoop.hive.serde2.objectinspector.StructField;
@@ -63,6 +65,8 @@ import 
org.apache.hadoop.hive.serde2.objectinspector.StructObjectInspector;
 import org.apache.hadoop.hive.serde2.typeinfo.TypeInfo;
 import org.apache.hadoop.hive.serde2.typeinfo.TypeInfoUtils;
 
+import com.google.common.base.Preconditions;
+
 /**
  * This class is common operator class for native vectorized map join.
  *
@@ -72,7 +76,43 @@ import org.apache.hadoop.hive.serde2.typeinfo.TypeInfoUtils;
  */
 public abstract class VectorMapJoinCommonOperator extends MapJoinOperator 
implements VectorizationContextRegion {
   private static final long serialVersionUID = 1L;
-  private static final Logger LOG = 
LoggerFactory.getLogger(VectorMapJoinCommonOperator.class.getName());
+
+  
//------------------------------------------------------------------------------------------------
+
+  private static final String CLASS_NAME = 
VectorMapJoinCommonOperator.class.getName();
+private static final Logger LOG = LoggerFactory.getLogger(CLASS_NAME);
+
+  protected abstract String getLoggingPrefix();
+
+  // For debug tracing: information about the map or reduce task, operator, 
operator class, etc.
+  protected transient String loggingPrefix;
+
+  protected String getLoggingPrefix(String className) {
+    if (loggingPrefix == null) {
+      initLoggingPrefix(className);
+    }
+    return loggingPrefix;
+  }
+
+  protected void initLoggingPrefix(String className) {
+    if (hconf == null) {
+      // Constructor time...
+      loggingPrefix = className;
+    } else {
+      // Determine the name of our map or reduce task for debug tracing.
+      BaseWork work = Utilities.getMapWork(hconf);
+      if (work == null) {
+        work = Utilities.getReduceWork(hconf);
+      }
+      loggingPrefix = className + " " + work.getName() + " " + getOperatorId();
+    }
+  }
+
+  
//------------------------------------------------------------------------------------------------
+
+  protected VectorMapJoinDesc vectorDesc;
+
+  protected VectorMapJoinInfo vectorMapJoinInfo;
 
   // Whether this operator is an outer join.
   protected boolean isOuterJoin;
@@ -88,10 +128,10 @@ public abstract class VectorMapJoinCommonOperator extends 
MapJoinOperator implem
   // a mixture of input big table columns and new scratch columns.
   protected VectorizationContext vOutContext;
 
-  // The output column projection of the vectorized row batch.  And, the type 
names of the output
+  // The output column projection of the vectorized row batch.  And, the type 
infos of the output
   // columns.
   protected int[] outputProjection;
-  protected String[] outputTypeNames;
+  protected TypeInfo[] outputTypeInfos;
 
   // These are the vectorized batch expressions for filtering, key 
expressions, and value
   // expressions.
@@ -101,15 +141,17 @@ public abstract class VectorMapJoinCommonOperator extends 
MapJoinOperator implem
 
   // This is map of which vectorized row batch columns are the big table key 
columns.  Since
   // we may have key expressions that produce new scratch columns, we need a 
mapping.
-  // And, we have their type names.
+  // And, we have their type infos.
   protected int[] bigTableKeyColumnMap;
-  protected ArrayList<String> bigTableKeyTypeNames;
+  protected String[] bigTableKeyColumnNames;
+  protected TypeInfo[] bigTableKeyTypeInfos;
 
   // Similarly, this is map of which vectorized row batch columns are the big 
table value columns.
   // Since we may have value expressions that produce new scratch columns, we 
need a mapping.
-  // And, we have their type names.
+  // And, we have their type infos.
   protected int[] bigTableValueColumnMap;
-  protected ArrayList<String> bigTableValueTypeNames;
+  protected String[] bigTableValueColumnNames;
+  protected TypeInfo[] bigTableValueTypeInfos;
 
   // This is a mapping of which big table columns (input and key/value 
expressions) will be
   // part of the big table portion of the join output result.
@@ -124,6 +166,8 @@ public abstract class VectorMapJoinCommonOperator extends 
MapJoinOperator implem
   // to output batch scratch columns for the small table portion.
   protected VectorColumnSourceMapping smallTableMapping;
 
+  protected VectorColumnSourceMapping projectionMapping;
+
   // These are the output columns for the small table and the outer small 
table keys.
   protected int[] smallTableOutputVectorColumns;
   protected int[] bigTableOuterKeyOutputVectorColumns;
@@ -137,9 +181,6 @@ public abstract class VectorMapJoinCommonOperator extends 
MapJoinOperator implem
   // transient.
   //---------------------------------------------------------------------------
 
-  // For debug tracing: the name of the map or reduce task.
-  protected transient String taskName;
-
   // The threshold where we should use a repeating vectorized row batch 
optimization for
   // generating join output results.
   protected transient boolean useOverflowRepeatedThreshold;
@@ -192,6 +233,9 @@ public abstract class VectorMapJoinCommonOperator extends 
MapJoinOperator implem
 
     MapJoinDesc desc = (MapJoinDesc) conf;
     this.conf = desc;
+    vectorDesc = (VectorMapJoinDesc) desc.getVectorDesc();
+    vectorMapJoinInfo = vectorDesc.getVectorMapJoinInfo();
+    Preconditions.checkState(vectorMapJoinInfo != null);
 
     this.vContext = vContext;
 
@@ -210,214 +254,28 @@ public abstract class VectorMapJoinCommonOperator 
extends MapJoinOperator implem
     bigTableFilterExpressions = 
vContext.getVectorExpressions(filterExpressions.get(posBigTable),
         VectorExpressionDescriptor.Mode.FILTER);
 
-    List<ExprNodeDesc> keyDesc = desc.getKeys().get(posBigTable);
-    bigTableKeyExpressions = vContext.getVectorExpressions(keyDesc);
-
-    // Since a key expression can be a calculation and the key will go into a 
scratch column,
-    // we need the mapping and type information.
-    bigTableKeyColumnMap = new int[bigTableKeyExpressions.length];
-    bigTableKeyTypeNames = new ArrayList<String>();
-    boolean onlyColumns = true;
-    for (int i = 0; i < bigTableKeyColumnMap.length; i++) {
-      VectorExpression ve = bigTableKeyExpressions[i];
-      if (!IdentityExpression.isColumnOnly(ve)) {
-        onlyColumns = false;
-      }
-      bigTableKeyTypeNames.add(keyDesc.get(i).getTypeString());
-      bigTableKeyColumnMap[i] = ve.getOutputColumn();
-    }
-    if (onlyColumns) {
-      bigTableKeyExpressions = null;
-    }
-
-    List<ExprNodeDesc> bigTableExprs = desc.getExprs().get(posBigTable);
-    bigTableValueExpressions = vContext.getVectorExpressions(bigTableExprs);
-
-    /*
-     * Similarly, we need a mapping since a value expression can be a 
calculation and the value
-     * will go into a scratch column.
-     */
-    bigTableValueColumnMap = new int[bigTableValueExpressions.length];
-    bigTableValueTypeNames = new ArrayList<String>();
-    onlyColumns = true;
-    for (int i = 0; i < bigTableValueColumnMap.length; i++) {
-      VectorExpression ve = bigTableValueExpressions[i];
-      if (!IdentityExpression.isColumnOnly(ve)) {
-        onlyColumns = false;
-      }
-      bigTableValueTypeNames.add(bigTableExprs.get(i).getTypeString());
-      bigTableValueColumnMap[i] = ve.getOutputColumn();
-    }
-    if (onlyColumns) {
-      bigTableValueExpressions = null;
-    }
-
-    determineCommonInfo(isOuterJoin);
-  }
-
-  protected void determineCommonInfo(boolean isOuter) throws HiveException {
-
-    bigTableRetainedMapping = new VectorColumnOutputMapping("Big Table 
Retained Mapping");
-
-    bigTableOuterKeyMapping = new VectorColumnOutputMapping("Big Table Outer 
Key Mapping");
-
-    // The order of the fields in the LazyBinary small table value must be 
used, so
-    // we use the source ordering flavor for the mapping.
-    smallTableMapping = new VectorColumnSourceMapping("Small Table Mapping");
-
-    // We use a mapping object here so we can build the projection in any 
order and
-    // get the ordered by 0 to n-1 output columns at the end.
-    //
-    // Also, to avoid copying a big table key into the small table result area 
for inner joins,
-    // we reference it with the projection so there can be duplicate output 
columns
-    // in the projection.
-    VectorColumnSourceMapping projectionMapping = new 
VectorColumnSourceMapping("Projection Mapping");
-
-    /*
-     * Gather up big and small table output result information from the 
MapJoinDesc.
-     */
-    List<Integer> bigTableRetainList = conf.getRetainList().get(posBigTable);
-    int bigTableRetainSize = bigTableRetainList.size();
-
-    int[] smallTableIndices;
-    int smallTableIndicesSize;
-    List<ExprNodeDesc> smallTableExprs = 
conf.getExprs().get(posSingleVectorMapJoinSmallTable);
-    if (conf.getValueIndices() != null && 
conf.getValueIndices().get(posSingleVectorMapJoinSmallTable) != null) {
-      smallTableIndices = 
conf.getValueIndices().get(posSingleVectorMapJoinSmallTable);
-      smallTableIndicesSize = smallTableIndices.length;
-    } else {
-      smallTableIndices = null;
-      smallTableIndicesSize = 0;
-    }
-
-    List<Integer> smallTableRetainList = 
conf.getRetainList().get(posSingleVectorMapJoinSmallTable);
-    int smallTableRetainSize = smallTableRetainList.size();
-
-    int smallTableResultSize = 0;
-    if (smallTableIndicesSize > 0) {
-      smallTableResultSize = smallTableIndicesSize;
-    } else if (smallTableRetainSize > 0) {
-      smallTableResultSize = smallTableRetainSize;
-    }
-
-    /*
-     * Determine the big table retained mapping first so we can optimize out 
(with
-     * projection) copying inner join big table keys in the subsequent small 
table results section.
-     */
-    int nextOutputColumn = (order[0] == posBigTable ? 0 : 
smallTableResultSize);
-    for (int i = 0; i < bigTableRetainSize; i++) {
-
-      // Since bigTableValueExpressions may do a calculation and produce a 
scratch column, we
-      // need to map to the right batch column.
-
-      int retainColumn = bigTableRetainList.get(i);
-      int batchColumnIndex = bigTableValueColumnMap[retainColumn];
-      String typeName = bigTableValueTypeNames.get(i);
-
-      // With this map we project the big table batch to make it look like an 
output batch.
-      projectionMapping.add(nextOutputColumn, batchColumnIndex, typeName);
-
-      // Collect columns we copy from the big table batch to the overflow 
batch.
-      if (!bigTableRetainedMapping.containsOutputColumn(batchColumnIndex)) {
-        // Tolerate repeated use of a big table column.
-        bigTableRetainedMapping.add(batchColumnIndex, batchColumnIndex, 
typeName);
-      }
-
-      nextOutputColumn++;
-    }
-
-    /*
-     * Now determine the small table results.
-     */
-    int firstSmallTableOutputColumn;
-    firstSmallTableOutputColumn = (order[0] == posBigTable ? 
bigTableRetainSize : 0);
-    int smallTableOutputCount = 0;
-    nextOutputColumn = firstSmallTableOutputColumn;
-
-    // Small table indices has more information (i.e. keys) than retain, so 
use it if it exists...
-    if (smallTableIndicesSize > 0) {
-      smallTableOutputCount = smallTableIndicesSize;
-
-      for (int i = 0; i < smallTableIndicesSize; i++) {
-        if (smallTableIndices[i] >= 0) {
-
-          // Zero and above numbers indicate a big table key is needed for
-          // small table result "area".
-
-          int keyIndex = smallTableIndices[i];
-
-          // Since bigTableKeyExpressions may do a calculation and produce a 
scratch column, we
-          // need to map the right column.
-          int batchKeyColumn = bigTableKeyColumnMap[keyIndex];
-          String typeName = bigTableKeyTypeNames.get(keyIndex);
-
-          if (!isOuter) {
-
-            // Optimize inner join keys of small table results.
-
-            // Project the big table key into the small table result "area".
-            projectionMapping.add(nextOutputColumn, batchKeyColumn, typeName);
-
-            if (!bigTableRetainedMapping.containsOutputColumn(batchKeyColumn)) 
{
-              // If necessary, copy the big table key into the overflow 
batch's small table
-              // result "area".
-              bigTableRetainedMapping.add(batchKeyColumn, batchKeyColumn, 
typeName);
-            }
-          } else {
-
-            // For outer joins, since the small table key can be null when 
there is no match,
-            // we must have a physical (scratch) column for those keys.  We 
cannot use the
-            // projection optimization used by inner joins above.
-
-            int scratchColumn = vOutContext.allocateScratchColumn(typeName);
-            projectionMapping.add(nextOutputColumn, scratchColumn, typeName);
-
-            bigTableRetainedMapping.add(batchKeyColumn, scratchColumn, 
typeName);
+    bigTableKeyColumnMap = vectorMapJoinInfo.getBigTableKeyColumnMap();
+    bigTableKeyColumnNames = vectorMapJoinInfo.getBigTableKeyColumnNames();
+    bigTableKeyTypeInfos = vectorMapJoinInfo.getBigTableKeyTypeInfos();
+    bigTableKeyExpressions = vectorMapJoinInfo.getBigTableKeyExpressions();
 
-            bigTableOuterKeyMapping.add(batchKeyColumn, scratchColumn, 
typeName);
-          }
-        } else {
+    bigTableValueColumnMap = vectorMapJoinInfo.getBigTableValueColumnMap();
+    bigTableValueColumnNames = vectorMapJoinInfo.getBigTableValueColumnNames();
+    bigTableValueTypeInfos = vectorMapJoinInfo.getBigTableValueTypeInfos();
+    bigTableValueExpressions = vectorMapJoinInfo.getBigTableValueExpressions();
 
-          // Negative numbers indicate a column to be (deserialize) read from 
the small table's
-          // LazyBinary value row.
-          int smallTableValueIndex = -smallTableIndices[i] - 1;
+    bigTableRetainedMapping = vectorMapJoinInfo.getBigTableRetainedMapping();
 
-          String typeName = smallTableExprs.get(i).getTypeString();
+    bigTableOuterKeyMapping =  vectorMapJoinInfo.getBigTableOuterKeyMapping();
 
-          // Make a new big table scratch column for the small table value.
-          int scratchColumn = vOutContext.allocateScratchColumn(typeName);
-          projectionMapping.add(nextOutputColumn, scratchColumn, typeName);
+    smallTableMapping = vectorMapJoinInfo.getSmallTableMapping();
 
-          smallTableMapping.add(smallTableValueIndex, scratchColumn, typeName);
-        }
-        nextOutputColumn++;
-      }
-    } else if (smallTableRetainSize > 0) {
-      smallTableOutputCount = smallTableRetainSize;
-
-      // Only small table values appear in join output result.
-
-      for (int i = 0; i < smallTableRetainSize; i++) {
-        int smallTableValueIndex = smallTableRetainList.get(i);
-
-        // Make a new big table scratch column for the small table value.
-        String typeName = smallTableExprs.get(i).getTypeString();
-        int scratchColumn = vOutContext.allocateScratchColumn(typeName);
+    projectionMapping = vectorMapJoinInfo.getProjectionMapping();
 
-        projectionMapping.add(nextOutputColumn, scratchColumn, typeName);
-
-        smallTableMapping.add(smallTableValueIndex, scratchColumn, typeName);
-        nextOutputColumn++;
-      }
-    }
-
-    // Convert dynamic arrays and maps to simple arrays.
-
-    bigTableRetainedMapping.finalize();
-
-    bigTableOuterKeyMapping.finalize();
+    determineCommonInfo(isOuterJoin);
+  }
 
-    smallTableMapping.finalize();
+  protected void determineCommonInfo(boolean isOuter) throws HiveException {
 
     bigTableOuterKeyOutputVectorColumns = 
bigTableOuterKeyMapping.getOutputColumns();
     smallTableOutputVectorColumns = smallTableMapping.getOutputColumns();
@@ -429,46 +287,37 @@ public abstract class VectorMapJoinCommonOperator extends 
MapJoinOperator implem
 
     smallTableByteColumnVectorColumns = 
getByteColumnVectorColumns(smallTableMapping);
 
-    projectionMapping.finalize();
-
-    // Verify we added an entry for each output.
-    assert projectionMapping.isSourceSequenceGood();
-
     outputProjection = projectionMapping.getOutputColumns();
-    outputTypeNames = projectionMapping.getTypeNames();
+    outputTypeInfos = projectionMapping.getTypeInfos();
 
     if (isLogDebugEnabled) {
       int[] orderDisplayable = new int[order.length];
       for (int i = 0; i < order.length; i++) {
         orderDisplayable[i] = (int) order[i];
       }
-      LOG.debug(taskName + ", " + getOperatorId() + " 
VectorMapJoinCommonOperator constructor order " + 
Arrays.toString(orderDisplayable));
-      LOG.debug(taskName + ", " + getOperatorId() + " 
VectorMapJoinCommonOperator constructor posBigTable " + (int) posBigTable);
-      LOG.debug(taskName + ", " + getOperatorId() + " 
VectorMapJoinCommonOperator constructor posSingleVectorMapJoinSmallTable " + 
(int) posSingleVectorMapJoinSmallTable);
-
-      LOG.debug(taskName + ", " + getOperatorId() + " 
VectorMapJoinCommonOperator constructor bigTableKeyColumnMap " + 
Arrays.toString(bigTableKeyColumnMap));
-      LOG.debug(taskName + ", " + getOperatorId() + " 
VectorMapJoinCommonOperator constructor bigTableKeyTypeNames " + 
bigTableKeyTypeNames);
+      LOG.debug(getLoggingPrefix() + " VectorMapJoinCommonOperator constructor 
order " + Arrays.toString(orderDisplayable));
+      LOG.debug(getLoggingPrefix() + " VectorMapJoinCommonOperator constructor 
posBigTable " + (int) posBigTable);
+      LOG.debug(getLoggingPrefix() + " VectorMapJoinCommonOperator constructor 
posSingleVectorMapJoinSmallTable " + (int) posSingleVectorMapJoinSmallTable);
 
-      LOG.debug(taskName + ", " + getOperatorId() + " 
VectorMapJoinCommonOperator constructor bigTableValueColumnMap " + 
Arrays.toString(bigTableValueColumnMap));
-      LOG.debug(taskName + ", " + getOperatorId() + " 
VectorMapJoinCommonOperator constructor bigTableValueTypeNames " + 
bigTableValueTypeNames);
+      LOG.debug(getLoggingPrefix() + " VectorMapJoinCommonOperator constructor 
bigTableKeyColumnMap " + Arrays.toString(bigTableKeyColumnMap));
+      LOG.debug(getLoggingPrefix() + " VectorMapJoinCommonOperator constructor 
bigTableKeyColumnNames " + Arrays.toString(bigTableKeyColumnNames));
+      LOG.debug(getLoggingPrefix() + " VectorMapJoinCommonOperator constructor 
bigTableKeyTypeInfos " + Arrays.toString(bigTableKeyTypeInfos));
 
-      LOG.debug(taskName + ", " + getOperatorId() + " 
VectorMapJoinCommonOperator constructor smallTableIndices " + 
Arrays.toString(smallTableIndices));
-      LOG.debug(taskName + ", " + getOperatorId() + " 
VectorMapJoinCommonOperator constructor smallTableRetainList " + 
smallTableRetainList);
+      LOG.debug(getLoggingPrefix() + " VectorMapJoinCommonOperator constructor 
bigTableValueColumnMap " + Arrays.toString(bigTableValueColumnMap));
+      LOG.debug(getLoggingPrefix() + " VectorMapJoinCommonOperator constructor 
bigTableValueColumnNames " + Arrays.toString(bigTableValueColumnNames));
+      LOG.debug(getLoggingPrefix() + " VectorMapJoinCommonOperator constructor 
bigTableValueTypeNames " + Arrays.toString(bigTableValueTypeInfos));
 
-      LOG.debug(taskName + ", " + getOperatorId() + " 
VectorMapJoinCommonOperator constructor firstSmallTableOutputColumn " + 
firstSmallTableOutputColumn);
-      LOG.debug(taskName + ", " + getOperatorId() + " 
VectorMapJoinCommonOperator constructor smallTableOutputCount " + 
smallTableOutputCount);
+      LOG.debug(getLoggingPrefix() + " VectorMapJoinCommonOperator constructor 
bigTableRetainedMapping " + bigTableRetainedMapping.toString());
 
-      LOG.debug(taskName + ", " + getOperatorId() + " 
VectorMapJoinCommonOperator constructor bigTableRetainedMapping " + 
bigTableRetainedMapping.toString());
+      LOG.debug(getLoggingPrefix() + " VectorMapJoinCommonOperator constructor 
bigTableOuterKeyMapping " + bigTableOuterKeyMapping.toString());
 
-      LOG.debug(taskName + ", " + getOperatorId() + " 
VectorMapJoinCommonOperator constructor bigTableOuterKeyMapping " + 
bigTableOuterKeyMapping.toString());
+      LOG.debug(getLoggingPrefix() + " VectorMapJoinCommonOperator constructor 
smallTableMapping " + smallTableMapping.toString());
 
-      LOG.debug(taskName + ", " + getOperatorId() + " 
VectorMapJoinCommonOperator constructor smallTableMapping " + 
smallTableMapping.toString());
+      LOG.debug(getLoggingPrefix() + " VectorMapJoinCommonOperator constructor 
bigTableByteColumnVectorColumns " + 
Arrays.toString(bigTableByteColumnVectorColumns));
+      LOG.debug(getLoggingPrefix() + " VectorMapJoinCommonOperator constructor 
smallTableByteColumnVectorColumns " + 
Arrays.toString(smallTableByteColumnVectorColumns));
 
-      LOG.debug(taskName + ", " + getOperatorId() + " 
VectorMapJoinCommonOperator constructor bigTableByteColumnVectorColumns " + 
Arrays.toString(bigTableByteColumnVectorColumns));
-      LOG.debug(taskName + ", " + getOperatorId() + " 
VectorMapJoinCommonOperator constructor smallTableByteColumnVectorColumns " + 
Arrays.toString(smallTableByteColumnVectorColumns));
-
-      LOG.debug(taskName + ", " + getOperatorId() + " 
VectorMapJoinCommonOperator constructor outputProjection " + 
Arrays.toString(outputProjection));
-      LOG.debug(taskName + ", " + getOperatorId() + " 
VectorMapJoinCommonOperator constructor outputTypeNames " + 
Arrays.toString(outputTypeNames));
+      LOG.debug(getLoggingPrefix() + " VectorMapJoinCommonOperator constructor 
outputProjection " + Arrays.toString(outputProjection));
+      LOG.debug(getLoggingPrefix() + " VectorMapJoinCommonOperator constructor 
outputTypeInfos " + Arrays.toString(outputTypeInfos));
     }
 
     setupVOutContext(conf.getOutputColumnNames());
@@ -482,10 +331,10 @@ public abstract class VectorMapJoinCommonOperator extends 
MapJoinOperator implem
     ArrayList<Integer> list = new ArrayList<Integer>();
     int count = mapping.getCount();
     int[] outputColumns = mapping.getOutputColumns();
-    String[] typeNames = mapping.getTypeNames();
+    TypeInfo[] typeInfos = mapping.getTypeInfos();
     for (int i = 0; i < count; i++) {
       int outputColumn = outputColumns[i];
-      String typeName = typeNames[i];
+      String typeName = typeInfos[i].getTypeName();
       if (VectorizationContext.isStringFamily(typeName)) {
         list.add(outputColumn);
       }
@@ -500,10 +349,10 @@ public abstract class VectorMapJoinCommonOperator extends 
MapJoinOperator implem
    */
   protected void setupVOutContext(List<String> outputColumnNames) {
     if (isLogDebugEnabled) {
-      LOG.debug(taskName + ", " + getOperatorId() + " 
VectorMapJoinCommonOperator constructor outputColumnNames " + 
outputColumnNames);
+      LOG.debug(getLoggingPrefix() + " VectorMapJoinCommonOperator constructor 
outputColumnNames " + outputColumnNames);
     }
     if (outputColumnNames.size() != outputProjection.length) {
-      throw new RuntimeException("Output column names " + outputColumnNames + 
" length and output projection " + Arrays.toString(outputProjection) + " / " + 
Arrays.toString(outputTypeNames) + " length mismatch");
+      throw new RuntimeException("Output column names " + outputColumnNames + 
" length and output projection " + Arrays.toString(outputProjection) + " / " + 
Arrays.toString(outputTypeInfos) + " length mismatch");
     }
     vOutContext.resetProjectionColumns();
     for (int i = 0; i < outputColumnNames.size(); ++i) {
@@ -512,7 +361,7 @@ public abstract class VectorMapJoinCommonOperator extends 
MapJoinOperator implem
       vOutContext.addProjectionColumn(columnName, outputColumn);
 
       if (isLogDebugEnabled) {
-        LOG.debug(taskName + ", " + getOperatorId() + " 
VectorMapJoinCommonOperator constructor addProjectionColumn " + i + " 
columnName " + columnName + " outputColumn " + outputColumn);
+        LOG.debug(getLoggingPrefix() + " VectorMapJoinCommonOperator 
constructor addProjectionColumn " + i + " columnName " + columnName + " 
outputColumn " + outputColumn);
       }
     }
   }
@@ -522,7 +371,7 @@ public abstract class VectorMapJoinCommonOperator extends 
MapJoinOperator implem
    */
   @Override
   protected HashTableLoader getHashTableLoader(Configuration hconf) {
-    VectorMapJoinDesc vectorDesc = conf.getVectorDesc();
+    VectorMapJoinDesc vectorDesc = (VectorMapJoinDesc) conf.getVectorDesc();
     HashTableImplementationType hashTableImplementationType = 
vectorDesc.hashTableImplementationType();
     HashTableLoader hashTableLoader;
     switch (vectorDesc.hashTableImplementationType()) {
@@ -546,15 +395,6 @@ public abstract class VectorMapJoinCommonOperator extends 
MapJoinOperator implem
   protected void initializeOp(Configuration hconf) throws HiveException {
     super.initializeOp(hconf);
 
-    if (isLogDebugEnabled) {
-      // Determine the name of our map or reduce task for debug tracing.
-      BaseWork work = Utilities.getMapWork(hconf);
-      if (work == null) {
-        work = Utilities.getReduceWork(hconf);
-      }
-      taskName = work.getName();
-    }
-
     /*
      * Get configuration parameters.
      */
@@ -570,9 +410,8 @@ public abstract class VectorMapJoinCommonOperator extends 
MapJoinOperator implem
       smallTableVectorDeserializeRow =
           new VectorDeserializeRow<LazyBinaryDeserializeRead>(
               new LazyBinaryDeserializeRead(
-                  VectorizedBatchUtil.typeInfosFromTypeNames(
-                      smallTableMapping.getTypeNames()),
-                      /* useExternalBuffer */ true));
+                  smallTableMapping.getTypeInfos(),
+                  /* useExternalBuffer */ true));
       
smallTableVectorDeserializeRow.init(smallTableMapping.getOutputColumns());
     }
 
@@ -596,13 +435,13 @@ public abstract class VectorMapJoinCommonOperator extends 
MapJoinOperator implem
 
     if (isLogDebugEnabled) {
       int[] currentScratchColumns = vOutContext.currentScratchColumns();
-      LOG.debug(taskName + ", " + getOperatorId() + " 
VectorMapJoinCommonOperator initializeOp currentScratchColumns " + 
Arrays.toString(currentScratchColumns));
+      LOG.debug(getLoggingPrefix() + " VectorMapJoinCommonOperator 
initializeOp currentScratchColumns " + Arrays.toString(currentScratchColumns));
 
       StructObjectInspector structOutputObjectInspector = 
(StructObjectInspector) outputObjInspector;
       List<? extends StructField> fields = 
structOutputObjectInspector.getAllStructFieldRefs();
       int i = 0;
       for (StructField field : fields) {
-        LOG.debug("VectorMapJoinInnerBigOnlyCommonOperator initializeOp " + i 
+ " field " + field.getFieldName() + " type " + 
field.getFieldObjectInspector().getTypeName());
+        LOG.debug(getLoggingPrefix() + " VectorMapJoinCommonOperator 
initializeOp " + i + " field " + field.getFieldName() + " type " + 
field.getFieldObjectInspector().getTypeName());
         i++;
       }
     }
@@ -613,7 +452,7 @@ public abstract class VectorMapJoinCommonOperator extends 
MapJoinOperator implem
     // setup mapJoinTables and serdes
     super.completeInitializationOp(os);
 
-    VectorMapJoinDesc vectorDesc = conf.getVectorDesc();
+    VectorMapJoinDesc vectorDesc = (VectorMapJoinDesc) conf.getVectorDesc();
     HashTableImplementationType hashTableImplementationType = 
vectorDesc.hashTableImplementationType();
     switch (vectorDesc.hashTableImplementationType()) {
     case OPTIMIZED:
@@ -655,7 +494,7 @@ public abstract class VectorMapJoinCommonOperator extends 
MapJoinOperator implem
     // First, just allocate just the projection columns we will be using.
     for (int i = 0; i < outputProjection.length; i++) {
       int outputColumn = outputProjection[i];
-      String typeName = outputTypeNames[i];
+      String typeName = outputTypeInfos[i].getTypeName();
       allocateOverflowBatchColumnVector(overflowBatch, outputColumn, typeName);
     }
 
@@ -687,7 +526,7 @@ public abstract class VectorMapJoinCommonOperator extends 
MapJoinOperator implem
       overflowBatch.cols[outputColumn] = 
VectorizedBatchUtil.createColumnVector(typeInfo);
 
       if (isLogDebugEnabled) {
-        LOG.debug(taskName + ", " + getOperatorId() + " 
VectorMapJoinCommonOperator initializeOp overflowBatch outputColumn " + 
outputColumn + " class " + 
overflowBatch.cols[outputColumn].getClass().getSimpleName());
+        LOG.debug(getLoggingPrefix() + " VectorMapJoinCommonOperator 
initializeOp overflowBatch outputColumn " + outputColumn + " class " + 
overflowBatch.cols[outputColumn].getClass().getSimpleName());
       }
     }
   }
@@ -724,9 +563,9 @@ public abstract class VectorMapJoinCommonOperator extends 
MapJoinOperator implem
   }
 
   protected void displayBatchColumns(VectorizedRowBatch batch, String 
batchName) {
-    LOG.debug("commonSetup " + batchName + " column count " + batch.numCols);
+    LOG.debug(getLoggingPrefix() + " VectorMapJoinCommonOperator commonSetup " 
+ batchName + " column count " + batch.numCols);
     for (int column = 0; column < batch.numCols; column++) {
-      LOG.debug("commonSetup " + batchName + "     column " + column + " type 
" + (batch.cols[column] == null ? "NULL" : 
batch.cols[column].getClass().getSimpleName()));
+      LOG.debug(getLoggingPrefix() + " VectorMapJoinCommonOperator commonSetup 
" + batchName + "     column " + column + " type " + (batch.cols[column] == 
null ? "NULL" : batch.cols[column].getClass().getSimpleName()));
     }
   }
 

http://git-wip-us.apache.org/repos/asf/hive/blob/f923db0b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/mapjoin/VectorMapJoinInnerBigOnlyLongOperator.java
----------------------------------------------------------------------
diff --git 
a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/mapjoin/VectorMapJoinInnerBigOnlyLongOperator.java
 
b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/mapjoin/VectorMapJoinInnerBigOnlyLongOperator.java
index 0bba141..43f3951 100644
--- 
a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/mapjoin/VectorMapJoinInnerBigOnlyLongOperator.java
+++ 
b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/mapjoin/VectorMapJoinInnerBigOnlyLongOperator.java
@@ -45,8 +45,17 @@ import 
org.apache.hadoop.hive.ql.exec.vector.LongColumnVector;
 public class VectorMapJoinInnerBigOnlyLongOperator extends 
VectorMapJoinInnerBigOnlyGenerateResultOperator {
 
   private static final long serialVersionUID = 1L;
-  private static final Logger LOG = 
LoggerFactory.getLogger(VectorMapJoinInnerBigOnlyLongOperator.class.getName());
+
+  
//------------------------------------------------------------------------------------------------
+
   private static final String CLASS_NAME = 
VectorMapJoinInnerBigOnlyLongOperator.class.getName();
+  private static final Logger LOG = LoggerFactory.getLogger(CLASS_NAME);
+
+  protected String getLoggingPrefix() {
+    return super.getLoggingPrefix(CLASS_NAME);
+  }
+
+  
//------------------------------------------------------------------------------------------------
 
   // (none)
 

http://git-wip-us.apache.org/repos/asf/hive/blob/f923db0b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/mapjoin/VectorMapJoinInnerBigOnlyMultiKeyOperator.java
----------------------------------------------------------------------
diff --git 
a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/mapjoin/VectorMapJoinInnerBigOnlyMultiKeyOperator.java
 
b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/mapjoin/VectorMapJoinInnerBigOnlyMultiKeyOperator.java
index 621804b..95fb0c2 100644
--- 
a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/mapjoin/VectorMapJoinInnerBigOnlyMultiKeyOperator.java
+++ 
b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/mapjoin/VectorMapJoinInnerBigOnlyMultiKeyOperator.java
@@ -40,6 +40,8 @@ import 
org.apache.hadoop.hive.ql.exec.vector.VectorSerializeRow;
 import org.apache.hadoop.hive.serde2.ByteStream.Output;
 import 
org.apache.hadoop.hive.serde2.binarysortable.fast.BinarySortableSerializeWrite;
 
+import com.google.common.base.Preconditions;
+
 /*
  * Specialized class for doing a vectorized map join that is an inner join on 
Multi-Key
  * and only big table columns appear in the join result so a hash multi-set is 
used.
@@ -48,8 +50,17 @@ import 
org.apache.hadoop.hive.serde2.binarysortable.fast.BinarySortableSerialize
 public class VectorMapJoinInnerBigOnlyMultiKeyOperator extends 
VectorMapJoinInnerBigOnlyGenerateResultOperator {
 
   private static final long serialVersionUID = 1L;
-  private static final Logger LOG = 
LoggerFactory.getLogger(VectorMapJoinInnerBigOnlyMultiKeyOperator.class.getName());
+
+  
//------------------------------------------------------------------------------------------------
+
   private static final String CLASS_NAME = 
VectorMapJoinInnerBigOnlyMultiKeyOperator.class.getName();
+  private static final Logger LOG = LoggerFactory.getLogger(CLASS_NAME);
+
+  protected String getLoggingPrefix() {
+    return super.getLoggingPrefix(CLASS_NAME);
+  }
+
+  
//------------------------------------------------------------------------------------------------
 
   // (none)
 
@@ -114,7 +125,7 @@ public class VectorMapJoinInnerBigOnlyMultiKeyOperator 
extends VectorMapJoinInne
 
         keyVectorSerializeWrite = new VectorSerializeRow(
                                         new 
BinarySortableSerializeWrite(bigTableKeyColumnMap.length));
-        keyVectorSerializeWrite.init(bigTableKeyTypeNames, 
bigTableKeyColumnMap);
+        keyVectorSerializeWrite.init(bigTableKeyTypeInfos, 
bigTableKeyColumnMap);
 
         currentKeyOutput = new Output();
         saveKeyOutput = new Output();

http://git-wip-us.apache.org/repos/asf/hive/blob/f923db0b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/mapjoin/VectorMapJoinInnerBigOnlyStringOperator.java
----------------------------------------------------------------------
diff --git 
a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/mapjoin/VectorMapJoinInnerBigOnlyStringOperator.java
 
b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/mapjoin/VectorMapJoinInnerBigOnlyStringOperator.java
index 10e75ab..044e3e6 100644
--- 
a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/mapjoin/VectorMapJoinInnerBigOnlyStringOperator.java
+++ 
b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/mapjoin/VectorMapJoinInnerBigOnlyStringOperator.java
@@ -46,8 +46,17 @@ import 
org.apache.hadoop.hive.ql.exec.vector.expressions.StringExpr;
 public class VectorMapJoinInnerBigOnlyStringOperator extends 
VectorMapJoinInnerBigOnlyGenerateResultOperator {
 
   private static final long serialVersionUID = 1L;
-  private static final Logger LOG = 
LoggerFactory.getLogger(VectorMapJoinInnerBigOnlyStringOperator.class.getName());
+
+  
//------------------------------------------------------------------------------------------------
+
   private static final String CLASS_NAME = 
VectorMapJoinInnerBigOnlyStringOperator.class.getName();
+  private static final Logger LOG = LoggerFactory.getLogger(CLASS_NAME);
+
+  protected String getLoggingPrefix() {
+    return super.getLoggingPrefix(CLASS_NAME);
+  }
+
+  
//------------------------------------------------------------------------------------------------
 
   // (none)
 

http://git-wip-us.apache.org/repos/asf/hive/blob/f923db0b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/mapjoin/VectorMapJoinInnerLongOperator.java
----------------------------------------------------------------------
diff --git 
a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/mapjoin/VectorMapJoinInnerLongOperator.java
 
b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/mapjoin/VectorMapJoinInnerLongOperator.java
index 804d69c..c85e1d8 100644
--- 
a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/mapjoin/VectorMapJoinInnerLongOperator.java
+++ 
b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/mapjoin/VectorMapJoinInnerLongOperator.java
@@ -44,8 +44,17 @@ import 
org.apache.hadoop.hive.ql.exec.vector.LongColumnVector;
 public class VectorMapJoinInnerLongOperator extends 
VectorMapJoinInnerGenerateResultOperator {
 
   private static final long serialVersionUID = 1L;
-  private static final Logger LOG = 
LoggerFactory.getLogger(VectorMapJoinInnerLongOperator.class.getName());
+
+  
//------------------------------------------------------------------------------------------------
+
   private static final String CLASS_NAME = 
VectorMapJoinInnerLongOperator.class.getName();
+  private static final Logger LOG = LoggerFactory.getLogger(CLASS_NAME);
+
+  protected String getLoggingPrefix() {
+    return super.getLoggingPrefix(CLASS_NAME);
+  }
+
+  
//------------------------------------------------------------------------------------------------
 
   // (none)
 

http://git-wip-us.apache.org/repos/asf/hive/blob/f923db0b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/mapjoin/VectorMapJoinInnerMultiKeyOperator.java
----------------------------------------------------------------------
diff --git 
a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/mapjoin/VectorMapJoinInnerMultiKeyOperator.java
 
b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/mapjoin/VectorMapJoinInnerMultiKeyOperator.java
index fcfa0bd..a108cd0 100644
--- 
a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/mapjoin/VectorMapJoinInnerMultiKeyOperator.java
+++ 
b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/mapjoin/VectorMapJoinInnerMultiKeyOperator.java
@@ -39,6 +39,8 @@ import 
org.apache.hadoop.hive.ql.exec.vector.VectorSerializeRow;
 import org.apache.hadoop.hive.serde2.ByteStream.Output;
 import 
org.apache.hadoop.hive.serde2.binarysortable.fast.BinarySortableSerializeWrite;
 
+import com.google.common.base.Preconditions;
+
 /*
  * Specialized class for doing a vectorized map join that is an inner join on 
a Multi-Key
  * using a hash map.
@@ -46,8 +48,17 @@ import 
org.apache.hadoop.hive.serde2.binarysortable.fast.BinarySortableSerialize
 public class VectorMapJoinInnerMultiKeyOperator extends 
VectorMapJoinInnerGenerateResultOperator {
 
   private static final long serialVersionUID = 1L;
-  private static final Logger LOG = 
LoggerFactory.getLogger(VectorMapJoinInnerMultiKeyOperator.class.getName());
+
+  
//------------------------------------------------------------------------------------------------
+
   private static final String CLASS_NAME = 
VectorMapJoinInnerMultiKeyOperator.class.getName();
+  private static final Logger LOG = 
LoggerFactory.getLogger(VectorMapJoinInnerMultiKeyOperator.class.getName());
+
+  protected String getLoggingPrefix() {
+    return super.getLoggingPrefix(CLASS_NAME);
+  }
+
+  
//------------------------------------------------------------------------------------------------
 
   // (none)
 
@@ -112,7 +123,7 @@ public class VectorMapJoinInnerMultiKeyOperator extends 
VectorMapJoinInnerGenera
 
         keyVectorSerializeWrite = new VectorSerializeRow(
                                         new 
BinarySortableSerializeWrite(bigTableKeyColumnMap.length));
-        keyVectorSerializeWrite.init(bigTableKeyTypeNames, 
bigTableKeyColumnMap);
+        keyVectorSerializeWrite.init(bigTableKeyTypeInfos, 
bigTableKeyColumnMap);
 
         currentKeyOutput = new Output();
         saveKeyOutput = new Output();

http://git-wip-us.apache.org/repos/asf/hive/blob/f923db0b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/mapjoin/VectorMapJoinInnerStringOperator.java
----------------------------------------------------------------------
diff --git 
a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/mapjoin/VectorMapJoinInnerStringOperator.java
 
b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/mapjoin/VectorMapJoinInnerStringOperator.java
index 0f9baae..3211d7d 100644
--- 
a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/mapjoin/VectorMapJoinInnerStringOperator.java
+++ 
b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/mapjoin/VectorMapJoinInnerStringOperator.java
@@ -45,8 +45,17 @@ import 
org.apache.hadoop.hive.ql.exec.vector.expressions.StringExpr;
 public class VectorMapJoinInnerStringOperator extends 
VectorMapJoinInnerGenerateResultOperator {
 
   private static final long serialVersionUID = 1L;
-  private static final Logger LOG = 
LoggerFactory.getLogger(VectorMapJoinInnerStringOperator.class.getName());
+
+  
//------------------------------------------------------------------------------------------------
+
   private static final String CLASS_NAME = 
VectorMapJoinInnerStringOperator.class.getName();
+  private static final Logger LOG = LoggerFactory.getLogger(CLASS_NAME);
+
+  protected String getLoggingPrefix() {
+    return super.getLoggingPrefix(CLASS_NAME);
+  }
+
+  
//------------------------------------------------------------------------------------------------
 
   // (none)
 

http://git-wip-us.apache.org/repos/asf/hive/blob/f923db0b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/mapjoin/VectorMapJoinLeftSemiLongOperator.java
----------------------------------------------------------------------
diff --git 
a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/mapjoin/VectorMapJoinLeftSemiLongOperator.java
 
b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/mapjoin/VectorMapJoinLeftSemiLongOperator.java
index 1149a9d..b02e6fd 100644
--- 
a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/mapjoin/VectorMapJoinLeftSemiLongOperator.java
+++ 
b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/mapjoin/VectorMapJoinLeftSemiLongOperator.java
@@ -45,8 +45,17 @@ import 
org.apache.hadoop.hive.ql.exec.vector.LongColumnVector;
 public class VectorMapJoinLeftSemiLongOperator extends 
VectorMapJoinLeftSemiGenerateResultOperator {
 
   private static final long serialVersionUID = 1L;
-  private static final Logger LOG = 
LoggerFactory.getLogger(VectorMapJoinInnerBigOnlyLongOperator.class.getName());
+
+  
//------------------------------------------------------------------------------------------------
+
   private static final String CLASS_NAME = 
VectorMapJoinLeftSemiLongOperator.class.getName();
+  private static final Logger LOG = LoggerFactory.getLogger(CLASS_NAME);
+
+  protected String getLoggingPrefix() {
+    return super.getLoggingPrefix(CLASS_NAME);
+  }
+
+  
//------------------------------------------------------------------------------------------------
 
   // (none)
 

http://git-wip-us.apache.org/repos/asf/hive/blob/f923db0b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/mapjoin/VectorMapJoinLeftSemiMultiKeyOperator.java
----------------------------------------------------------------------
diff --git 
a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/mapjoin/VectorMapJoinLeftSemiMultiKeyOperator.java
 
b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/mapjoin/VectorMapJoinLeftSemiMultiKeyOperator.java
index e0baebc..36b8f3f 100644
--- 
a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/mapjoin/VectorMapJoinLeftSemiMultiKeyOperator.java
+++ 
b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/mapjoin/VectorMapJoinLeftSemiMultiKeyOperator.java
@@ -40,6 +40,8 @@ import 
org.apache.hadoop.hive.ql.exec.vector.VectorSerializeRow;
 import org.apache.hadoop.hive.serde2.ByteStream.Output;
 import 
org.apache.hadoop.hive.serde2.binarysortable.fast.BinarySortableSerializeWrite;
 
+import com.google.common.base.Preconditions;
+
 /*
  * Specialized class for doing a vectorized map join that is an left semi join 
on Multi-Key
  * using hash set.
@@ -47,8 +49,17 @@ import 
org.apache.hadoop.hive.serde2.binarysortable.fast.BinarySortableSerialize
 public class VectorMapJoinLeftSemiMultiKeyOperator extends 
VectorMapJoinLeftSemiGenerateResultOperator {
 
   private static final long serialVersionUID = 1L;
-  private static final Logger LOG = 
LoggerFactory.getLogger(VectorMapJoinInnerBigOnlyLongOperator.class.getName());
+
+  
//------------------------------------------------------------------------------------------------
+
   private static final String CLASS_NAME = 
VectorMapJoinLeftSemiMultiKeyOperator.class.getName();
+  private static final Logger LOG = LoggerFactory.getLogger(CLASS_NAME);
+
+  protected String getLoggingPrefix() {
+    return super.getLoggingPrefix(CLASS_NAME);
+  }
+
+  
//------------------------------------------------------------------------------------------------
 
   // (none)
 
@@ -113,7 +124,7 @@ public class VectorMapJoinLeftSemiMultiKeyOperator extends 
VectorMapJoinLeftSemi
 
         keyVectorSerializeWrite = new VectorSerializeRow(
                                         new 
BinarySortableSerializeWrite(bigTableKeyColumnMap.length));
-        keyVectorSerializeWrite.init(bigTableKeyTypeNames, 
bigTableKeyColumnMap);
+        keyVectorSerializeWrite.init(bigTableKeyTypeInfos, 
bigTableKeyColumnMap);
 
         currentKeyOutput = new Output();
         saveKeyOutput = new Output();

http://git-wip-us.apache.org/repos/asf/hive/blob/f923db0b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/mapjoin/VectorMapJoinLeftSemiStringOperator.java
----------------------------------------------------------------------
diff --git 
a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/mapjoin/VectorMapJoinLeftSemiStringOperator.java
 
b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/mapjoin/VectorMapJoinLeftSemiStringOperator.java
index 49e1177..0b3de0a 100644
--- 
a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/mapjoin/VectorMapJoinLeftSemiStringOperator.java
+++ 
b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/mapjoin/VectorMapJoinLeftSemiStringOperator.java
@@ -46,8 +46,17 @@ import 
org.apache.hadoop.hive.ql.exec.vector.expressions.StringExpr;
 public class VectorMapJoinLeftSemiStringOperator extends 
VectorMapJoinLeftSemiGenerateResultOperator {
 
   private static final long serialVersionUID = 1L;
-  private static final Logger LOG = 
LoggerFactory.getLogger(VectorMapJoinInnerBigOnlyLongOperator.class.getName());
+
+  
//------------------------------------------------------------------------------------------------
+
   private static final String CLASS_NAME = 
VectorMapJoinLeftSemiStringOperator.class.getName();
+  private static final Logger LOG = LoggerFactory.getLogger(CLASS_NAME);
+
+  protected String getLoggingPrefix() {
+    return super.getLoggingPrefix(CLASS_NAME);
+  }
+
+  
//------------------------------------------------------------------------------------------------
 
   // (none)
 

http://git-wip-us.apache.org/repos/asf/hive/blob/f923db0b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/mapjoin/VectorMapJoinOuterLongOperator.java
----------------------------------------------------------------------
diff --git 
a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/mapjoin/VectorMapJoinOuterLongOperator.java
 
b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/mapjoin/VectorMapJoinOuterLongOperator.java
index 58bd0ab..72309e8 100644
--- 
a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/mapjoin/VectorMapJoinOuterLongOperator.java
+++ 
b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/mapjoin/VectorMapJoinOuterLongOperator.java
@@ -45,8 +45,17 @@ import 
org.apache.hadoop.hive.ql.exec.vector.LongColumnVector;
  */
 public class VectorMapJoinOuterLongOperator extends 
VectorMapJoinOuterGenerateResultOperator {
   private static final long serialVersionUID = 1L;
-  private static final Logger LOG = 
LoggerFactory.getLogger(VectorMapJoinOuterLongOperator.class.getName());
+
+  
//------------------------------------------------------------------------------------------------
+
   private static final String CLASS_NAME = 
VectorMapJoinOuterLongOperator.class.getName();
+  private static final Logger LOG = LoggerFactory.getLogger(CLASS_NAME);
+
+  protected String getLoggingPrefix() {
+    return super.getLoggingPrefix(CLASS_NAME);
+  }
+
+  
//------------------------------------------------------------------------------------------------
 
   // (none)
 

http://git-wip-us.apache.org/repos/asf/hive/blob/f923db0b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/mapjoin/VectorMapJoinOuterMultiKeyOperator.java
----------------------------------------------------------------------
diff --git 
a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/mapjoin/VectorMapJoinOuterMultiKeyOperator.java
 
b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/mapjoin/VectorMapJoinOuterMultiKeyOperator.java
index 7f9afd2..a4fc7d3 100644
--- 
a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/mapjoin/VectorMapJoinOuterMultiKeyOperator.java
+++ 
b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/mapjoin/VectorMapJoinOuterMultiKeyOperator.java
@@ -40,6 +40,8 @@ import 
org.apache.hadoop.hive.ql.exec.vector.VectorSerializeRow;
 import org.apache.hadoop.hive.serde2.ByteStream.Output;
 import 
org.apache.hadoop.hive.serde2.binarysortable.fast.BinarySortableSerializeWrite;
 
+import com.google.common.base.Preconditions;
+
 /*
  * Specialized class for doing a vectorized map join that is an outer join on 
Multi-Key
  * using a hash map.
@@ -47,8 +49,17 @@ import 
org.apache.hadoop.hive.serde2.binarysortable.fast.BinarySortableSerialize
 public class VectorMapJoinOuterMultiKeyOperator extends 
VectorMapJoinOuterGenerateResultOperator {
 
   private static final long serialVersionUID = 1L;
-  private static final Logger LOG = 
LoggerFactory.getLogger(VectorMapJoinOuterMultiKeyOperator.class.getName());
+
+  
//------------------------------------------------------------------------------------------------
+
   private static final String CLASS_NAME = 
VectorMapJoinOuterMultiKeyOperator.class.getName();
+  private static final Logger LOG = LoggerFactory.getLogger(CLASS_NAME);
+
+  protected String getLoggingPrefix() {
+    return super.getLoggingPrefix(CLASS_NAME);
+  }
+
+  
//------------------------------------------------------------------------------------------------
 
   // (none)
 
@@ -112,7 +123,7 @@ public class VectorMapJoinOuterMultiKeyOperator extends 
VectorMapJoinOuterGenera
 
         keyVectorSerializeWrite = new VectorSerializeRow(
                         new 
BinarySortableSerializeWrite(bigTableKeyColumnMap.length));
-        keyVectorSerializeWrite.init(bigTableKeyTypeNames, 
bigTableKeyColumnMap);
+        keyVectorSerializeWrite.init(bigTableKeyTypeInfos, 
bigTableKeyColumnMap);
 
         currentKeyOutput = new Output();
         saveKeyOutput = new Output();

http://git-wip-us.apache.org/repos/asf/hive/blob/f923db0b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/mapjoin/VectorMapJoinOuterStringOperator.java
----------------------------------------------------------------------
diff --git 
a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/mapjoin/VectorMapJoinOuterStringOperator.java
 
b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/mapjoin/VectorMapJoinOuterStringOperator.java
index 8ed1ed4..6e7e5cb 100644
--- 
a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/mapjoin/VectorMapJoinOuterStringOperator.java
+++ 
b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/mapjoin/VectorMapJoinOuterStringOperator.java
@@ -45,8 +45,17 @@ import 
org.apache.hadoop.hive.ql.exec.vector.expressions.StringExpr;
 public class VectorMapJoinOuterStringOperator extends 
VectorMapJoinOuterGenerateResultOperator {
 
   private static final long serialVersionUID = 1L;
-  private static final Logger LOG = 
LoggerFactory.getLogger(VectorMapJoinOuterStringOperator.class.getName());
+
+  
//------------------------------------------------------------------------------------------------
+
   private static final String CLASS_NAME = 
VectorMapJoinOuterStringOperator.class.getName();
+  private static final Logger LOG = LoggerFactory.getLogger(CLASS_NAME);
+
+  protected String getLoggingPrefix() {
+    return super.getLoggingPrefix(CLASS_NAME);
+  }
+
+  
//------------------------------------------------------------------------------------------------
 
   // (none)
 

http://git-wip-us.apache.org/repos/asf/hive/blob/f923db0b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/mapjoin/fast/VectorMapJoinFastTableContainer.java
----------------------------------------------------------------------
diff --git 
a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/mapjoin/fast/VectorMapJoinFastTableContainer.java
 
b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/mapjoin/fast/VectorMapJoinFastTableContainer.java
index 9f3b107..069cc9a 100644
--- 
a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/mapjoin/fast/VectorMapJoinFastTableContainer.java
+++ 
b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/mapjoin/fast/VectorMapJoinFastTableContainer.java
@@ -94,7 +94,7 @@ public class VectorMapJoinFastTableContainer implements 
VectorMapJoinTableContai
   private VectorMapJoinFastHashTable createHashTable(int newThreshold) {
 
     boolean isOuterJoin = !desc.isNoOuterJoin();
-    VectorMapJoinDesc vectorDesc = desc.getVectorDesc();
+    VectorMapJoinDesc vectorDesc = (VectorMapJoinDesc) desc.getVectorDesc();
     HashTableImplementationType hashTableImplementationType = 
vectorDesc.hashTableImplementationType();
     HashTableKind hashTableKind = vectorDesc.hashTableKind();
     HashTableKeyType hashTableKeyType = vectorDesc.hashTableKeyType();

http://git-wip-us.apache.org/repos/asf/hive/blob/f923db0b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/mapjoin/optimized/VectorMapJoinOptimizedCreateHashTable.java
----------------------------------------------------------------------
diff --git 
a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/mapjoin/optimized/VectorMapJoinOptimizedCreateHashTable.java
 
b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/mapjoin/optimized/VectorMapJoinOptimizedCreateHashTable.java
index f34b1cd..111a6d2 100644
--- 
a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/mapjoin/optimized/VectorMapJoinOptimizedCreateHashTable.java
+++ 
b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/mapjoin/optimized/VectorMapJoinOptimizedCreateHashTable.java
@@ -40,7 +40,7 @@ public class VectorMapJoinOptimizedCreateHashTable {
     ReusableGetAdaptor hashMapRowGetter = 
mapJoinTableContainer.createGetter(refKey);
 
     boolean isOuterJoin = !desc.isNoOuterJoin();
-    VectorMapJoinDesc vectorDesc = desc.getVectorDesc();
+    VectorMapJoinDesc vectorDesc = (VectorMapJoinDesc) desc.getVectorDesc();
     HashTableKind hashTableKind = vectorDesc.hashTableKind();
     HashTableKeyType hashTableKeyType = vectorDesc.hashTableKeyType();
     boolean minMaxEnabled = vectorDesc.minMaxEnabled();

http://git-wip-us.apache.org/repos/asf/hive/blob/f923db0b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/reducesink/VectorReduceSinkCommonOperator.java
----------------------------------------------------------------------
diff --git 
a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/reducesink/VectorReduceSinkCommonOperator.java
 
b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/reducesink/VectorReduceSinkCommonOperator.java
index 8133aef..42ca4b7 100644
--- 
a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/reducesink/VectorReduceSinkCommonOperator.java
+++ 
b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/reducesink/VectorReduceSinkCommonOperator.java
@@ -160,7 +160,7 @@ public abstract class VectorReduceSinkCommonOperator 
extends TerminalOperator<Re
 
     ReduceSinkDesc desc = (ReduceSinkDesc) conf;
     this.conf = desc;
-    vectorDesc = desc.getVectorDesc();
+    vectorDesc = (VectorReduceSinkDesc) desc.getVectorDesc();
     vectorReduceSinkInfo = vectorDesc.getVectorReduceSinkInfo();
     this.vContext = vContext;
 

http://git-wip-us.apache.org/repos/asf/hive/blob/f923db0b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/udf/VectorUDFAdaptor.java
----------------------------------------------------------------------
diff --git 
a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/udf/VectorUDFAdaptor.java 
b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/udf/VectorUDFAdaptor.java
index 20cfb89..6806ab4 100644
--- 
a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/udf/VectorUDFAdaptor.java
+++ 
b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/udf/VectorUDFAdaptor.java
@@ -374,6 +374,11 @@ public class VectorUDFAdaptor extends VectorExpression {
   }
 
   @Override
+  public String vectorExpressionParameters() {
+    return expr.getExprString();
+  }
+
+  @Override
   public VectorExpressionDescriptor.Descriptor getDescriptor() {
     return (new VectorExpressionDescriptor.Builder()).build();
   }

Reply via email to