You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hive.apache.org by jd...@apache.org on 2016/05/03 03:38:00 UTC

[43/50] [abbrv] hive git commit: HIVE-12878: Support Vectorization for TEXTFILE and other formats (Matt McCline, reviewed by Sergey Shelukhin)

http://git-wip-us.apache.org/repos/asf/hive/blob/d5285d8e/ql/src/java/org/apache/hadoop/hive/ql/optimizer/physical/Vectorizer.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/physical/Vectorizer.java b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/physical/Vectorizer.java
index 1ddd9be..4a156a2 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/physical/Vectorizer.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/physical/Vectorizer.java
@@ -29,7 +29,6 @@ import java.util.LinkedHashMap;
 import java.util.List;
 import java.util.Map;
 import java.util.Map.Entry;
-import java.util.Properties;
 import java.util.Set;
 import java.util.Stack;
 import java.util.regex.Pattern;
@@ -38,7 +37,6 @@ import org.apache.commons.lang3.tuple.ImmutablePair;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 import org.apache.hadoop.hive.conf.HiveConf;
-import org.apache.hadoop.hive.metastore.api.hive_metastoreConstants;
 import org.apache.hadoop.hive.ql.exec.*;
 import org.apache.hadoop.hive.ql.exec.mr.MapRedTask;
 import org.apache.hadoop.hive.ql.exec.persistence.MapJoinKey;
@@ -98,7 +96,6 @@ import org.apache.hadoop.hive.ql.plan.JoinDesc;
 import org.apache.hadoop.hive.ql.plan.MapJoinDesc;
 import org.apache.hadoop.hive.ql.plan.MapWork;
 import org.apache.hadoop.hive.ql.plan.OperatorDesc;
-import org.apache.hadoop.hive.ql.plan.VectorPartitionConversion;
 import org.apache.hadoop.hive.ql.plan.PartitionDesc;
 import org.apache.hadoop.hive.ql.plan.ReduceSinkDesc;
 import org.apache.hadoop.hive.ql.plan.ReduceWork;
@@ -113,6 +110,7 @@ import org.apache.hadoop.hive.ql.plan.VectorMapJoinDesc;
 import org.apache.hadoop.hive.ql.plan.VectorMapJoinDesc.HashTableImplementationType;
 import org.apache.hadoop.hive.ql.plan.VectorMapJoinDesc.HashTableKeyType;
 import org.apache.hadoop.hive.ql.plan.VectorMapJoinDesc.HashTableKind;
+import org.apache.hadoop.hive.ql.plan.VectorPartitionDesc.VectorDeserializeType;
 import org.apache.hadoop.hive.ql.plan.VectorReduceSinkDesc;
 import org.apache.hadoop.hive.ql.plan.VectorReduceSinkInfo;
 import org.apache.hadoop.hive.ql.plan.VectorPartitionDesc;
@@ -160,8 +158,12 @@ import org.apache.hadoop.hive.ql.udf.UDFYear;
 import org.apache.hadoop.hive.ql.udf.generic.*;
 import org.apache.hadoop.hive.serde.serdeConstants;
 import org.apache.hadoop.hive.serde2.Deserializer;
+import org.apache.hadoop.hive.serde2.NullStructSerDe;
+import org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe;
+import org.apache.hadoop.hive.serde2.lazybinary.LazyBinarySerDe;
 import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspector;
 import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspector.Category;
+import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspectorUtils;
 import org.apache.hadoop.hive.serde2.objectinspector.PrimitiveObjectInspector.PrimitiveCategory;
 import org.apache.hadoop.hive.serde2.objectinspector.StructField;
 import org.apache.hadoop.hive.serde2.objectinspector.StructObjectInspector;
@@ -169,8 +171,10 @@ import org.apache.hadoop.hive.serde2.typeinfo.PrimitiveTypeInfo;
 import org.apache.hadoop.hive.serde2.typeinfo.StructTypeInfo;
 import org.apache.hadoop.hive.serde2.typeinfo.TypeInfo;
 import org.apache.hadoop.hive.serde2.typeinfo.TypeInfoUtils;
+import org.apache.hadoop.mapred.SequenceFileInputFormat;
+import org.apache.hadoop.mapred.TextInputFormat;
 
-import com.google.common.base.Joiner;
+import com.google.common.base.Preconditions;
 
 public class Vectorizer implements PhysicalPlanResolver {
 
@@ -184,8 +188,15 @@ public class Vectorizer implements PhysicalPlanResolver {
   Set<String> supportedAggregationUdfs = new HashSet<String>();
 
   private HiveConf hiveConf;
+
   private boolean isSpark;
 
+  boolean useVectorizedInputFileFormat;
+  boolean useVectorDeserialize;
+  boolean useRowDeserialize;
+
+  boolean isSchemaEvolution;
+
   public Vectorizer() {
 
     StringBuilder patternBuilder = new StringBuilder();
@@ -341,6 +352,7 @@ public class Vectorizer implements PhysicalPlanResolver {
     List<String> columnNames;
     List<TypeInfo> typeInfos;
     int partitionColumnCount;
+    boolean useVectorizedInputFileFormat;
 
     String[] scratchTypeNameArray;
 
@@ -362,7 +374,9 @@ public class Vectorizer implements PhysicalPlanResolver {
     public void setScratchTypeNameArray(String[] scratchTypeNameArray) {
       this.scratchTypeNameArray = scratchTypeNameArray;
     }
-
+    public void setUseVectorizedInputFileFormat(boolean useVectorizedInputFileFormat) {
+      this.useVectorizedInputFileFormat = useVectorizedInputFileFormat;
+    }
     public void setNonVectorizedOps(Set<Operator<? extends OperatorDesc>> nonVectorizedOps) {
       this.nonVectorizedOps = nonVectorizedOps;
     }
@@ -383,6 +397,8 @@ public class Vectorizer implements PhysicalPlanResolver {
             partitionColumnCount,
             scratchTypeNameArray);
       baseWork.setVectorizedRowBatchCtx(vectorizedRowBatchCtx);
+
+      baseWork.setUseVectorizedInputFileFormat(useVectorizedInputFileFormat);
     }
   }
 
@@ -443,6 +459,10 @@ public class Vectorizer implements PhysicalPlanResolver {
           + ReduceSinkOperator.getOperatorName()), np);
     }
 
+    /*
+     * Determine if there is only one TableScanOperator.  Currently in Map vectorization, we do not
+     * try to vectorize multiple input trees.
+     */
     private ImmutablePair<String, TableScanOperator> verifyOnlyOneTableScanOperator(MapWork mapWork) {
 
       // Eliminate MR plans with more than one TableScanOperator.
@@ -476,8 +496,6 @@ public class Vectorizer implements PhysicalPlanResolver {
     private void getTableScanOperatorSchemaInfo(TableScanOperator tableScanOperator,
         List<String> logicalColumnNameList, List<TypeInfo> logicalTypeInfoList) {
 
-      TableScanDesc tableScanDesc = tableScanOperator.getConf();
-
       // Add all non-virtual columns to make a vectorization context for
       // the TableScan operator.
       RowSchema rowSchema = tableScanOperator.getSchema();
@@ -494,35 +512,141 @@ public class Vectorizer implements PhysicalPlanResolver {
       }
     }
 
-     private String getColumns(List<String> columnNames, int start, int length,
-        Character separator) {
-      return Joiner.on(separator).join(columnNames.subList(start, start + length));
-    }
-
-    private String getTypes(List<TypeInfo> typeInfos, int start, int length) {
-      return TypeInfoUtils.getTypesString(typeInfos.subList(start, start + length));
-    }
-
-    private boolean verifyAndSetVectorPartDesc(PartitionDesc pd) {
+    private String getHiveOptionsString() {
+      StringBuilder sb = new StringBuilder();
+      sb.append(HiveConf.ConfVars.HIVE_VECTORIZATION_USE_VECTORIZED_INPUT_FILE_FORMAT.varname);
+      sb.append("=");
+      sb.append(useVectorizedInputFileFormat);
+      sb.append(", ");
+      sb.append(HiveConf.ConfVars.HIVE_VECTORIZATION_USE_VECTOR_DESERIALIZE.varname);
+      sb.append("=");
+      sb.append(useVectorDeserialize);
+      sb.append(", and ");
+      sb.append(HiveConf.ConfVars.HIVE_VECTORIZATION_USE_ROW_DESERIALIZE.varname);
+      sb.append("=");
+      sb.append(useRowDeserialize);
+      return sb.toString();
+    }
+
+    /*
+     * There are 3 modes of reading for vectorization:
+     *
+     *   1) One for the Vectorized Input File Format which returns VectorizedRowBatch as the row.
+     *
+     *   2) One for using VectorDeserializeRow to deserialize each row into the VectorizedRowBatch.
+     *      Currently, these Input File Formats:
+     *        TEXTFILE
+     *        SEQUENCEFILE
+     *
+     *   3) And one using the regular partition deserializer to get the row object and assigning
+     *      the row object into the VectorizedRowBatch with VectorAssignRow.
+     *      This picks up Input File Format not supported by the other two.
+     */
+    private boolean verifyAndSetVectorPartDesc(PartitionDesc pd, boolean isAcidTable) {
+
+      String inputFileFormatClassName = pd.getInputFileFormatClassName();
 
       // Look for Pass-Thru case where InputFileFormat has VectorizedInputFormatInterface
       // and reads VectorizedRowBatch as a "row".
 
-      if (Utilities.isInputFileFormatVectorized(pd)) {
+      if (isAcidTable || useVectorizedInputFileFormat) {
+
+        if (Utilities.isInputFileFormatVectorized(pd)) {
+
+          if (!useVectorizedInputFileFormat) {
+            LOG.info("ACID tables con only be vectorized for the input file format -- " +
+                "i.e. when Hive Configuration option " +
+                HiveConf.ConfVars.HIVE_VECTORIZATION_USE_VECTORIZED_INPUT_FILE_FORMAT.varname +
+                "=true");
+            return false;
+          }
+
+          pd.setVectorPartitionDesc(
+              VectorPartitionDesc.createVectorizedInputFileFormat(
+                  inputFileFormatClassName, Utilities.isInputFileFormatSelfDescribing(pd)));
+
+          return true;
+        }
+
+        // Today, ACID tables are only ORC and that format is vectorizable.  Verify this
+        // assumption.
+        Preconditions.checkState(!isAcidTable);
+      }
+
+      if (!(isSchemaEvolution || isAcidTable) &&
+        (useVectorDeserialize || useRowDeserialize)) {
+        LOG.info("Input format: " + inputFileFormatClassName + " cannot be vectorized" +
+            " when both " + HiveConf.ConfVars.HIVE_SCHEMA_EVOLUTION.varname + "=false and " +
+            " ACID table is " + isAcidTable + " and " +
+            " given the Hive Configuration options " + getHiveOptionsString());
+        return false;
+      }
+
+      String deserializerClassName = pd.getDeserializerClassName();
+
+      // Look for InputFileFormat / Serde combinations we can deserialize more efficiently
+      // using VectorDeserializeRow and a deserialize class with the DeserializeRead interface.
+      //
+      // Do the "vectorized" row-by-row deserialization into a VectorizedRowBatch in the
+      // VectorMapOperator.
+
+      if (useVectorDeserialize) {
+
+        // Currently, we support LazySimple deserialization:
+        //
+        //    org.apache.hadoop.mapred.TextInputFormat
+        //    org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
+        //
+        // AND
+        //
+        //    org.apache.hadoop.mapred.SequenceFileInputFormat
+        //    org.apache.hadoop.hive.serde2.lazybinary.LazyBinarySerDe
+
+        if (inputFileFormatClassName.equals(TextInputFormat.class.getName()) &&
+            deserializerClassName.equals(LazySimpleSerDe.class.getName())) {
+
+          pd.setVectorPartitionDesc(
+              VectorPartitionDesc.createVectorDeserialize(
+                  inputFileFormatClassName, VectorDeserializeType.LAZY_SIMPLE));
+
+          return true;
+        } else if (inputFileFormatClassName.equals(SequenceFileInputFormat.class.getName()) &&
+            deserializerClassName.equals(LazyBinarySerDe.class.getName())) {
 
-        pd.setVectorPartitionDesc(VectorPartitionDesc.createVectorizedInputFileFormat());
+          pd.setVectorPartitionDesc(
+              VectorPartitionDesc.createVectorDeserialize(
+                  inputFileFormatClassName, VectorDeserializeType.LAZY_BINARY));
+
+          return true;
+        }
+      }
+
+      // Otherwise, if enabled, deserialize rows using regular Serde and add the object
+      // inspect-able Object[] row to a VectorizedRowBatch in the VectorMapOperator.
+
+      if (useRowDeserialize) {
+
+        pd.setVectorPartitionDesc(
+            VectorPartitionDesc.createRowDeserialize(
+                inputFileFormatClassName,
+                Utilities.isInputFileFormatSelfDescribing(pd),
+                deserializerClassName));
 
         return true;
+
       }
 
-      LOG.info("Input format: " + pd.getInputFileFormatClassName()
-          + ", doesn't provide vectorized input");
+      LOG.info("Input format: " + inputFileFormatClassName + " cannot be vectorized" +
+          " given the Hive Configuration options " + getHiveOptionsString());
 
       return false;
     }
 
     private boolean validateInputFormatAndSchemaEvolution(MapWork mapWork, String alias,
-        TableScanOperator tableScanOperator, VectorTaskColumnInfo vectorTaskColumnInfo) {
+        TableScanOperator tableScanOperator, VectorTaskColumnInfo vectorTaskColumnInfo)
+            throws SemanticException {
+
+      boolean isAcidTable = tableScanOperator.getConf().isAcidTable();
 
       // These names/types are the data columns plus partition columns.
       final List<String> allColumnNameList = new ArrayList<String>();
@@ -531,23 +655,16 @@ public class Vectorizer implements PhysicalPlanResolver {
       getTableScanOperatorSchemaInfo(tableScanOperator, allColumnNameList, allTypeInfoList);
       final int allColumnCount = allColumnNameList.size();
 
-      // Validate input format and schema evolution capability.
-
-      // For the table, enter a null value in the multi-key map indicating no conversion necessary
-      // if the schema matches the table.
-
-      HashMap<ImmutablePair, boolean[]> conversionMap = new HashMap<ImmutablePair, boolean[]>();
-
+      /*
+       * Validate input formats of all the partitions can be vectorized.
+       */
       boolean isFirst = true;
       int dataColumnCount = 0;
       int partitionColumnCount = 0;
 
-      List<String> dataColumnList = null;
-      String dataColumnsString = "";
-      List<TypeInfo> dataTypeInfoList = null;
+      List<String> tableDataColumnList = null;
+      List<TypeInfo> tableDataTypeInfoList = null;
 
-      // Validate the input format
-      VectorPartitionConversion partitionConversion = new VectorPartitionConversion();
       LinkedHashMap<String, ArrayList<String>> pathToAliases = mapWork.getPathToAliases();
       LinkedHashMap<String, PartitionDesc> pathToPartitionInfo = mapWork.getPathToPartitionInfo();
       for (Entry<String, ArrayList<String>> entry: pathToAliases.entrySet()) {
@@ -563,31 +680,18 @@ public class Vectorizer implements PhysicalPlanResolver {
           // We seen this already.
           continue;
         }
-        if (!verifyAndSetVectorPartDesc(partDesc)) {
+        if (!verifyAndSetVectorPartDesc(partDesc, isAcidTable)) {
           return false;
         }
         VectorPartitionDesc vectorPartDesc = partDesc.getVectorPartitionDesc();
-        LOG.info("Vectorizer path: " + path + ", read type " +
-            vectorPartDesc.getVectorMapOperatorReadType().name() + ", aliases " + aliases);
-
-        Properties partProps = partDesc.getProperties();
-
-        String nextDataColumnsString =
-            partProps.getProperty(hive_metastoreConstants.META_TABLE_COLUMNS);
-        String[] nextDataColumns = nextDataColumnsString.split(",");
-
-        String nextDataTypesString =
-            partProps.getProperty(hive_metastoreConstants.META_TABLE_COLUMN_TYPES);
-
-        // We convert to an array of TypeInfo using a library routine since it parses the information
-        // and can handle use of different separators, etc.  We cannot use the raw type string
-        // for comparison in the map because of the different separators used.
-        List<TypeInfo> nextDataTypeInfoList =
-            TypeInfoUtils.getTypeInfosFromTypeString(nextDataTypesString);
+        if (LOG.isInfoEnabled()) {
+          LOG.info("Vectorizer path: " + path + ", " + vectorPartDesc.toString() +
+              ", aliases " + aliases);
+        }
 
         if (isFirst) {
 
-          // We establish with the first one whether the table is partitioned or not.
+          // Determine the data and partition columns using the first partition descriptor.
 
           LinkedHashMap<String, String> partSpec = partDesc.getPartSpec();
           if (partSpec != null && partSpec.size() > 0) {
@@ -598,85 +702,83 @@ public class Vectorizer implements PhysicalPlanResolver {
             dataColumnCount = allColumnCount;
           }
 
-          dataColumnList = allColumnNameList.subList(0, dataColumnCount);
-          dataColumnsString = getColumns(allColumnNameList, 0, dataColumnCount, ',');
-          dataTypeInfoList = allTypeInfoList.subList(0, dataColumnCount);
-
-          // Add the table (non-partitioned) columns and types into the map as not needing
-          // conversion (i.e. null).
-          conversionMap.put(
-              new ImmutablePair(dataColumnsString, dataTypeInfoList), null);
+          tableDataColumnList = allColumnNameList.subList(0, dataColumnCount);
+          tableDataTypeInfoList = allTypeInfoList.subList(0, dataColumnCount);
 
           isFirst = false;
         }
 
-        ImmutablePair columnNamesAndTypesCombination =
-            new ImmutablePair(nextDataColumnsString, nextDataTypeInfoList);
-
-        boolean[] conversionFlags;
-        if (conversionMap.containsKey(columnNamesAndTypesCombination)) {
-
-          conversionFlags = conversionMap.get(columnNamesAndTypesCombination);
-
-        } else {
-
-          List<String> nextDataColumnList = Arrays.asList(nextDataColumns);
-
-          // Validate the column names that are present are the same.  Missing columns will be
-          // implicitly defaulted to null.
+        // We need to get the partition's column names from the partition serde.
+        // (e.g. Avro provides the table schema and ignores the partition schema..).
+        //
+        Deserializer deserializer;
+        StructObjectInspector partObjectInspector;
+        try {
+          deserializer = partDesc.getDeserializer(hiveConf);
+          partObjectInspector = (StructObjectInspector) deserializer.getObjectInspector();
+        } catch (Exception e) {
+          throw new SemanticException(e);
+        }
+        String nextDataColumnsString = ObjectInspectorUtils.getFieldNames(partObjectInspector);
+        String[] nextDataColumns = nextDataColumnsString.split(",");
+        List<String> nextDataColumnList = Arrays.asList(nextDataColumns);
+
+        /*
+         * Validate the column names that are present are the same.  Missing columns will be
+         * implicitly defaulted to null.
+         */
+        if (nextDataColumnList.size() > tableDataColumnList.size()) {
+          LOG.info(
+              String.format(
+                  "Could not vectorize partition %s " +
+                  "(deserializer " + deserializer.getClass().getName() + ")" +
+                  "The partition column names %d is greater than the number of table columns %d",
+                  path, nextDataColumnList.size(), tableDataColumnList.size()));
+          return false;
+        }
+        if (!(deserializer instanceof NullStructSerDe)) {
 
-          if (nextDataColumnList.size() > dataColumnList.size()) {
-            LOG.info(
-                String.format("Could not vectorize partition %s.  The partition column names %d is greater than the number of table columns %d",
-                    path, nextDataColumnList.size(), dataColumnList.size()));
-            return false;
-          }
+          // (Don't insist NullStructSerDe produce correct column names).
           for (int i = 0; i < nextDataColumnList.size(); i++) {
             String nextColumnName = nextDataColumnList.get(i);
-            String tableColumnName = dataColumnList.get(i);
+            String tableColumnName = tableDataColumnList.get(i);
             if (!nextColumnName.equals(tableColumnName)) {
               LOG.info(
-                  String.format("Could not vectorize partition %s.  The partition column name %s is does not match table column name %s",
+                  String.format(
+                      "Could not vectorize partition %s " +
+                      "(deserializer " + deserializer.getClass().getName() + ")" +
+                      "The partition column name %s is does not match table column name %s",
                       path, nextColumnName, tableColumnName));
               return false;
             }
           }
+        }
 
-          // The table column types might have been changed with ALTER.  There are restrictions
-          // here for vectorization.
-
-          // Some readers / deserializers take responsibility for conversion themselves.
-
-          // If we need to check for conversion, the conversion object may come back null
-          // indicating from a vectorization point of view the conversion is implicit.  That is,
-          // all implicit integer upgrades.
+        List<TypeInfo> nextDataTypeInfoList;
+        if (vectorPartDesc.getIsInputFileFormatSelfDescribing()) {
 
-          if (vectorPartDesc.getNeedsDataTypeConversionCheck() &&
-             !nextDataTypeInfoList.equals(dataTypeInfoList)) {
+          /*
+           * Self-Describing Input Format will convert its data to the table schema.
+           */
+          nextDataTypeInfoList = tableDataTypeInfoList;
 
-             // The results will be in 2 members: validConversion and conversionFlags
-            partitionConversion.validateConversion(nextDataTypeInfoList, dataTypeInfoList);
-             if (!partitionConversion.getValidConversion()) {
-               return false;
-             }
-             conversionFlags = partitionConversion.getResultConversionFlags();
-           } else {
-            conversionFlags = null;
-          }
-
-          // We enter this in our map so we don't have to check again for subsequent partitions.
+        } else {
+          String nextDataTypesString = ObjectInspectorUtils.getFieldTypes(partObjectInspector);
 
-          conversionMap.put(columnNamesAndTypesCombination, conversionFlags);
+          // We convert to an array of TypeInfo using a library routine since it parses the information
+          // and can handle use of different separators, etc.  We cannot use the raw type string
+          // for comparison in the map because of the different separators used.
+          nextDataTypeInfoList =
+              TypeInfoUtils.getTypeInfosFromTypeString(nextDataTypesString);
         }
 
-        vectorPartDesc.setConversionFlags(conversionFlags);
-
-        vectorPartDesc.setTypeInfos(nextDataTypeInfoList);
+        vectorPartDesc.setDataTypeInfos(nextDataTypeInfoList);
       }
 
       vectorTaskColumnInfo.setColumnNames(allColumnNameList);
       vectorTaskColumnInfo.setTypeInfos(allTypeInfoList);
       vectorTaskColumnInfo.setPartitionColumnCount(partitionColumnCount);
+      vectorTaskColumnInfo.setUseVectorizedInputFileFormat(useVectorizedInputFileFormat);
 
       return true;
     }
@@ -1203,7 +1305,23 @@ public class Vectorizer implements PhysicalPlanResolver {
       LOG.info("Vectorization is disabled");
       return physicalContext;
     }
+
     isSpark = (HiveConf.getVar(hiveConf, HiveConf.ConfVars.HIVE_EXECUTION_ENGINE).equals("spark"));
+
+    useVectorizedInputFileFormat =
+        HiveConf.getBoolVar(hiveConf,
+            HiveConf.ConfVars.HIVE_VECTORIZATION_USE_VECTORIZED_INPUT_FILE_FORMAT);
+    useVectorDeserialize =
+        HiveConf.getBoolVar(hiveConf,
+            HiveConf.ConfVars.HIVE_VECTORIZATION_USE_VECTOR_DESERIALIZE);
+    useRowDeserialize =
+        HiveConf.getBoolVar(hiveConf,
+            HiveConf.ConfVars.HIVE_VECTORIZATION_USE_ROW_DESERIALIZE);
+
+    isSchemaEvolution =
+        HiveConf.getBoolVar(hiveConf,
+            HiveConf.ConfVars.HIVE_SCHEMA_EVOLUTION);
+
     // create dispatcher and graph walker
     Dispatcher disp = new VectorizationDispatcher(physicalContext);
     TaskGraphWalker ogw = new TaskGraphWalker(disp);

http://git-wip-us.apache.org/repos/asf/hive/blob/d5285d8e/ql/src/java/org/apache/hadoop/hive/ql/plan/BaseWork.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/plan/BaseWork.java b/ql/src/java/org/apache/hadoop/hive/ql/plan/BaseWork.java
index 429a058..20f787b 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/plan/BaseWork.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/plan/BaseWork.java
@@ -28,6 +28,7 @@ import java.util.Stack;
 
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
+import org.apache.hadoop.hive.conf.HiveConf;
 import org.apache.hadoop.hive.ql.exec.HashTableDummyOperator;
 import org.apache.hadoop.hive.ql.exec.Operator;
 import org.apache.hadoop.hive.ql.exec.vector.VectorizedRowBatchCtx;
@@ -68,6 +69,8 @@ public abstract class BaseWork extends AbstractOperatorDesc {
 
   protected VectorizedRowBatchCtx vectorizedRowBatchCtx;
 
+  protected boolean useVectorizedInputFileFormat;
+
   protected boolean llapMode = false;
   protected boolean uberMode = false;
 
@@ -158,6 +161,9 @@ public abstract class BaseWork extends AbstractOperatorDesc {
 
   // -----------------------------------------------------------------------------------------------
 
+  /*
+   * The vectorization context for creating the VectorizedRowBatch for the node.
+   */
   public VectorizedRowBatchCtx getVectorizedRowBatchCtx() {
     return vectorizedRowBatchCtx;
   }
@@ -166,6 +172,23 @@ public abstract class BaseWork extends AbstractOperatorDesc {
     this.vectorizedRowBatchCtx = vectorizedRowBatchCtx;
   }
 
+  /*
+   * Whether the HiveConf.ConfVars.HIVE_VECTORIZATION_USE_VECTORIZED_INPUT_FILE_FORMAT variable
+   * (hive.vectorized.use.vectorized.input.format) was true when the Vectorizer class evaluated
+   * vectorizing this node.
+   *
+   * When Vectorized Input File Format looks at this flag, it can determine whether it should
+   * operate vectorized or not.  In some modes, the node can be vectorized but use row
+   * serialization.
+   */
+  public void setUseVectorizedInputFileFormat(boolean useVectorizedInputFileFormat) {
+    this.useVectorizedInputFileFormat = useVectorizedInputFileFormat;
+  }
+
+  public boolean getUseVectorizedInputFileFormat() {
+    return useVectorizedInputFileFormat;
+  }
+
   // -----------------------------------------------------------------------------------------------
 
   /**

http://git-wip-us.apache.org/repos/asf/hive/blob/d5285d8e/ql/src/java/org/apache/hadoop/hive/ql/plan/MapWork.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/plan/MapWork.java b/ql/src/java/org/apache/hadoop/hive/ql/plan/MapWork.java
index 0851d9e..f034812 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/plan/MapWork.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/plan/MapWork.java
@@ -38,6 +38,7 @@ import org.apache.hadoop.hive.conf.HiveConf.ConfVars;
 import org.apache.hadoop.hive.ql.exec.FileSinkOperator;
 import org.apache.hadoop.hive.ql.exec.Operator;
 import org.apache.hadoop.hive.ql.exec.OperatorUtils;
+import org.apache.hadoop.hive.ql.exec.vector.VectorizedRowBatch;
 import org.apache.hadoop.hive.ql.io.AcidUtils;
 import org.apache.hadoop.hive.ql.io.HiveInputFormat;
 import org.apache.hadoop.hive.ql.optimizer.physical.BucketingSortingCtx.BucketCol;
@@ -130,6 +131,8 @@ public class MapWork extends BaseWork {
 
   private boolean doSplitsGrouping = true;
 
+  private VectorizedRowBatch vectorizedRowBatch;
+
   // bitsets can't be correctly serialized by Kryo's default serializer
   // BitSet::wordsInUse is transient, so force dumping into a lower form
   private byte[] includedBuckets;
@@ -635,4 +638,12 @@ public class MapWork extends BaseWork {
     // see comment next to the field
     this.includedBuckets = includedBuckets.toByteArray();
   }
+
+  public void setVectorizedRowBatch(VectorizedRowBatch vectorizedRowBatch) {
+    this.vectorizedRowBatch = vectorizedRowBatch;
+  }
+
+  public VectorizedRowBatch getVectorizedRowBatch() {
+    return vectorizedRowBatch;
+  }
 }

http://git-wip-us.apache.org/repos/asf/hive/blob/d5285d8e/ql/src/java/org/apache/hadoop/hive/ql/plan/VectorPartitionConversion.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/plan/VectorPartitionConversion.java b/ql/src/java/org/apache/hadoop/hive/ql/plan/VectorPartitionConversion.java
index 8fe298d..f46581a 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/plan/VectorPartitionConversion.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/plan/VectorPartitionConversion.java
@@ -19,9 +19,6 @@
 package org.apache.hadoop.hive.ql.plan;
 
 import java.util.HashMap;
-import java.util.List;
-
-import org.apache.commons.lang.ArrayUtils;
 import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspector.Category;
 import org.apache.hadoop.hive.serde2.objectinspector.PrimitiveObjectInspector.PrimitiveCategory;
 import org.apache.hadoop.hive.serde2.typeinfo.PrimitiveTypeInfo;
@@ -33,134 +30,75 @@ import org.apache.hadoop.hive.serde2.typeinfo.TypeInfo;
  */
 public class VectorPartitionConversion  {
 
-  private static long serialVersionUID = 1L;
-
-  private boolean validConversion;
-  private boolean[] resultConversionFlags;
-
-  private TypeInfo invalidFromTypeInfo;
-  private TypeInfo invalidToTypeInfo;
-
-  public boolean getValidConversion() {
-    return validConversion;
-  }
-
-  public boolean[] getResultConversionFlags() {
-    return resultConversionFlags;
-  }
-
-  public TypeInfo getInvalidFromTypeInfo() {
-    return invalidFromTypeInfo;
-  }
-
-  public TypeInfo getInvalidToTypeInfo() {
-    return invalidToTypeInfo;
-  }
-
   // Currently, we only support these no-precision-loss or promotion data type conversions:
-  //  //
-  //  Short -> Int                  IMPLICIT WITH VECTORIZATION
-  //  Short -> BigInt               IMPLICIT WITH VECTORIZATION
-  //  Int --> BigInt                IMPLICIT WITH VECTORIZATION
   //
-  // CONSIDER ADDING:
-  //  Float -> Double               IMPLICIT WITH VECTORIZATION
-  //  (Char | VarChar) -> String    IMPLICIT WITH VECTORIZATION
+  //  TinyInt --> SmallInt
+  //  TinyInt --> Int
+  //  TinyInt --> BigInt
+  //
+  //  SmallInt -> Int
+  //  SmallInt -> BigInt
+  //
+  //  Int --> BigInt
   //
-  private static HashMap<PrimitiveCategory, PrimitiveCategory[]> validFromPrimitiveMap =
+  //  Float -> Double
+  //
+  //  Since we stare Char without padding, it can become a String implicitly.
+  //  (Char | VarChar) -> String
+  //
+  private static HashMap<PrimitiveCategory, PrimitiveCategory[]> implicitPrimitiveMap =
       new HashMap<PrimitiveCategory, PrimitiveCategory[]>();
   static {
-    validFromPrimitiveMap.put(
+    implicitPrimitiveMap.put(
+        PrimitiveCategory.BOOLEAN,
+        new PrimitiveCategory[] {
+            PrimitiveCategory.BYTE, PrimitiveCategory.SHORT, PrimitiveCategory.INT, PrimitiveCategory.LONG });
+    implicitPrimitiveMap.put(
+        PrimitiveCategory.BYTE,
+        new PrimitiveCategory[] {
+            PrimitiveCategory.SHORT, PrimitiveCategory.INT, PrimitiveCategory.LONG });
+    implicitPrimitiveMap.put(
         PrimitiveCategory.SHORT,
-        new PrimitiveCategory[] { PrimitiveCategory.INT, PrimitiveCategory.LONG });
-    validFromPrimitiveMap.put(
+        new PrimitiveCategory[] {
+            PrimitiveCategory.INT, PrimitiveCategory.LONG });
+    implicitPrimitiveMap.put(
         PrimitiveCategory.INT,
-        new PrimitiveCategory[] { PrimitiveCategory.LONG });
+        new PrimitiveCategory[] {
+            PrimitiveCategory.LONG });
+    implicitPrimitiveMap.put(
+        PrimitiveCategory.FLOAT,
+        new PrimitiveCategory[] {
+            PrimitiveCategory.DOUBLE });
+    implicitPrimitiveMap.put(
+        PrimitiveCategory.CHAR,
+        new PrimitiveCategory[] {
+            PrimitiveCategory.STRING });
+    implicitPrimitiveMap.put(
+        PrimitiveCategory.VARCHAR,
+        new PrimitiveCategory[] {
+            PrimitiveCategory.STRING });
   }
 
-  private boolean validateOne(TypeInfo fromTypeInfo, TypeInfo toTypeInfo) {
-
-    if (fromTypeInfo.equals(toTypeInfo)) {
-      return false;
-    }
+  public static boolean isImplicitVectorColumnConversion(TypeInfo fromTypeInfo,
+      TypeInfo toTypeInfo) {
 
     if (fromTypeInfo.getCategory() == Category.PRIMITIVE &&
         toTypeInfo.getCategory() == Category.PRIMITIVE) {
 
-      PrimitiveCategory fromPrimitiveCategory = ((PrimitiveTypeInfo) fromTypeInfo).getPrimitiveCategory();
-      PrimitiveCategory toPrimitiveCategory = ((PrimitiveTypeInfo) toTypeInfo).getPrimitiveCategory();
-
-      PrimitiveCategory[] toPrimitiveCategories =
-          validFromPrimitiveMap.get(fromPrimitiveCategory);
-      if (toPrimitiveCategories == null ||
-          !ArrayUtils.contains(toPrimitiveCategories, toPrimitiveCategory)) {
-        invalidFromTypeInfo = fromTypeInfo;
-        invalidToTypeInfo = toTypeInfo;
-
-        // Tell caller a bad one was found.
-        validConversion = false;
-        return false;
-      }
-    } else {
-      // Ignore checking complex types.  Assume they will not be included in the query.
-    }
-
-    return true;
-  }
-
-  public void validateConversion(List<TypeInfo> fromTypeInfoList,
-      List<TypeInfo> toTypeInfoList) {
-
-    final int columnCount = fromTypeInfoList.size();
-    resultConversionFlags = new boolean[columnCount];
-
-    // The method validateOne will turn this off when invalid conversion is found.
-    validConversion = true;
-
-    boolean atLeastOneConversion = false;
-    for (int i = 0; i < columnCount; i++) {
-      TypeInfo fromTypeInfo = fromTypeInfoList.get(i);
-      TypeInfo toTypeInfo = toTypeInfoList.get(i);
-
-      resultConversionFlags[i] = validateOne(fromTypeInfo, toTypeInfo);
-      if (!validConversion) {
-        return;
-      }
-    }
-
-    if (atLeastOneConversion) {
-      // Leave resultConversionFlags set.
-    } else {
-      resultConversionFlags = null;
-    }
-  }
-
-  public void validateConversion(TypeInfo[] fromTypeInfos, TypeInfo[] toTypeInfos) {
-
-    final int columnCount = fromTypeInfos.length;
-    resultConversionFlags = new boolean[columnCount];
-
-    // The method validateOne will turn this off when invalid conversion is found.
-    validConversion = true;
-
-    boolean atLeastOneConversion = false;
-    for (int i = 0; i < columnCount; i++) {
-      TypeInfo fromTypeInfo = fromTypeInfos[i];
-      TypeInfo toTypeInfo = toTypeInfos[i];
-
-      resultConversionFlags[i] = validateOne(fromTypeInfo, toTypeInfo);
-      if (!validConversion) {
-        return;
-      }
-      if (resultConversionFlags[i]) {
-        atLeastOneConversion = true;
+      PrimitiveCategory fromPrimitiveCategory =
+          ((PrimitiveTypeInfo) fromTypeInfo).getPrimitiveCategory();
+      PrimitiveCategory toPrimitiveCategory =
+          ((PrimitiveTypeInfo) toTypeInfo).getPrimitiveCategory();
+      PrimitiveCategory[] toPrimitiveCategories = implicitPrimitiveMap.get(fromPrimitiveCategory);
+      if (toPrimitiveCategories != null) {
+        for (PrimitiveCategory candidatePrimitiveCategory : toPrimitiveCategories) {
+          if (candidatePrimitiveCategory == toPrimitiveCategory) {
+            return true;
+          }
+        }
       }
+      return false;
     }
-
-    if (atLeastOneConversion) {
-      // Leave resultConversionFlags set.
-    } else {
-      resultConversionFlags = null;
-    }
+    return false;
   }
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hive/blob/d5285d8e/ql/src/java/org/apache/hadoop/hive/ql/plan/VectorPartitionDesc.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/plan/VectorPartitionDesc.java b/ql/src/java/org/apache/hadoop/hive/ql/plan/VectorPartitionDesc.java
index 45151f2..2b61ec0 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/plan/VectorPartitionDesc.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/plan/VectorPartitionDesc.java
@@ -39,44 +39,120 @@ public class VectorPartitionDesc  {
   //    No data type conversion check?  Assume ALTER TABLE prevented conversions that
   //    VectorizedInputFileFormat cannot handle...
   //
+  // VECTOR_DESERIALIZE:
+  //    LAZY_SIMPLE:
+  //        Capable of converting on its own.
+  //    LAZY_BINARY
+  //        Partition schema assumed to match file contents.
+  //        Conversion necessary from partition field values to vector columns.
+  // ROW_DESERIALIZE
+  //    Partition schema assumed to match file contents.
+  //    Conversion necessary from partition field values to vector columns.
+  //
 
   public static enum VectorMapOperatorReadType {
     NONE,
-    VECTORIZED_INPUT_FILE_FORMAT
+    VECTORIZED_INPUT_FILE_FORMAT,
+    VECTOR_DESERIALIZE,
+    ROW_DESERIALIZE
   }
 
+  public static enum VectorDeserializeType {
+    NONE,
+    LAZY_SIMPLE,
+    LAZY_BINARY
+  }
 
   private final VectorMapOperatorReadType vectorMapOperatorReadType;
+  private final VectorDeserializeType vectorDeserializeType;
 
-  private final boolean needsDataTypeConversionCheck;
+  private final String rowDeserializerClassName;
+  private final String inputFileFormatClassName;
 
-  private boolean[] conversionFlags;
+  boolean isInputFileFormatSelfDescribing;
 
-  private TypeInfo[] typeInfos;
+  private TypeInfo[] dataTypeInfos;
 
-  private VectorPartitionDesc(VectorMapOperatorReadType vectorMapOperatorReadType,
-      boolean needsDataTypeConversionCheck) {
+  private VectorPartitionDesc(String inputFileFormatClassName,
+      boolean isInputFileFormatSelfDescribing, VectorMapOperatorReadType vectorMapOperatorReadType) {
     this.vectorMapOperatorReadType = vectorMapOperatorReadType;
-    this.needsDataTypeConversionCheck = needsDataTypeConversionCheck;
+    this.vectorDeserializeType = VectorDeserializeType.NONE;
+    this.inputFileFormatClassName = inputFileFormatClassName;
+    rowDeserializerClassName = null;
+    this.isInputFileFormatSelfDescribing = isInputFileFormatSelfDescribing;
+    dataTypeInfos = null;
+  }
 
-    conversionFlags = null;
-    typeInfos = null;
+  /**
+   * Create a VECTOR_DESERIALIZE flavor object.
+   * @param vectorMapOperatorReadType
+   * @param vectorDeserializeType
+   * @param needsDataTypeConversionCheck
+   */
+  private VectorPartitionDesc(String inputFileFormatClassName,
+      VectorDeserializeType vectorDeserializeType) {
+    this.vectorMapOperatorReadType = VectorMapOperatorReadType.VECTOR_DESERIALIZE;
+    this.vectorDeserializeType = vectorDeserializeType;
+    this.inputFileFormatClassName = inputFileFormatClassName;
+    rowDeserializerClassName = null;
+    isInputFileFormatSelfDescribing = false;
+    dataTypeInfos = null;
   }
 
-  public static VectorPartitionDesc createVectorizedInputFileFormat() {
-    return new VectorPartitionDesc(VectorMapOperatorReadType.VECTORIZED_INPUT_FILE_FORMAT, true);
+  /**
+   * Create a ROW_DESERIALIZE flavor object.
+   * @param rowDeserializerClassName
+   * @param inputFileFormatClassName
+   */
+  private VectorPartitionDesc(String inputFileFormatClassName,
+      boolean isInputFileFormatSelfDescribing, String rowDeserializerClassName) {
+    this.vectorMapOperatorReadType = VectorMapOperatorReadType.ROW_DESERIALIZE;
+    this.vectorDeserializeType = VectorDeserializeType.NONE;
+    this.inputFileFormatClassName = inputFileFormatClassName;
+    this.rowDeserializerClassName = rowDeserializerClassName;
+    this.isInputFileFormatSelfDescribing = isInputFileFormatSelfDescribing;
+    dataTypeInfos = null;
   }
 
+  public static VectorPartitionDesc createVectorizedInputFileFormat(String inputFileFormatClassName,
+      boolean isInputFileFormatSelfDescribing) {
+    return new VectorPartitionDesc(
+        inputFileFormatClassName,
+        isInputFileFormatSelfDescribing,
+        VectorMapOperatorReadType.VECTORIZED_INPUT_FILE_FORMAT);
+  }
+
+  public static VectorPartitionDesc createVectorDeserialize(String inputFileFormatClassName,
+      VectorDeserializeType vectorDeserializeType) {
+    return new VectorPartitionDesc(inputFileFormatClassName, vectorDeserializeType);
+  }
+
+  public static VectorPartitionDesc createRowDeserialize(String inputFileFormatClassName,
+      boolean isInputFileFormatSelfDescribing, String rowDeserializerClassName) {
+    return new VectorPartitionDesc(rowDeserializerClassName, isInputFileFormatSelfDescribing,
+        inputFileFormatClassName);
+  }
 
   @Override
   public VectorPartitionDesc clone() {
-    VectorPartitionDesc result =
-        new VectorPartitionDesc(vectorMapOperatorReadType,
-            needsDataTypeConversionCheck);
-    result.conversionFlags =
-        (conversionFlags == null ? null :
-          Arrays.copyOf(conversionFlags, conversionFlags.length));
-    result.typeInfos = Arrays.copyOf(typeInfos, typeInfos.length);
+    VectorPartitionDesc result;
+    switch (vectorMapOperatorReadType) {
+    case VECTORIZED_INPUT_FILE_FORMAT:
+      result = new VectorPartitionDesc(inputFileFormatClassName, isInputFileFormatSelfDescribing,
+          vectorMapOperatorReadType);
+      break;
+    case VECTOR_DESERIALIZE:
+      result = new VectorPartitionDesc(inputFileFormatClassName, vectorDeserializeType);
+      break;
+    case ROW_DESERIALIZE:
+      result = new VectorPartitionDesc(inputFileFormatClassName, isInputFileFormatSelfDescribing,
+          rowDeserializerClassName);
+      break;
+    default:
+      throw new RuntimeException("Unexpected vector map operator read type " + vectorMapOperatorReadType.name());
+    }
+    result.dataTypeInfos = Arrays.copyOf(dataTypeInfos, dataTypeInfos.length);
+
     return result;
   }
 
@@ -84,27 +160,55 @@ public class VectorPartitionDesc  {
     return vectorMapOperatorReadType;
   }
 
-  public boolean getNeedsDataTypeConversionCheck() {
-    return needsDataTypeConversionCheck;
+  public String getInputFileFormatClassName() {
+    return inputFileFormatClassName;
+  }
+
+  public VectorDeserializeType getVectorDeserializeType() {
+    return vectorDeserializeType;
   }
 
-  public void setConversionFlags(boolean[] conversionFlags) {
-    this.conversionFlags = conversionFlags;
+  public String getRowDeserializerClassName() {
+    return rowDeserializerClassName;
   }
 
-  public boolean[] getConversionFlags() {
-    return conversionFlags;
+  public boolean getIsInputFileFormatSelfDescribing() {
+    return isInputFileFormatSelfDescribing;
   }
 
-  public TypeInfo[] getTypeInfos() {
-    return typeInfos;
+  public TypeInfo[] getDataTypeInfos() {
+    return dataTypeInfos;
   }
 
-  public void setTypeInfos(List<TypeInfo> typeInfoList) {
-    typeInfos = typeInfoList.toArray(new TypeInfo[0]);
+  public void setDataTypeInfos(List<TypeInfo> dataTypeInfoList) {
+    dataTypeInfos = dataTypeInfoList.toArray(new TypeInfo[0]);
   }
 
-  public int getNonPartColumnCount() {
-    return typeInfos.length;
+  public int getDataColumnCount() {
+    return dataTypeInfos.length;
+  }
+
+  @Override
+  public String toString() {
+    StringBuilder sb = new StringBuilder();
+    sb.append("vector map operator read type ");
+    sb.append(vectorMapOperatorReadType.name());
+    sb.append(", input file format class name ");
+    sb.append(inputFileFormatClassName);
+    switch (vectorMapOperatorReadType) {
+    case VECTORIZED_INPUT_FILE_FORMAT:
+      break;
+    case VECTOR_DESERIALIZE:
+      sb.append(", deserialize type ");
+      sb.append(vectorDeserializeType.name());
+      break;
+    case ROW_DESERIALIZE:
+      sb.append(", deserializer class name ");
+      sb.append(rowDeserializerClassName);
+      break;
+    default:
+      throw new RuntimeException("Unexpected vector map operator read type " + vectorMapOperatorReadType.name());
+    }
+    return sb.toString();
   }
 }

http://git-wip-us.apache.org/repos/asf/hive/blob/d5285d8e/ql/src/test/org/apache/hadoop/hive/ql/exec/vector/TestVectorRowObject.java
----------------------------------------------------------------------
diff --git a/ql/src/test/org/apache/hadoop/hive/ql/exec/vector/TestVectorRowObject.java b/ql/src/test/org/apache/hadoop/hive/ql/exec/vector/TestVectorRowObject.java
index a5946d1..959a2af 100644
--- a/ql/src/test/org/apache/hadoop/hive/ql/exec/vector/TestVectorRowObject.java
+++ b/ql/src/test/org/apache/hadoop/hive/ql/exec/vector/TestVectorRowObject.java
@@ -34,13 +34,13 @@ import junit.framework.TestCase;
  */
 public class TestVectorRowObject extends TestCase {
 
-  void examineBatch(VectorizedRowBatch batch, VectorExtractRowSameBatch vectorExtractRow,
+  void examineBatch(VectorizedRowBatch batch, VectorExtractRow vectorExtractRow,
               Object[][] randomRows, int firstRandomRowIndex ) {
 
     int rowSize = vectorExtractRow.getCount();
     Object[] row = new Object[rowSize];
     for (int i = 0; i < batch.size; i++) {
-      vectorExtractRow.extractRow(i, row);
+      vectorExtractRow.extractRow(batch, i, row);
       Object[] expectedRow = randomRows[firstRandomRowIndex + i];
       for (int c = 0; c < rowSize; c++) {
         if (!row[c].equals(expectedRow[c])) {
@@ -67,20 +67,18 @@ public class TestVectorRowObject extends TestCase {
       cv.noNulls = false;
     }
 
-    VectorAssignRowSameBatch vectorAssignRow = new VectorAssignRowSameBatch();
+    VectorAssignRow vectorAssignRow = new VectorAssignRow();
     vectorAssignRow.init(source.typeNames());
-    vectorAssignRow.setOneBatch(batch);
-    
-    VectorExtractRowSameBatch vectorExtractRow = new VectorExtractRowSameBatch();
+
+    VectorExtractRow vectorExtractRow = new VectorExtractRow();
     vectorExtractRow.init(source.typeNames());
-    vectorExtractRow.setOneBatch(batch);
 
     Object[][] randomRows = source.randomRows(100000);
     int firstRandomRowIndex = 0;
     for (int i = 0; i < randomRows.length; i++) {
       Object[] row = randomRows[i];
 
-      vectorAssignRow.assignRow(batch.size, row);
+      vectorAssignRow.assignRow(batch, batch.size, row);
       batch.size++;
       if (batch.size == batch.DEFAULT_SIZE) {
         examineBatch(batch, vectorExtractRow, randomRows, firstRandomRowIndex);

http://git-wip-us.apache.org/repos/asf/hive/blob/d5285d8e/ql/src/test/org/apache/hadoop/hive/ql/exec/vector/TestVectorSerDeRow.java
----------------------------------------------------------------------
diff --git a/ql/src/test/org/apache/hadoop/hive/ql/exec/vector/TestVectorSerDeRow.java b/ql/src/test/org/apache/hadoop/hive/ql/exec/vector/TestVectorSerDeRow.java
index 7c0c8d1..e37d2bf 100644
--- a/ql/src/test/org/apache/hadoop/hive/ql/exec/vector/TestVectorSerDeRow.java
+++ b/ql/src/test/org/apache/hadoop/hive/ql/exec/vector/TestVectorSerDeRow.java
@@ -59,6 +59,7 @@ import org.apache.hadoop.hive.serde2.lazybinary.fast.LazyBinarySerializeWrite;
 import org.apache.hadoop.hive.serde2.objectinspector.PrimitiveObjectInspector.PrimitiveCategory;
 import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspectorUtils;
 import org.apache.hadoop.hive.serde2.objectinspector.StructObjectInspector;
+import org.apache.hadoop.hive.serde2.typeinfo.CharTypeInfo;
 import org.apache.hadoop.hive.serde2.typeinfo.DecimalTypeInfo;
 import org.apache.hadoop.hive.serde2.typeinfo.PrimitiveTypeInfo;
 import org.apache.hadoop.hive.serde2.typeinfo.TypeInfo;
@@ -100,7 +101,7 @@ public class TestVectorSerDeRow extends TestCase {
       switch (primitiveCategory) {
       case BOOLEAN:
         {
-          Boolean value = deserializeRead.readBoolean();
+          Boolean value = deserializeRead.currentBoolean;
           BooleanWritable expectedWritable = (BooleanWritable) expected;
           if (!value.equals(expectedWritable.get())) {
             TestCase.fail("Boolean field mismatch (expected " + expected + " found " + value + ")");
@@ -109,7 +110,7 @@ public class TestVectorSerDeRow extends TestCase {
         break;
       case BYTE:
         {
-          Byte value = deserializeRead.readByte();
+          Byte value = deserializeRead.currentByte;
           ByteWritable expectedWritable = (ByteWritable) expected;
           if (!value.equals(expectedWritable.get())) {
             TestCase.fail("Byte field mismatch (expected " + (int) expected + " found " + (int) value + ")");
@@ -118,7 +119,7 @@ public class TestVectorSerDeRow extends TestCase {
         break;
       case SHORT:
         {
-          Short value = deserializeRead.readShort();
+          Short value = deserializeRead.currentShort;
           ShortWritable expectedWritable = (ShortWritable) expected;
           if (!value.equals(expectedWritable.get())) {
             TestCase.fail("Short field mismatch (expected " + expected + " found " + value + ")");
@@ -127,7 +128,7 @@ public class TestVectorSerDeRow extends TestCase {
         break;
       case INT:
         {
-          Integer value = deserializeRead.readInt();
+          Integer value = deserializeRead.currentInt;
           IntWritable expectedWritable = (IntWritable) expected;
           if (!value.equals(expectedWritable.get())) {
             TestCase.fail("Int field mismatch (expected " + expected + " found " + value + ")");
@@ -136,7 +137,7 @@ public class TestVectorSerDeRow extends TestCase {
         break;
       case LONG:
         {
-          Long value = deserializeRead.readLong();
+          Long value = deserializeRead.currentLong;
           LongWritable expectedWritable = (LongWritable) expected;
           if (!value.equals(expectedWritable.get())) {
             TestCase.fail("Long field mismatch (expected " + expected + " found " + value + ")");
@@ -145,18 +146,16 @@ public class TestVectorSerDeRow extends TestCase {
         break;
       case DATE:
         {
-          DeserializeRead.ReadDateResults readDateResults = deserializeRead.createReadDateResults();
-          deserializeRead.readDate(readDateResults);
-          Date value = readDateResults.getDate();
+          DateWritable value = deserializeRead.currentDateWritable;
           DateWritable expectedWritable = (DateWritable) expected;
-          if (!value.equals(expectedWritable.get())) {
+          if (!value.equals(expectedWritable)) {
             TestCase.fail("Date field mismatch (expected " + expected.toString() + " found " + value.toString() + ")");
           }
         }
         break;
       case FLOAT:
         {
-          Float value = deserializeRead.readFloat();
+          Float value = deserializeRead.currentFloat;
           FloatWritable expectedWritable = (FloatWritable) expected;
           if (!value.equals(expectedWritable.get())) {
             TestCase.fail("Float field mismatch (expected " + expected + " found " + value + ")");
@@ -165,7 +164,7 @@ public class TestVectorSerDeRow extends TestCase {
         break;
       case DOUBLE:
         {
-          Double value = deserializeRead.readDouble();
+          Double value = deserializeRead.currentDouble;
           DoubleWritable expectedWritable = (DoubleWritable) expected;
           if (!value.equals(expectedWritable.get())) {
             TestCase.fail("Double field mismatch (expected " + expected + " found " + value + ")");
@@ -173,57 +172,69 @@ public class TestVectorSerDeRow extends TestCase {
         }
         break;
       case STRING:
-        {
-          DeserializeRead.ReadStringResults readStringResults = deserializeRead.createReadStringResults();
-          deserializeRead.readString(readStringResults);
-
-          char[] charsBuffer = new char[readStringResults.bytes.length];
-          for (int c = 0; c < charsBuffer.length; c++) {
-            charsBuffer[c] = (char) (readStringResults.bytes[c] & 0xFF);
-          }
-
-          byte[] stringBytes = Arrays.copyOfRange(readStringResults.bytes, readStringResults.start, readStringResults.start + readStringResults.length);
-
-          char[] charsRange = new char[stringBytes.length];
-          for (int c = 0; c < charsRange.length; c++) {
-            charsRange[c] = (char) (stringBytes[c] & 0xFF);
-          }
-
-          Text text = new Text(stringBytes);
-          String value = text.toString();
-          Text expectedWritable = (Text) expected;
-          if (!value.equals(expectedWritable.toString())) {
-            TestCase.fail("String field mismatch (expected '" + expectedWritable.toString() + "' found '" + value + "')");
-          }
-        }
-        break;
       case CHAR:
-        {
-          DeserializeRead.ReadHiveCharResults readHiveCharResults = deserializeRead.createReadHiveCharResults();
-          deserializeRead.readHiveChar(readHiveCharResults);
-          HiveChar hiveChar = readHiveCharResults.getHiveChar();
-          HiveCharWritable expectedWritable = (HiveCharWritable) expected;
-          if (!hiveChar.equals(expectedWritable.getHiveChar())) {
-            TestCase.fail("Char field mismatch (expected '" + expectedWritable.getHiveChar() + "' found '" + hiveChar + "')");
-          }
-        }
-        break;
       case VARCHAR:
+      case BINARY:
         {
-          DeserializeRead.ReadHiveVarcharResults readHiveVarcharResults = deserializeRead.createReadHiveVarcharResults();
-          deserializeRead.readHiveVarchar(readHiveVarcharResults);
-          HiveVarchar hiveVarchar = readHiveVarcharResults.getHiveVarchar();
-          HiveVarcharWritable expectedWritable = (HiveVarcharWritable) expected;
-          if (!hiveVarchar.equals(expectedWritable.getHiveVarchar())) {
-            TestCase.fail("Varchar field mismatch (expected '" + expectedWritable.getHiveVarchar() + "' found '" + hiveVarchar + "')");
+          byte[] stringBytes =
+              Arrays.copyOfRange(
+                  deserializeRead.currentBytes,
+                  deserializeRead.currentBytesStart,
+                  deserializeRead.currentBytesStart + deserializeRead.currentBytesLength);
+
+          Text text = new Text(stringBytes);
+          String string = text.toString();
+
+          switch (primitiveCategory) {
+          case STRING:
+            {
+              Text expectedWritable = (Text) expected;
+              if (!string.equals(expectedWritable.toString())) {
+                TestCase.fail("String field mismatch (expected '" + expectedWritable.toString() + "' found '" + string + "')");
+              }
+            }
+            break;
+          case CHAR:
+            {
+              HiveChar hiveChar = new HiveChar(string, ((CharTypeInfo) primitiveTypeInfo).getLength());
+
+              HiveCharWritable expectedWritable = (HiveCharWritable) expected;
+              if (!hiveChar.equals(expectedWritable.getHiveChar())) {
+                TestCase.fail("Char field mismatch (expected '" + expectedWritable.getHiveChar() + "' found '" + hiveChar + "')");
+              }
+            }
+            break;
+          case VARCHAR:
+            {
+              HiveVarchar hiveVarchar = new HiveVarchar(string, ((VarcharTypeInfo) primitiveTypeInfo).getLength());
+              HiveVarcharWritable expectedWritable = (HiveVarcharWritable) expected;
+              if (!hiveVarchar.equals(expectedWritable.getHiveVarchar())) {
+                TestCase.fail("Varchar field mismatch (expected '" + expectedWritable.getHiveVarchar() + "' found '" + hiveVarchar + "')");
+              }
+            }
+            break;
+          case BINARY:
+            {
+               BytesWritable expectedWritable = (BytesWritable) expected;
+              if (stringBytes.length != expectedWritable.getLength()){
+                TestCase.fail("Byte Array field mismatch (expected " + expected + " found " + stringBytes + ")");
+              }
+              byte[] expectedBytes = expectedWritable.getBytes();
+              for (int b = 0; b < stringBytes.length; b++) {
+                if (stringBytes[b] != expectedBytes[b]) {
+                  TestCase.fail("Byte Array field mismatch (expected " + expected + " found " + stringBytes + ")");
+                }
+              }
+            }
+            break;
+          default:
+            throw new HiveException("Unexpected primitive category " + primitiveCategory);
           }
         }
         break;
       case DECIMAL:
         {
-          DeserializeRead.ReadDecimalResults readDecimalResults = deserializeRead.createReadDecimalResults();
-          deserializeRead.readHiveDecimal(readDecimalResults);
-          HiveDecimal value = readDecimalResults.getHiveDecimal();
+          HiveDecimal value = deserializeRead.currentHiveDecimalWritable.getHiveDecimal();
           if (value == null) {
             TestCase.fail("Decimal field evaluated to NULL");
           }
@@ -238,9 +249,7 @@ public class TestVectorSerDeRow extends TestCase {
         break;
     case TIMESTAMP:
       {
-        DeserializeRead.ReadTimestampResults readTimestampResults = deserializeRead.createReadTimestampResults();
-        deserializeRead.readTimestamp(readTimestampResults);
-        Timestamp value = readTimestampResults.getTimestamp();
+        Timestamp value = deserializeRead.currentTimestampWritable.getTimestamp();
         TimestampWritable expectedWritable = (TimestampWritable) expected;
         if (!value.equals(expectedWritable.getTimestamp())) {
           TestCase.fail("Timestamp field mismatch (expected " + expectedWritable.getTimestamp() + " found " + value.toString() + ")");
@@ -249,9 +258,7 @@ public class TestVectorSerDeRow extends TestCase {
       break;
     case INTERVAL_YEAR_MONTH:
       {
-        DeserializeRead.ReadIntervalYearMonthResults readIntervalYearMonthResults = deserializeRead.createReadIntervalYearMonthResults();
-        deserializeRead.readIntervalYearMonth(readIntervalYearMonthResults);
-        HiveIntervalYearMonth value = readIntervalYearMonthResults.getHiveIntervalYearMonth();
+        HiveIntervalYearMonth value = deserializeRead.currentHiveIntervalYearMonthWritable.getHiveIntervalYearMonth();
         HiveIntervalYearMonthWritable expectedWritable = (HiveIntervalYearMonthWritable) expected;
         HiveIntervalYearMonth expectedValue = expectedWritable.getHiveIntervalYearMonth();
         if (!value.equals(expectedValue)) {
@@ -261,9 +268,7 @@ public class TestVectorSerDeRow extends TestCase {
       break;
     case INTERVAL_DAY_TIME:
       {
-        DeserializeRead.ReadIntervalDayTimeResults readIntervalDayTimeResults = deserializeRead.createReadIntervalDayTimeResults();
-        deserializeRead.readIntervalDayTime(readIntervalDayTimeResults);
-        HiveIntervalDayTime value = readIntervalDayTimeResults.getHiveIntervalDayTime();
+        HiveIntervalDayTime value = deserializeRead.currentHiveIntervalDayTimeWritable.getHiveIntervalDayTime();
         HiveIntervalDayTimeWritable expectedWritable = (HiveIntervalDayTimeWritable) expected;
         HiveIntervalDayTime expectedValue = expectedWritable.getHiveIntervalDayTime();
         if (!value.equals(expectedValue)) {
@@ -271,26 +276,10 @@ public class TestVectorSerDeRow extends TestCase {
         }
       }
       break;
-    case BINARY:
-      {
-        DeserializeRead.ReadBinaryResults readBinaryResults = deserializeRead.createReadBinaryResults();
-        deserializeRead.readBinary(readBinaryResults);
-        byte[] byteArray = Arrays.copyOfRange(readBinaryResults.bytes, readBinaryResults.start, readBinaryResults.start + readBinaryResults.length);
-        BytesWritable expectedWritable = (BytesWritable) expected;
-        if (byteArray.length != expectedWritable.getLength()){
-          TestCase.fail("Byte Array field mismatch (expected " + expected + " found " + byteArray + ")");
-        }
-        byte[] expectedBytes = expectedWritable.getBytes();
-        for (int b = 0; b < byteArray.length; b++) {
-          if (byteArray[b] != expectedBytes[b]) {
-            TestCase.fail("Byte Array field mismatch (expected " + expected + " found " + byteArray + ")");
-          }
-        }
-      }
-      break;
-      default:
-        throw new HiveException("Unexpected primitive category " + primitiveCategory);
-      }
+
+    default:
+      throw new HiveException("Unexpected primitive category " + primitiveCategory);
+    }
     }
     deserializeRead.extraFieldsCheck();
     TestCase.assertTrue(!deserializeRead.readBeyondConfiguredFieldsWarned());
@@ -331,9 +320,8 @@ public class TestVectorSerDeRow extends TestCase {
     batchContext.init(source.rowStructObjectInspector(), emptyScratchTypeNames);
     VectorizedRowBatch batch = batchContext.createVectorizedRowBatch();
 
-    VectorAssignRowSameBatch vectorAssignRow = new VectorAssignRowSameBatch();
+    VectorAssignRow vectorAssignRow = new VectorAssignRow();
     vectorAssignRow.init(source.typeNames());
-    vectorAssignRow.setOneBatch(batch);
 
     int fieldCount = source.typeNames().size();
     DeserializeRead deserializeRead;
@@ -369,7 +357,7 @@ public class TestVectorSerDeRow extends TestCase {
     for (int i = 0; i < randomRows.length; i++) {
       Object[] row = randomRows[i];
 
-      vectorAssignRow.assignRow(batch.size, row);
+      vectorAssignRow.assignRow(batch, batch.size, row);
       batch.size++;
       if (batch.size == batch.DEFAULT_SIZE) {
         serializeBatch(batch, vectorSerializeRow, deserializeRead, source, randomRows, firstRandomRowIndex);
@@ -382,13 +370,13 @@ public class TestVectorSerDeRow extends TestCase {
     }
   }
 
-  void examineBatch(VectorizedRowBatch batch, VectorExtractRowSameBatch vectorExtractRow,
+  void examineBatch(VectorizedRowBatch batch, VectorExtractRow vectorExtractRow,
       Object[][] randomRows, int firstRandomRowIndex ) {
 
     int rowSize = vectorExtractRow.getCount();
     Object[] row = new Object[rowSize];
     for (int i = 0; i < batch.size; i++) {
-      vectorExtractRow.extractRow(i, row);
+      vectorExtractRow.extractRow(batch, i, row);
 
       Object[] expectedRow = randomRows[firstRandomRowIndex + i];
 
@@ -603,9 +591,8 @@ public class TestVectorSerDeRow extends TestCase {
       cv.noNulls = false;
     }
 
-    VectorExtractRowSameBatch vectorExtractRow = new VectorExtractRowSameBatch();
+    VectorExtractRow vectorExtractRow = new VectorExtractRow();
     vectorExtractRow.init(source.typeNames());
-    vectorExtractRow.setOneBatch(batch);
 
     Object[][] randomRows = source.randomRows(100000);
     int firstRandomRowIndex = 0;
@@ -614,7 +601,7 @@ public class TestVectorSerDeRow extends TestCase {
 
       Output output = serializeRow(row, source, serializeWrite);
       vectorDeserializeRow.setBytes(output.getData(), 0, output.getLength());
-      vectorDeserializeRow.deserializeByValue(batch, batch.size);
+      vectorDeserializeRow.deserialize(batch, batch.size);
       batch.size++;
       if (batch.size == batch.DEFAULT_SIZE) {
         examineBatch(batch, vectorExtractRow, randomRows, firstRandomRowIndex);

http://git-wip-us.apache.org/repos/asf/hive/blob/d5285d8e/ql/src/test/org/apache/hadoop/hive/ql/io/orc/TestInputOutputFormat.java
----------------------------------------------------------------------
diff --git a/ql/src/test/org/apache/hadoop/hive/ql/io/orc/TestInputOutputFormat.java b/ql/src/test/org/apache/hadoop/hive/ql/io/orc/TestInputOutputFormat.java
index 85923a8..4eb0249 100644
--- a/ql/src/test/org/apache/hadoop/hive/ql/io/orc/TestInputOutputFormat.java
+++ b/ql/src/test/org/apache/hadoop/hive/ql/io/orc/TestInputOutputFormat.java
@@ -80,6 +80,7 @@ import org.apache.hadoop.hive.ql.metadata.HiveException;
 import org.apache.hadoop.hive.ql.plan.MapWork;
 import org.apache.hadoop.hive.ql.plan.PartitionDesc;
 import org.apache.hadoop.hive.ql.plan.TableDesc;
+import org.apache.hadoop.hive.ql.plan.VectorPartitionDesc;
 import org.apache.hadoop.hive.serde.serdeConstants;
 import org.apache.hadoop.hive.serde2.ColumnProjectionUtils;
 import org.apache.hadoop.hive.serde2.SerDe;
@@ -1700,7 +1701,10 @@ public class TestInputOutputFormat {
     Utilities.clearWorkMap(conf);
     conf.set("hive.exec.plan", workDir.toString());
     conf.set("mapred.job.tracker", "local");
-    conf.set("hive.vectorized.execution.enabled", Boolean.toString(isVectorized));
+    String isVectorizedString = Boolean.toString(isVectorized);
+    conf.set("hive.vectorized.execution.enabled", isVectorizedString);
+    conf.set(Utilities.VECTOR_MODE, isVectorizedString);
+    conf.set(Utilities.USE_VECTORIZED_INPUT_FILE_FORMAT, isVectorizedString);
     conf.set("fs.mock.impl", MockFileSystem.class.getName());
     conf.set("mapred.mapper.class", ExecMapper.class.getName());
     Path root = new Path(warehouseDir, tableName);
@@ -1767,6 +1771,10 @@ public class TestInputOutputFormat {
       LinkedHashMap<String, String> partSpec =
           new LinkedHashMap<String, String>();
       PartitionDesc part = new PartitionDesc(tbl, partSpec);
+      if (isVectorized) {
+        part.setVectorPartitionDesc(
+            VectorPartitionDesc.createVectorizedInputFileFormat("MockInputFileFormatClassName", false));
+      }
       partMap.put(partPath[p], part);
     }
     mapWork.setPathToAliases(aliasMap);

http://git-wip-us.apache.org/repos/asf/hive/blob/d5285d8e/ql/src/test/queries/clientpositive/avro_schema_evolution_native.q
----------------------------------------------------------------------
diff --git a/ql/src/test/queries/clientpositive/avro_schema_evolution_native.q b/ql/src/test/queries/clientpositive/avro_schema_evolution_native.q
index b32e1ec..efeb167 100644
--- a/ql/src/test/queries/clientpositive/avro_schema_evolution_native.q
+++ b/ql/src/test/queries/clientpositive/avro_schema_evolution_native.q
@@ -1,3 +1,4 @@
+set hive.cli.print.header=true;
 set hive.mapred.mode=nonstrict;
 -- SORT_QUERY_RESULTS
 -- Verify that table scans work with partitioned Avro tables
@@ -19,6 +20,7 @@ STORED AS AVRO;
 SET hive.exec.dynamic.partition.mode=nonstrict;
 INSERT OVERWRITE TABLE episodes_partitioned PARTITION (doctor_pt)
 SELECT title, air_date, doctor, doctor as doctor_pt FROM episodes;
+DESCRIBE FORMATTED episodes_partitioned;
 
 ALTER TABLE episodes_partitioned
 SET SERDE 'org.apache.hadoop.hive.serde2.avro.AvroSerDe'
@@ -51,7 +53,12 @@ SERDEPROPERTIES ('avro.schema.literal'='{
      }
   ]
 }');
+DESCRIBE FORMATTED episodes_partitioned;
 
+set hive.fetch.task.conversion=more;
+
+EXPLAIN
+SELECT * FROM episodes_partitioned WHERE doctor_pt > 6;
 
 SELECT * FROM episodes_partitioned WHERE doctor_pt > 6;
 
@@ -60,4 +67,15 @@ SELECT * FROM episodes_partitioned ORDER BY air_date LIMIT 5;
 -- Fetch w/filter to specific partition
 SELECT * FROM episodes_partitioned WHERE doctor_pt = 6;
 -- Fetch w/non-existent partition
+SELECT * FROM episodes_partitioned WHERE doctor_pt = 7 LIMIT 5;
+
+set hive.fetch.task.conversion=none;
+
+EXPLAIN
+SELECT * FROM episodes_partitioned WHERE doctor_pt > 6;
+
+SELECT * FROM episodes_partitioned WHERE doctor_pt > 6;
+
+SELECT * FROM episodes_partitioned ORDER BY air_date LIMIT 5;
+SELECT * FROM episodes_partitioned WHERE doctor_pt = 6;
 SELECT * FROM episodes_partitioned WHERE doctor_pt = 7 LIMIT 5;
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hive/blob/d5285d8e/ql/src/test/queries/clientpositive/bucket_groupby.q
----------------------------------------------------------------------
diff --git a/ql/src/test/queries/clientpositive/bucket_groupby.q b/ql/src/test/queries/clientpositive/bucket_groupby.q
index ea35bd7..a36c79d 100644
--- a/ql/src/test/queries/clientpositive/bucket_groupby.q
+++ b/ql/src/test/queries/clientpositive/bucket_groupby.q
@@ -1,3 +1,4 @@
+SET hive.vectorized.execution.enabled=false;
 set hive.mapred.mode=nonstrict;
 create table clustergroupby(key string, value string) partitioned by(ds string);
 describe extended clustergroupby;
@@ -6,16 +7,16 @@ alter table clustergroupby clustered by (key) into 1 buckets;
 insert overwrite table clustergroupby partition (ds='100') select key, value from src sort by key;
 
 explain
-select key, count(1) from clustergroupby where ds='100' group by key limit 10;
-select key, count(1) from clustergroupby where ds='100' group by key limit 10;
+select key, count(1) from clustergroupby where ds='100' group by key order by key limit 10;
+select key, count(1) from clustergroupby where ds='100' group by key order by key limit 10;
 
 describe extended clustergroupby;
 insert overwrite table clustergroupby partition (ds='101') select key, value from src distribute by key;
 
 --normal--
 explain
-select key, count(1) from clustergroupby  where ds='101'  group by key limit 10;
-select key, count(1) from clustergroupby  where ds='101' group by key limit 10;
+select key, count(1) from clustergroupby  where ds='101'  group by key order by key limit 10;
+select key, count(1) from clustergroupby  where ds='101' group by key order by key limit 10;
 
 --function--
 explain
@@ -27,13 +28,13 @@ select abs(length(key)), count(1) from clustergroupby  where ds='101' group by a
 
 --constant--
 explain
-select key, count(1) from clustergroupby  where ds='101'  group by key,3 limit 10;
-select key, count(1) from clustergroupby  where ds='101' group by key,3 limit 10;
+select key, count(1) from clustergroupby  where ds='101'  group by key,3 order by key,3 limit 10;
+select key, count(1) from clustergroupby  where ds='101' group by key,3 order by key,3 limit 10;
 
 --subquery--
 explain
-select key, count(1) from (select value as key, key as value from clustergroupby where ds='101')subq group by key limit 10;
-select key, count(1) from (select value as key, key as value from clustergroupby where ds='101')subq group by key limit 10;
+select key, count(1) from (select value as key, key as value from clustergroupby where ds='101')subq group by key order by key limit 10;
+select key, count(1) from (select value as key, key as value from clustergroupby where ds='101')subq group by key order by key limit 10;
 
 explain
 select key, count(1) from clustergroupby  group by key;
@@ -52,11 +53,11 @@ describe extended clustergroupby;
 insert overwrite table clustergroupby partition (ds='102') select key, value from src distribute by value sort by key, value;
 
 explain
-select key, count(1) from clustergroupby  where ds='102'  group by key limit 10;
-select key, count(1) from clustergroupby  where ds='102' group by key limit 10;
+select key, count(1) from clustergroupby  where ds='102'  group by key order by key limit 10;
+select key, count(1) from clustergroupby  where ds='102' group by key order by key limit 10;
 explain
-select value, count(1) from clustergroupby  where ds='102'  group by value limit 10;
-select value, count(1) from clustergroupby  where ds='102'  group by value limit 10;
+select value, count(1) from clustergroupby  where ds='102'  group by value order by value limit 10;
+select value, count(1) from clustergroupby  where ds='102'  group by value order by value limit 10;
 explain
 select key, count(1) from clustergroupby  where ds='102'  group by key, value limit 10;
 select key, count(1) from clustergroupby  where ds='102'  group by key, value limit 10;
@@ -69,8 +70,8 @@ alter table clustergroupby clustered by (value, key) sorted by (key) into 1 buck
 describe extended clustergroupby;
 insert overwrite table clustergroupby partition (ds='103') select key, value from src distribute by value, key sort by key;
 explain
-select key, count(1) from clustergroupby  where ds='103'  group by key limit 10;
-select key, count(1) from clustergroupby  where ds='103' group by key limit 10;
+select key, count(1) from clustergroupby  where ds='103'  group by key order by key limit 10;
+select key, count(1) from clustergroupby  where ds='103' group by key order by key limit 10;
 explain
-select key, count(1) from clustergroupby  where ds='103'  group by value, key limit 10;
-select key, count(1) from clustergroupby  where ds='103' group by  value, key limit 10;
+select key, count(1) from clustergroupby  where ds='103'  group by value, key order by key limit 10;
+select key, count(1) from clustergroupby  where ds='103' group by  value, key order by key limit 10;

http://git-wip-us.apache.org/repos/asf/hive/blob/d5285d8e/ql/src/test/queries/clientpositive/groupby_sort_10.q
----------------------------------------------------------------------
diff --git a/ql/src/test/queries/clientpositive/groupby_sort_10.q b/ql/src/test/queries/clientpositive/groupby_sort_10.q
index 910a272..3517693 100644
--- a/ql/src/test/queries/clientpositive/groupby_sort_10.q
+++ b/ql/src/test/queries/clientpositive/groupby_sort_10.q
@@ -2,6 +2,8 @@ set hive.mapred.mode=nonstrict;
 set hive.exec.reducers.max = 10;
 set hive.map.groupby.sorted=true;
 
+-- SORT_QUERY_RESULTS
+
 CREATE TABLE T1(key STRING, val STRING) PARTITIONED BY (ds string)
 CLUSTERED BY (key) SORTED BY (key) INTO 2 BUCKETS;
 

http://git-wip-us.apache.org/repos/asf/hive/blob/d5285d8e/ql/src/test/queries/clientpositive/schema_evol_orc_acidvec_mapwork_part.q
----------------------------------------------------------------------
diff --git a/ql/src/test/queries/clientpositive/schema_evol_orc_acidvec_mapwork_part.q b/ql/src/test/queries/clientpositive/schema_evol_orc_acidvec_mapwork_part.q
index fc935d5..b0e57fb 100644
--- a/ql/src/test/queries/clientpositive/schema_evol_orc_acidvec_mapwork_part.q
+++ b/ql/src/test/queries/clientpositive/schema_evol_orc_acidvec_mapwork_part.q
@@ -4,6 +4,9 @@ set hive.support.concurrency=true;
 set hive.txn.manager=org.apache.hadoop.hive.ql.lockmgr.DbTxnManager;
 SET hive.exec.schema.evolution=false;
 SET hive.vectorized.execution.enabled=true;
+SET hive.vectorized.use.vectorized.input.format=true;
+SET hive.vectorized.use.vector.serde.deserialize=false;
+SET hive.vectorized.use.row.serde.deserialize=false;
 set hive.fetch.task.conversion=none;
 set hive.exec.dynamic.partition.mode=nonstrict;
 

http://git-wip-us.apache.org/repos/asf/hive/blob/d5285d8e/ql/src/test/queries/clientpositive/schema_evol_orc_acidvec_mapwork_table.q
----------------------------------------------------------------------
diff --git a/ql/src/test/queries/clientpositive/schema_evol_orc_acidvec_mapwork_table.q b/ql/src/test/queries/clientpositive/schema_evol_orc_acidvec_mapwork_table.q
index e49a0f3..ca6822c 100644
--- a/ql/src/test/queries/clientpositive/schema_evol_orc_acidvec_mapwork_table.q
+++ b/ql/src/test/queries/clientpositive/schema_evol_orc_acidvec_mapwork_table.q
@@ -3,6 +3,9 @@ set hive.support.concurrency=true;
 set hive.txn.manager=org.apache.hadoop.hive.ql.lockmgr.DbTxnManager;
 SET hive.exec.schema.evolution=false;
 SET hive.vectorized.execution.enabled=true;
+SET hive.vectorized.use.vectorized.input.format=true;
+SET hive.vectorized.use.vector.serde.deserialize=false;
+SET hive.vectorized.use.row.serde.deserialize=false;
 set hive.fetch.task.conversion=none;
 set hive.exec.dynamic.partition.mode=nonstrict;
 

http://git-wip-us.apache.org/repos/asf/hive/blob/d5285d8e/ql/src/test/queries/clientpositive/schema_evol_orc_nonvec_mapwork_table.q
----------------------------------------------------------------------
diff --git a/ql/src/test/queries/clientpositive/schema_evol_orc_nonvec_mapwork_table.q b/ql/src/test/queries/clientpositive/schema_evol_orc_nonvec_mapwork_table.q
index 6c256ea..f05f02a 100644
--- a/ql/src/test/queries/clientpositive/schema_evol_orc_nonvec_mapwork_table.q
+++ b/ql/src/test/queries/clientpositive/schema_evol_orc_nonvec_mapwork_table.q
@@ -1,7 +1,5 @@
 set hive.cli.print.header=true;
 set hive.support.concurrency=true;
-set hive.txn.manager=org.apache.hadoop.hive.ql.lockmgr.DbTxnManager;
-
 SET hive.vectorized.execution.enabled=false;
 set hive.fetch.task.conversion=none;
 set hive.exec.dynamic.partition.mode=nonstrict;

http://git-wip-us.apache.org/repos/asf/hive/blob/d5285d8e/ql/src/test/queries/clientpositive/schema_evol_orc_vec_mapwork_part.q
----------------------------------------------------------------------
diff --git a/ql/src/test/queries/clientpositive/schema_evol_orc_vec_mapwork_part.q b/ql/src/test/queries/clientpositive/schema_evol_orc_vec_mapwork_part.q
index 30b19bb..da726c5 100644
--- a/ql/src/test/queries/clientpositive/schema_evol_orc_vec_mapwork_part.q
+++ b/ql/src/test/queries/clientpositive/schema_evol_orc_vec_mapwork_part.q
@@ -2,6 +2,9 @@ set hive.mapred.mode=nonstrict;
 set hive.cli.print.header=true;
 SET hive.exec.schema.evolution=true;
 SET hive.vectorized.execution.enabled=true;
+SET hive.vectorized.use.vectorized.input.format=true;
+SET hive.vectorized.use.vector.serde.deserialize=false;
+SET hive.vectorized.use.row.serde.deserialize=false;
 set hive.fetch.task.conversion=more;
 set hive.exec.dynamic.partition.mode=nonstrict;
 

http://git-wip-us.apache.org/repos/asf/hive/blob/d5285d8e/ql/src/test/queries/clientpositive/schema_evol_orc_vec_mapwork_table.q
----------------------------------------------------------------------
diff --git a/ql/src/test/queries/clientpositive/schema_evol_orc_vec_mapwork_table.q b/ql/src/test/queries/clientpositive/schema_evol_orc_vec_mapwork_table.q
index 6df2095..393967f 100644
--- a/ql/src/test/queries/clientpositive/schema_evol_orc_vec_mapwork_table.q
+++ b/ql/src/test/queries/clientpositive/schema_evol_orc_vec_mapwork_table.q
@@ -1,8 +1,9 @@
 set hive.cli.print.header=true;
-set hive.support.concurrency=true;
-set hive.txn.manager=org.apache.hadoop.hive.ql.lockmgr.DbTxnManager;
-
+SET hive.exec.schema.evolution=true;
 SET hive.vectorized.execution.enabled=true;
+SET hive.vectorized.use.vectorized.input.format=true;
+SET hive.vectorized.use.vector.serde.deserialize=false;
+SET hive.vectorized.use.row.serde.deserialize=false;
 set hive.fetch.task.conversion=none;
 set hive.exec.dynamic.partition.mode=nonstrict;
 

http://git-wip-us.apache.org/repos/asf/hive/blob/d5285d8e/ql/src/test/queries/clientpositive/schema_evol_text_fetchwork_table.q
----------------------------------------------------------------------
diff --git a/ql/src/test/queries/clientpositive/schema_evol_text_fetchwork_table.q b/ql/src/test/queries/clientpositive/schema_evol_text_fetchwork_table.q
deleted file mode 100644
index 44f7264..0000000
--- a/ql/src/test/queries/clientpositive/schema_evol_text_fetchwork_table.q
+++ /dev/null
@@ -1,56 +0,0 @@
-set hive.cli.print.header=true;
-set hive.support.concurrency=true;
-set hive.txn.manager=org.apache.hadoop.hive.ql.lockmgr.DbTxnManager;
-SET hive.exec.schema.evolution=true;
-SET hive.vectorized.execution.enabled=false;
-set hive.fetch.task.conversion=none;
-set hive.exec.dynamic.partition.mode=nonstrict;
-
-
--- SORT_QUERY_RESULTS
---
--- FILE VARIATION: TEXT, Non-Vectorized, MapWork, Table
---
---
--- SECTION VARIATION: ALTER TABLE ADD COLUMNS ... STATIC INSERT
----
-CREATE TABLE table1(a INT, b STRING) STORED AS TEXTFILE;
-
-insert into table table1 values(1, 'original'),(2, 'original'), (3, 'original'),(4, 'original');
-
--- Table-Non-Cascade ADD COLUMNS ...
-alter table table1 add columns(c int, d string);
-
-insert into table table1 values(1, 'new', 10, 'ten'),(2, 'new', 20, 'twenty'), (3, 'new', 30, 'thirty'),(4, 'new', 40, 'forty');
-
-insert into table table1 values(5, 'new', 100, 'hundred'),(6, 'new', 200, 'two hundred');
-
--- SELECT permutation columns to make sure NULL defaulting works right
-select a,b from table1;
-select a,b,c from table1;
-select a,b,c,d from table1;
-select a,c,d from table1;
-select a,d from table1;
-select c from table1;
-select d from table1;
-
---
--- SECTION VARIATION: ALTER TABLE CHANGE COLUMN ... STATIC INSERT
--- smallint = (2-byte signed integer, from -32,768 to 32,767)
---
-CREATE TABLE table2(a smallint, b STRING) STORED AS TEXTFILE;
-
-insert into table table2 values(1000, 'original'),(6737, 'original'), ('3', 'original'),('4', 'original');
-
--- Table-Non-Cascade CHANGE COLUMNS ...
-alter table table2 change column a a int;
-
-insert into table table2 values(72909, 'new'),(200, 'new'), (32768, 'new'),(40000, 'new');
-
-insert into table table2 values(5000, 'new'),(90000, 'new');
-
-select a,b from table2;
-
-
-DROP TABLE table1;
-DROP TABLE table2;
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hive/blob/d5285d8e/ql/src/test/queries/clientpositive/schema_evol_text_mapwork_table.q
----------------------------------------------------------------------
diff --git a/ql/src/test/queries/clientpositive/schema_evol_text_mapwork_table.q b/ql/src/test/queries/clientpositive/schema_evol_text_mapwork_table.q
deleted file mode 100644
index 44f7264..0000000
--- a/ql/src/test/queries/clientpositive/schema_evol_text_mapwork_table.q
+++ /dev/null
@@ -1,56 +0,0 @@
-set hive.cli.print.header=true;
-set hive.support.concurrency=true;
-set hive.txn.manager=org.apache.hadoop.hive.ql.lockmgr.DbTxnManager;
-SET hive.exec.schema.evolution=true;
-SET hive.vectorized.execution.enabled=false;
-set hive.fetch.task.conversion=none;
-set hive.exec.dynamic.partition.mode=nonstrict;
-
-
--- SORT_QUERY_RESULTS
---
--- FILE VARIATION: TEXT, Non-Vectorized, MapWork, Table
---
---
--- SECTION VARIATION: ALTER TABLE ADD COLUMNS ... STATIC INSERT
----
-CREATE TABLE table1(a INT, b STRING) STORED AS TEXTFILE;
-
-insert into table table1 values(1, 'original'),(2, 'original'), (3, 'original'),(4, 'original');
-
--- Table-Non-Cascade ADD COLUMNS ...
-alter table table1 add columns(c int, d string);
-
-insert into table table1 values(1, 'new', 10, 'ten'),(2, 'new', 20, 'twenty'), (3, 'new', 30, 'thirty'),(4, 'new', 40, 'forty');
-
-insert into table table1 values(5, 'new', 100, 'hundred'),(6, 'new', 200, 'two hundred');
-
--- SELECT permutation columns to make sure NULL defaulting works right
-select a,b from table1;
-select a,b,c from table1;
-select a,b,c,d from table1;
-select a,c,d from table1;
-select a,d from table1;
-select c from table1;
-select d from table1;
-
---
--- SECTION VARIATION: ALTER TABLE CHANGE COLUMN ... STATIC INSERT
--- smallint = (2-byte signed integer, from -32,768 to 32,767)
---
-CREATE TABLE table2(a smallint, b STRING) STORED AS TEXTFILE;
-
-insert into table table2 values(1000, 'original'),(6737, 'original'), ('3', 'original'),('4', 'original');
-
--- Table-Non-Cascade CHANGE COLUMNS ...
-alter table table2 change column a a int;
-
-insert into table table2 values(72909, 'new'),(200, 'new'), (32768, 'new'),(40000, 'new');
-
-insert into table table2 values(5000, 'new'),(90000, 'new');
-
-select a,b from table2;
-
-
-DROP TABLE table1;
-DROP TABLE table2;
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hive/blob/d5285d8e/ql/src/test/queries/clientpositive/schema_evol_text_nonvec_fetchwork_part.q
----------------------------------------------------------------------
diff --git a/ql/src/test/queries/clientpositive/schema_evol_text_nonvec_fetchwork_part.q b/ql/src/test/queries/clientpositive/schema_evol_text_nonvec_fetchwork_part.q
deleted file mode 100644
index 4d78642..0000000
--- a/ql/src/test/queries/clientpositive/schema_evol_text_nonvec_fetchwork_part.q
+++ /dev/null
@@ -1,98 +0,0 @@
-set hive.mapred.mode=nonstrict;
-set hive.cli.print.header=true;
-SET hive.exec.schema.evolution=true;
-SET hive.vectorized.execution.enabled=false;
-set hive.fetch.task.conversion=more;
-set hive.exec.dynamic.partition.mode=nonstrict;
-
-
--- SORT_QUERY_RESULTS
---
--- FILE VARIATION: TEXT, Non-Vectorized, FetchWork, Partitioned
---
---
--- SECTION VARIATION: ALTER TABLE ADD COLUMNS ... STATIC INSERT
----
-CREATE TABLE partitioned1(a INT, b STRING) PARTITIONED BY(part INT) STORED AS TEXTFILE;
-
-insert into table partitioned1 partition(part=1) values(1, 'original'),(2, 'original'), (3, 'original'),(4, 'original');
-
--- Table-Non-Cascade ADD COLUMNS ...
-alter table partitioned1 add columns(c int, d string);
-
-insert into table partitioned1 partition(part=2) values(1, 'new', 10, 'ten'),(2, 'new', 20, 'twenty'), (3, 'new', 30, 'thirty'),(4, 'new', 40, 'forty');
-
-insert into table partitioned1 partition(part=1) values(5, 'new', 100, 'hundred'),(6, 'new', 200, 'two hundred');
-
--- SELECT permutation columns to make sure NULL defaulting works right
-select part,a,b from partitioned1;
-select part,a,b,c from partitioned1;
-select part,a,b,c,d from partitioned1;
-select part,a,c,d from partitioned1;
-select part,a,d from partitioned1;
-select part,c from partitioned1;
-select part,d from partitioned1;
-
---
--- SECTION VARIATION: ALTER TABLE CHANGE COLUMN ... STATIC INSERT
--- smallint = (2-byte signed integer, from -32,768 to 32,767)
---
-CREATE TABLE partitioned2(a smallint, b STRING) PARTITIONED BY(part INT) STORED AS TEXTFILE;
-
-insert into table partitioned2 partition(part=1) values(1000, 'original'),(6737, 'original'), ('3', 'original'),('4', 'original');
-
--- Table-Non-Cascade CHANGE COLUMNS ...
-alter table partitioned2 change column a a int;
-
-insert into table partitioned2 partition(part=2) values(72909, 'new'),(200, 'new'), (32768, 'new'),(40000, 'new');
-
-insert into table partitioned2 partition(part=1) values(5000, 'new'),(90000, 'new');
-
-select part,a,b from partitioned2;
-
-
---
---
--- SECTION VARIATION: ALTER TABLE ADD COLUMNS ... DYNAMIC INSERT
----
-CREATE TABLE partitioned3(a INT, b STRING) PARTITIONED BY(part INT) STORED AS TEXTFILE;
-
-insert into table partitioned3 partition(part=1) values(1, 'original'),(2, 'original'), (3, 'original'),(4, 'original');
-
--- Table-Non-Cascade ADD COLUMNS ...
-alter table partitioned3 add columns(c int, d string);
-
-insert into table partitioned3 partition(part) values(1, 'new', 10, 'ten', 2),(2, 'new', 20, 'twenty', 2), (3, 'new', 30, 'thirty', 2),(4, 'new', 40, 'forty', 2),
-    (5, 'new', 100, 'hundred', 1),(6, 'new', 200, 'two hundred', 1);
-
--- SELECT permutation columns to make sure NULL defaulting works right
-select part,a,b from partitioned1;
-select part,a,b,c from partitioned1;
-select part,a,b,c,d from partitioned1;
-select part,a,c,d from partitioned1;
-select part,a,d from partitioned1;
-select part,c from partitioned1;
-select part,d from partitioned1;
-
-
---
--- SECTION VARIATION: ALTER TABLE CHANGE COLUMN ... DYNAMIC INSERT
--- smallint = (2-byte signed integer, from -32,768 to 32,767)
---
-CREATE TABLE partitioned4(a smallint, b STRING) PARTITIONED BY(part INT) STORED AS TEXTFILE;
-
-insert into table partitioned4 partition(part=1) values(1000, 'original'),(6737, 'original'), ('3', 'original'),('4', 'original');
-
--- Table-Non-Cascade CHANGE COLUMNS ...
-alter table partitioned4 change column a a int;
-
-insert into table partitioned4 partition(part) values(72909, 'new', 2),(200, 'new', 2), (32768, 'new', 2),(40000, 'new', 2),
-    (5000, 'new', 1),(90000, 'new', 1);
-
-select part,a,b from partitioned4;
-
-
-DROP TABLE partitioned1;
-DROP TABLE partitioned2;
-DROP TABLE partitioned3;
-DROP TABLE partitioned4;
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hive/blob/d5285d8e/ql/src/test/queries/clientpositive/schema_evol_text_nonvec_fetchwork_table.q
----------------------------------------------------------------------
diff --git a/ql/src/test/queries/clientpositive/schema_evol_text_nonvec_fetchwork_table.q b/ql/src/test/queries/clientpositive/schema_evol_text_nonvec_fetchwork_table.q
deleted file mode 100644
index 0834351..0000000
--- a/ql/src/test/queries/clientpositive/schema_evol_text_nonvec_fetchwork_table.q
+++ /dev/null
@@ -1,67 +0,0 @@
-set hive.cli.print.header=true;
-SET hive.exec.schema.evolution=true;
-SET hive.vectorized.execution.enabled=true;
-set hive.fetch.task.conversion=more;
-
--- SORT_QUERY_RESULTS
---
--- FILE VARIATION: ORC, Non-Vectorized, MapWork, Table
---
---
--- SECTION VARIATION: ALTER TABLE ADD COLUMNS
----
-CREATE TABLE table1(a INT, b STRING) STORED AS TEXTFILE;
-
-insert into table table1 values(1, 'original'),(2, 'original'), (3, 'original'),(4, 'original');
-
-select a,b from table1;
-
--- ADD COLUMNS
-alter table table1 add columns(c int, d string);
-
-insert into table table1 values(1, 'new', 10, 'ten'),(2, 'new', 20, 'twenty'), (3, 'new', 30, 'thirty'),(4, 'new', 40, 'forty');
-
-select a,b,c,d from table1;
-
--- ADD COLUMNS
-alter table table1 add columns(e string);
-
-insert into table table1 values(5, 'new', 100, 'hundred', 'another1'),(6, 'new', 200, 'two hundred', 'another2');
-
-select a,b,c,d,e from table1;
-
-
---
--- SECTION VARIATION: ALTER TABLE CHANGE COLUMN
--- smallint = (2-byte signed integer, from -32,768 to 32,767)
---
-CREATE TABLE table3(a smallint, b STRING) STORED AS TEXTFILE;
-
-insert into table table3 values(1000, 'original'),(6737, 'original'), ('3', 'original'),('4', 'original');
-
-select a,b from table3;
-
--- ADD COLUMNS ... RESTRICT
-alter table table3 change column a a int;
-
-insert into table table3 values(72909, 'new'),(200, 'new'), (32768, 'new'),(40000, 'new');
-
-select a,b from table3;
-
--- ADD COLUMNS ... RESTRICT
-alter table table3 add columns(e string);
-
-insert into table table3 values(5000, 'new', 'another5'),(90000, 'new', 'another6');
-
-select a,b from table3;
-
-
--- ADD COLUMNS ... RESTRICT
-alter table table3 change column a a int;
-
-select a,b from table3;
-
-
-DROP TABLE table1;
-DROP TABLE table2;
-DROP TABLE table3;
\ No newline at end of file