http://git-wip-us.apache.org/repos/asf/hive/blob/f923db0b/ql/src/java/org/apache/hadoop/hive/ql/plan/TableScanDesc.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/plan/TableScanDesc.java 
b/ql/src/java/org/apache/hadoop/hive/ql/plan/TableScanDesc.java
index ebe613e..78b2e8b 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/plan/TableScanDesc.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/plan/TableScanDesc.java
@@ -30,6 +30,7 @@ import org.apache.hadoop.hive.ql.metadata.Table;
 import org.apache.hadoop.hive.ql.metadata.VirtualColumn;
 import org.apache.hadoop.hive.ql.parse.TableSample;
 import org.apache.hadoop.hive.ql.plan.Explain.Level;
+import org.apache.hadoop.hive.ql.plan.Explain.Vectorization;
 import org.apache.hadoop.hive.serde.serdeConstants;
 
 
@@ -396,4 +397,29 @@ public class TableScanDesc extends AbstractOperatorDesc {
     return opProps;
   }
 
+  public class TableScanOperatorExplainVectorization extends 
OperatorExplainVectorization {
+
+    private final TableScanDesc tableScanDesc;
+    private final VectorTableScanDesc vectorTableScanDesc;
+
+    public TableScanOperatorExplainVectorization(TableScanDesc tableScanDesc, 
VectorDesc vectorDesc) {
+      // Native vectorization supported.
+      super(vectorDesc, true);
+      this.tableScanDesc = tableScanDesc;
+      vectorTableScanDesc = (VectorTableScanDesc) vectorDesc;
+    }
+
+    @Explain(vectorization = Vectorization.EXPRESSION, displayName = 
"projectedOutputColumns", explainLevels = { Level.DEFAULT, Level.EXTENDED })
+    public String getProjectedOutputColumns() {
+      return Arrays.toString(vectorTableScanDesc.getProjectedOutputColumns());
+    }
+  }
+
+  @Explain(vectorization = Vectorization.OPERATOR, displayName = "TableScan 
Vectorization", explainLevels = { Level.DEFAULT, Level.EXTENDED })
+  public TableScanOperatorExplainVectorization getTableScanVectorization() {
+    if (vectorDesc == null) {
+      return null;
+    }
+    return new TableScanOperatorExplainVectorization(this, vectorDesc);
+  }
 }

http://git-wip-us.apache.org/repos/asf/hive/blob/f923db0b/ql/src/java/org/apache/hadoop/hive/ql/plan/TezWork.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/plan/TezWork.java 
b/ql/src/java/org/apache/hadoop/hive/ql/plan/TezWork.java
index 7a70e6b..a037ea3 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/plan/TezWork.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/plan/TezWork.java
@@ -40,7 +40,7 @@ import org.apache.hadoop.hive.ql.exec.tez.DagUtils;
 import org.apache.hadoop.hive.ql.plan.TezEdgeProperty.EdgeType;
 import org.apache.hadoop.mapred.JobConf;
 import org.apache.hadoop.hive.ql.plan.Explain.Level;
-
+import org.apache.hadoop.hive.ql.plan.Explain.Vectorization;
 
 /**
  * TezWork. This class encapsulates all the work objects that can be executed
@@ -49,7 +49,8 @@ import org.apache.hadoop.hive.ql.plan.Explain.Level;
  *
  */
 @SuppressWarnings("serial")
-@Explain(displayName = "Tez", explainLevels = { Level.USER, Level.DEFAULT, 
Level.EXTENDED })
+@Explain(displayName = "Tez", explainLevels = { Level.USER, Level.DEFAULT, 
Level.EXTENDED },
+    vectorization = Vectorization.SUMMARY_PATH)
 public class TezWork extends AbstractOperatorDesc {
 
   public enum VertexType {
@@ -107,7 +108,8 @@ public class TezWork extends AbstractOperatorDesc {
   /**
    * getWorkMap returns a map of "vertex name" to BaseWork
    */
-  @Explain(displayName = "Vertices", explainLevels = { Level.USER, 
Level.DEFAULT, Level.EXTENDED })
+  @Explain(displayName = "Vertices", explainLevels = { Level.USER, 
Level.DEFAULT, Level.EXTENDED },
+      vectorization = Vectorization.SUMMARY_PATH)
   public Map<String, BaseWork> getWorkMap() {
     Map<String, BaseWork> result = new LinkedHashMap<String, BaseWork>();
     for (BaseWork w: getAllWork()) {
@@ -306,7 +308,8 @@ public class TezWork extends AbstractOperatorDesc {
     }
   }
 
-  @Explain(displayName = "Edges", explainLevels = { Level.USER, Level.DEFAULT, 
Level.EXTENDED })
+  @Explain(displayName = "Edges", explainLevels = { Level.USER, Level.DEFAULT, 
Level.EXTENDED },
+      vectorization = Vectorization.SUMMARY_PATH)
   public Map<String, List<Dependency>> getDependencyMap() {
     Map<String, List<Dependency>> result = new LinkedHashMap<String, 
List<Dependency>>();
     for (Map.Entry<BaseWork, List<BaseWork>> entry: 
invertedWorkGraph.entrySet()) {

http://git-wip-us.apache.org/repos/asf/hive/blob/f923db0b/ql/src/java/org/apache/hadoop/hive/ql/plan/VectorAppMasterEventDesc.java
----------------------------------------------------------------------
diff --git 
a/ql/src/java/org/apache/hadoop/hive/ql/plan/VectorAppMasterEventDesc.java 
b/ql/src/java/org/apache/hadoop/hive/ql/plan/VectorAppMasterEventDesc.java
new file mode 100644
index 0000000..2e11321
--- /dev/null
+++ b/ql/src/java/org/apache/hadoop/hive/ql/plan/VectorAppMasterEventDesc.java
@@ -0,0 +1,35 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.hive.ql.plan;
+
+/**
+ * VectorAppMasterEventDesc.
+ *
+ * Extra parameters beyond AppMasterEventDesc just for the 
VectorAppMasterEventDescOperator.
+ *
+ * We don't extend AppMasterEventDesc because the base OperatorDesc doesn't 
support
+ * clone and adding it is a lot work for little gain.
+ */
+public class VectorAppMasterEventDesc extends AbstractVectorDesc  {
+
+  private static long serialVersionUID = 1L;
+
+  public VectorAppMasterEventDesc() {
+  }
+}

http://git-wip-us.apache.org/repos/asf/hive/blob/f923db0b/ql/src/java/org/apache/hadoop/hive/ql/plan/VectorDesc.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/plan/VectorDesc.java 
b/ql/src/java/org/apache/hadoop/hive/ql/plan/VectorDesc.java
index 3a2efdb..078408c 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/plan/VectorDesc.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/plan/VectorDesc.java
@@ -22,4 +22,9 @@ import java.io.Serializable;
 
 public interface VectorDesc extends Serializable, Cloneable {
   public Object clone() throws CloneNotSupportedException;
+
+  public void setVectorOp(Class<?> vectorOpClass);
+
+  public Class<?> getVectorOpClass();
+
 }

http://git-wip-us.apache.org/repos/asf/hive/blob/f923db0b/ql/src/java/org/apache/hadoop/hive/ql/plan/VectorFileSinkDesc.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/plan/VectorFileSinkDesc.java 
b/ql/src/java/org/apache/hadoop/hive/ql/plan/VectorFileSinkDesc.java
new file mode 100644
index 0000000..325ac91
--- /dev/null
+++ b/ql/src/java/org/apache/hadoop/hive/ql/plan/VectorFileSinkDesc.java
@@ -0,0 +1,35 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.hive.ql.plan;
+
+/**
+ * VectorFileSinkDesc.
+ *
+ * Extra parameters beyond FileSinkDesc just for the VectorFileSinkOperator.
+ *
+ * We don't extend FileSinkDesc because the base OperatorDesc doesn't support
+ * clone and adding it is a lot work for little gain.
+ */
+public class VectorFileSinkDesc extends AbstractVectorDesc  {
+
+  private static long serialVersionUID = 1L;
+
+  public VectorFileSinkDesc() {
+  }
+}

http://git-wip-us.apache.org/repos/asf/hive/blob/f923db0b/ql/src/java/org/apache/hadoop/hive/ql/plan/VectorFilterDesc.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/plan/VectorFilterDesc.java 
b/ql/src/java/org/apache/hadoop/hive/ql/plan/VectorFilterDesc.java
new file mode 100644
index 0000000..6feed84
--- /dev/null
+++ b/ql/src/java/org/apache/hadoop/hive/ql/plan/VectorFilterDesc.java
@@ -0,0 +1,48 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.hive.ql.plan;
+
+import org.apache.hadoop.hive.ql.exec.vector.expressions.VectorExpression;
+
+/**
+ * VectorFilterDesc.
+ *
+ * Extra parameters beyond FilterDesc just for the VectorFilterOperator.
+ *
+ * We don't extend FilterDesc because the base OperatorDesc doesn't support
+ * clone and adding it is a lot work for little gain.
+ */
+public class VectorFilterDesc extends AbstractVectorDesc  {
+
+  private static long serialVersionUID = 1L;
+
+  private VectorExpression predicateExpression;
+
+  public VectorFilterDesc() {
+  }
+
+  public void setPredicateExpression(VectorExpression predicateExpression) {
+    this.predicateExpression = predicateExpression;
+  }
+
+  public VectorExpression getPredicateExpression() {
+    return predicateExpression;
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hive/blob/f923db0b/ql/src/java/org/apache/hadoop/hive/ql/plan/VectorGroupByDesc.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/plan/VectorGroupByDesc.java 
b/ql/src/java/org/apache/hadoop/hive/ql/plan/VectorGroupByDesc.java
index 08f8ebf..f8554e2 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/plan/VectorGroupByDesc.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/plan/VectorGroupByDesc.java
@@ -18,6 +18,9 @@
 
 package org.apache.hadoop.hive.ql.plan;
 
+import org.apache.hadoop.hive.ql.exec.vector.expressions.VectorExpression;
+import 
org.apache.hadoop.hive.ql.exec.vector.expressions.aggregates.VectorAggregateExpression;
+
 /**
  * VectorGroupByDesc.
  *
@@ -59,6 +62,10 @@ public class VectorGroupByDesc extends AbstractVectorDesc  {
 
   private boolean isVectorOutput;
 
+  private VectorExpression[] keyExpressions;
+  private VectorAggregateExpression[] aggregators;
+  private int[] projectedOutputColumns;
+
   public VectorGroupByDesc() {
     this.processingMode = ProcessingMode.NONE;
     this.isVectorOutput = false;
@@ -79,6 +86,30 @@ public class VectorGroupByDesc extends AbstractVectorDesc  {
     this.isVectorOutput = isVectorOutput;
   }
 
+  public void setKeyExpressions(VectorExpression[] keyExpressions) {
+    this.keyExpressions = keyExpressions;
+  }
+
+  public VectorExpression[] getKeyExpressions() {
+    return keyExpressions;
+  }
+
+  public void setAggregators(VectorAggregateExpression[] aggregators) {
+    this.aggregators = aggregators;
+  }
+
+  public VectorAggregateExpression[] getAggregators() {
+    return aggregators;
+  }
+
+  public void setProjectedOutputColumns(int[] projectedOutputColumns) {
+    this.projectedOutputColumns = projectedOutputColumns;
+  }
+
+  public int[] getProjectedOutputColumns() {
+    return projectedOutputColumns;
+  }
+
   /**
    * Which ProcessingMode for VectorGroupByOperator?
    *

http://git-wip-us.apache.org/repos/asf/hive/blob/f923db0b/ql/src/java/org/apache/hadoop/hive/ql/plan/VectorLimitDesc.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/plan/VectorLimitDesc.java 
b/ql/src/java/org/apache/hadoop/hive/ql/plan/VectorLimitDesc.java
new file mode 100644
index 0000000..c9bc45a
--- /dev/null
+++ b/ql/src/java/org/apache/hadoop/hive/ql/plan/VectorLimitDesc.java
@@ -0,0 +1,35 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.hive.ql.plan;
+
+/**
+ * VectorLimitDesc.
+ *
+ * Extra parameters beyond LimitDesc just for the VectorLimitOperator.
+ *
+ * We don't extend LimitDesc because the base OperatorDesc doesn't support
+ * clone and adding it is a lot work for little gain.
+ */
+public class VectorLimitDesc extends AbstractVectorDesc  {
+
+  private static long serialVersionUID = 1L;
+
+  public VectorLimitDesc() {
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hive/blob/f923db0b/ql/src/java/org/apache/hadoop/hive/ql/plan/VectorMapJoinDesc.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/plan/VectorMapJoinDesc.java 
b/ql/src/java/org/apache/hadoop/hive/ql/plan/VectorMapJoinDesc.java
index 8ea230f..1252140 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/plan/VectorMapJoinDesc.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/plan/VectorMapJoinDesc.java
@@ -18,9 +18,13 @@
 
 package org.apache.hadoop.hive.ql.plan;
 
+import java.util.List;
+
 import org.apache.hadoop.hive.serde2.typeinfo.PrimitiveTypeInfo;
 import org.apache.hadoop.hive.serde2.typeinfo.TypeInfoFactory;
 
+import com.google.common.base.Preconditions;
+
 /**
  * VectorGroupByDesc.
  *
@@ -79,23 +83,38 @@ public class VectorMapJoinDesc extends AbstractVectorDesc  {
     }
   }
 
+  public static enum OperatorVariation {
+    NONE,
+    INNER_BIG_ONLY,
+    INNER,
+    LEFT_SEMI,
+    OUTER
+  }
+
   private HashTableImplementationType hashTableImplementationType;
   private HashTableKind hashTableKind;
   private HashTableKeyType hashTableKeyType;
+  private OperatorVariation operatorVariation;
   private boolean minMaxEnabled;
 
+  private VectorMapJoinInfo vectorMapJoinInfo;
+
   public VectorMapJoinDesc() {
     hashTableImplementationType = HashTableImplementationType.NONE;
     hashTableKind = HashTableKind.NONE;
     hashTableKeyType = HashTableKeyType.NONE;
+    operatorVariation = OperatorVariation.NONE;
     minMaxEnabled = false;
+    vectorMapJoinInfo = null;
   }
 
   public VectorMapJoinDesc(VectorMapJoinDesc clone) {
     this.hashTableImplementationType = clone.hashTableImplementationType;
     this.hashTableKind = clone.hashTableKind;
     this.hashTableKeyType = clone.hashTableKeyType;
+    this.operatorVariation = clone.operatorVariation;
     this.minMaxEnabled = clone.minMaxEnabled;
+    this.vectorMapJoinInfo = clone.vectorMapJoinInfo;
   }
 
   public HashTableImplementationType hashTableImplementationType() {
@@ -122,6 +141,14 @@ public class VectorMapJoinDesc extends AbstractVectorDesc  
{
     this.hashTableKeyType = hashTableKeyType;
   }
 
+  public OperatorVariation operatorVariation() {
+    return operatorVariation;
+  }
+
+  public void setOperatorVariation(OperatorVariation operatorVariation) {
+    this.operatorVariation = operatorVariation;
+  }
+
   public boolean minMaxEnabled() {
     return minMaxEnabled;
   }
@@ -129,4 +156,87 @@ public class VectorMapJoinDesc extends AbstractVectorDesc  
{
   public void setMinMaxEnabled(boolean minMaxEnabled) {
     this.minMaxEnabled = minMaxEnabled;
   }
+
+  public void setVectorMapJoinInfo(VectorMapJoinInfo vectorMapJoinInfo) {
+    Preconditions.checkState(vectorMapJoinInfo != null);
+    this.vectorMapJoinInfo = vectorMapJoinInfo;
+  }
+
+  public VectorMapJoinInfo getVectorMapJoinInfo() {
+    return vectorMapJoinInfo;
+  }
+
+  private boolean isVectorizationMapJoinNativeEnabled;
+  private String engine;
+  private boolean oneMapJoinCondition;
+  private boolean hasNullSafes;
+  private boolean isFastHashTableEnabled;
+  private boolean isHybridHashJoin;
+  private boolean supportsKeyTypes;
+  private List<String> notSupportedKeyTypes;
+  private boolean isEmptyKey;
+  private boolean smallTableExprVectorizes;
+
+  public void setIsVectorizationMapJoinNativeEnabled(boolean 
isVectorizationMapJoinNativeEnabled) {
+    this.isVectorizationMapJoinNativeEnabled = 
isVectorizationMapJoinNativeEnabled;
+  }
+  public boolean getIsVectorizationMapJoinNativeEnabled() {
+    return isVectorizationMapJoinNativeEnabled;
+  }
+  public void setEngine(String engine) {
+    this.engine = engine;
+  }
+  public String getEngine() {
+    return engine;
+  }
+  public void setOneMapJoinCondition(boolean oneMapJoinCondition) {
+    this.oneMapJoinCondition = oneMapJoinCondition;
+  }
+  public boolean getOneMapJoinCondition() {
+    return oneMapJoinCondition;
+  }
+  public void setHasNullSafes(boolean hasNullSafes) {
+    this.hasNullSafes = hasNullSafes;
+  }
+  public boolean getHasNullSafes() {
+    return hasNullSafes;
+  }
+  public void setSupportsKeyTypes(boolean supportsKeyTypes) {
+    this.supportsKeyTypes = supportsKeyTypes;
+  }
+  public boolean getSupportsKeyTypes() {
+    return supportsKeyTypes;
+  }
+  public void setNotSupportedKeyTypes(List<String> notSupportedKeyTypes) {
+    this.notSupportedKeyTypes = notSupportedKeyTypes;
+  }
+  public List<String> getNotSupportedKeyTypes() {
+    return notSupportedKeyTypes;
+  }
+  public void setIsEmptyKey(boolean isEmptyKey) {
+    this.isEmptyKey = isEmptyKey;
+  }
+  public boolean getIsEmptyKey() {
+    return isEmptyKey;
+  }
+  public void setSmallTableExprVectorizes(boolean smallTableExprVectorizes) {
+    this.smallTableExprVectorizes = smallTableExprVectorizes;
+  }
+  public boolean getSmallTableExprVectorizes() {
+    return smallTableExprVectorizes;
+  }
+
+  public void setIsFastHashTableEnabled(boolean isFastHashTableEnabled) {
+    this.isFastHashTableEnabled = isFastHashTableEnabled;
+  }
+  public boolean getIsFastHashTableEnabled() {
+    return isFastHashTableEnabled;
+  }
+  public void setIsHybridHashJoin(boolean isHybridHashJoin) {
+    this.isHybridHashJoin = isHybridHashJoin;
+  }
+  public boolean getIsHybridHashJoin() {
+    return isHybridHashJoin;
+  }
+
 }

http://git-wip-us.apache.org/repos/asf/hive/blob/f923db0b/ql/src/java/org/apache/hadoop/hive/ql/plan/VectorMapJoinInfo.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/plan/VectorMapJoinInfo.java 
b/ql/src/java/org/apache/hadoop/hive/ql/plan/VectorMapJoinInfo.java
new file mode 100644
index 0000000..2cf2e72
--- /dev/null
+++ b/ql/src/java/org/apache/hadoop/hive/ql/plan/VectorMapJoinInfo.java
@@ -0,0 +1,169 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.hive.ql.plan;
+
+import org.apache.hadoop.hive.ql.exec.vector.VectorColumnOutputMapping;
+import org.apache.hadoop.hive.ql.exec.vector.VectorColumnSourceMapping;
+import org.apache.hadoop.hive.ql.exec.vector.expressions.VectorExpression;
+import org.apache.hadoop.hive.serde2.typeinfo.TypeInfo;
+
+/**
+ * VectorMapJoinInfo.
+ *
+ * A convenience data structure that has information needed to vectorize map 
join.
+ *
+ * It is created by the Vectorizer when it is determining whether it can 
specialize so the
+ * information doesn't have to be recreated again and again by the 
VectorMapJoinOperator's
+ * constructors and later during execution.
+ */
+public class VectorMapJoinInfo {
+
+  private static long serialVersionUID = 1L;
+
+  private int[] bigTableKeyColumnMap;
+  private String[] bigTableKeyColumnNames;
+  private TypeInfo[] bigTableKeyTypeInfos;
+  private VectorExpression[] bigTableKeyExpressions;
+
+  private int[] bigTableValueColumnMap;
+  private String[] bigTableValueColumnNames;
+  private TypeInfo[] bigTableValueTypeInfos;
+  private VectorExpression[] bigTableValueExpressions;
+
+  private VectorColumnOutputMapping bigTableRetainedMapping;
+  private VectorColumnOutputMapping bigTableOuterKeyMapping;
+  private VectorColumnSourceMapping smallTableMapping;
+
+  private VectorColumnSourceMapping projectionMapping;
+
+  public VectorMapJoinInfo() {
+    bigTableKeyColumnMap = null;
+    bigTableKeyColumnNames = null;
+    bigTableKeyTypeInfos = null;
+    bigTableKeyExpressions = null;
+
+    bigTableValueColumnMap = null;
+    bigTableValueColumnNames = null;
+    bigTableValueTypeInfos = null;
+    bigTableValueExpressions = null;
+
+    bigTableRetainedMapping = null;
+    bigTableOuterKeyMapping = null;
+    smallTableMapping = null;
+
+    projectionMapping = null;
+  }
+
+  public int[] getBigTableKeyColumnMap() {
+    return bigTableKeyColumnMap;
+  }
+
+  public void setBigTableKeyColumnMap(int[] bigTableKeyColumnMap) {
+    this.bigTableKeyColumnMap = bigTableKeyColumnMap;
+  }
+
+  public String[] getBigTableKeyColumnNames() {
+    return bigTableKeyColumnNames;
+  }
+
+  public void setBigTableKeyColumnNames(String[] bigTableKeyColumnNames) {
+    this.bigTableKeyColumnNames = bigTableKeyColumnNames;
+  }
+
+  public TypeInfo[] getBigTableKeyTypeInfos() {
+    return bigTableKeyTypeInfos;
+  }
+
+  public void setBigTableKeyTypeInfos(TypeInfo[] bigTableKeyTypeInfos) {
+    this.bigTableKeyTypeInfos = bigTableKeyTypeInfos;
+  }
+
+  public VectorExpression[] getBigTableKeyExpressions() {
+    return bigTableKeyExpressions;
+  }
+
+  public void setBigTableKeyExpressions(VectorExpression[] 
bigTableKeyExpressions) {
+    this.bigTableKeyExpressions = bigTableKeyExpressions;
+  }
+
+
+  public int[] getBigTableValueColumnMap() {
+    return bigTableValueColumnMap;
+  }
+
+  public void setBigTableValueColumnMap(int[] bigTableValueColumnMap) {
+    this.bigTableValueColumnMap = bigTableValueColumnMap;
+  }
+
+  public String[] getBigTableValueColumnNames() {
+    return bigTableValueColumnNames;
+  }
+
+  public void setBigTableValueColumnNames(String[] bigTableValueColumnNames) {
+    this.bigTableValueColumnNames = bigTableValueColumnNames;
+  }
+
+  public TypeInfo[] getBigTableValueTypeInfos() {
+    return bigTableValueTypeInfos;
+  }
+
+  public void setBigTableValueTypeInfos(TypeInfo[] bigTableValueTypeInfos) {
+    this.bigTableValueTypeInfos = bigTableValueTypeInfos;
+  }
+
+  public VectorExpression[] getBigTableValueExpressions() {
+    return bigTableValueExpressions;
+  }
+
+  public void setBigTableValueExpressions(VectorExpression[] 
bigTableValueExpressions) {
+    this.bigTableValueExpressions = bigTableValueExpressions;
+  }
+
+  public void setBigTableRetainedMapping(VectorColumnOutputMapping 
bigTableRetainedMapping) {
+    this.bigTableRetainedMapping = bigTableRetainedMapping;
+  }
+
+  public VectorColumnOutputMapping getBigTableRetainedMapping() {
+    return bigTableRetainedMapping;
+  }
+
+  public void setBigTableOuterKeyMapping(VectorColumnOutputMapping 
bigTableOuterKeyMapping) {
+    this.bigTableOuterKeyMapping = bigTableOuterKeyMapping;
+  }
+
+  public VectorColumnOutputMapping getBigTableOuterKeyMapping() {
+    return bigTableOuterKeyMapping;
+  }
+
+  public void setSmallTableMapping(VectorColumnSourceMapping 
smallTableMapping) {
+    this.smallTableMapping = smallTableMapping;
+  }
+
+  public VectorColumnSourceMapping getSmallTableMapping() {
+    return smallTableMapping;
+  }
+
+  public void setProjectionMapping(VectorColumnSourceMapping 
projectionMapping) {
+    this.projectionMapping = projectionMapping;
+  }
+
+  public VectorColumnSourceMapping getProjectionMapping() {
+    return projectionMapping;
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hive/blob/f923db0b/ql/src/java/org/apache/hadoop/hive/ql/plan/VectorReduceSinkDesc.java
----------------------------------------------------------------------
diff --git 
a/ql/src/java/org/apache/hadoop/hive/ql/plan/VectorReduceSinkDesc.java 
b/ql/src/java/org/apache/hadoop/hive/ql/plan/VectorReduceSinkDesc.java
index c56bff6..288a440 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/plan/VectorReduceSinkDesc.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/plan/VectorReduceSinkDesc.java
@@ -61,4 +61,72 @@ public class VectorReduceSinkDesc extends AbstractVectorDesc 
 {
   public VectorReduceSinkInfo getVectorReduceSinkInfo() {
     return vectorReduceSinkInfo;
   }
+
+  private boolean isVectorizationReduceSinkNativeEnabled;
+  private String engine;
+  private boolean acidChange;
+  private boolean hasBuckets;
+  private boolean hasTopN;
+  private boolean useUniformHash;
+  private boolean hasDistinctColumns;
+  private boolean isKeyBinarySortable;
+  private boolean isValueLazyBinary;
+
+  /*
+   * The following conditions are for native Vector ReduceSink.
+   */
+  public void setIsVectorizationReduceSinkNativeEnabled(boolean 
isVectorizationReduceSinkNativeEnabled) {
+    this.isVectorizationReduceSinkNativeEnabled = 
isVectorizationReduceSinkNativeEnabled;
+  }
+  public boolean getIsVectorizationReduceSinkNativeEnabled() {
+    return isVectorizationReduceSinkNativeEnabled;
+  }
+  public void setEngine(String engine) {
+    this.engine = engine;
+  }
+  public String getEngine() {
+    return engine;
+  }
+  public void setAcidChange(boolean acidChange) {
+    this.acidChange = acidChange;
+  }
+  public boolean getAcidChange() {
+    return acidChange;
+  }
+  public void setHasBuckets(boolean hasBuckets) {
+    this.hasBuckets = hasBuckets;
+  }  
+  public boolean getHasBuckets() {
+    return hasBuckets;
+  }
+  public void setHasTopN(boolean hasTopN) {
+    this.hasTopN = hasTopN;
+  }
+  public boolean getHasTopN() {
+    return hasTopN;
+  }
+  public void setUseUniformHash(boolean useUniformHash) {
+    this.useUniformHash = useUniformHash;
+  }
+  public boolean getUseUniformHash() {
+    return useUniformHash;
+  }
+  public void setHasDistinctColumns(boolean hasDistinctColumns) {
+    this.hasDistinctColumns = hasDistinctColumns;
+  }
+  public boolean getHasDistinctColumns() {
+    return hasDistinctColumns;
+  }
+  public void setIsKeyBinarySortable(boolean isKeyBinarySortable) {
+    this.isKeyBinarySortable = isKeyBinarySortable;
+  }
+  public boolean getIsKeyBinarySortable() {
+    return isKeyBinarySortable;
+  }
+  public void setIsValueLazyBinary(boolean isValueLazyBinary) {
+    this.isValueLazyBinary = isValueLazyBinary;
+  }
+  public boolean getIsValueLazyBinary() {
+    return isValueLazyBinary;
+  }
 }

http://git-wip-us.apache.org/repos/asf/hive/blob/f923db0b/ql/src/java/org/apache/hadoop/hive/ql/plan/VectorSMBJoinDesc.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/plan/VectorSMBJoinDesc.java 
b/ql/src/java/org/apache/hadoop/hive/ql/plan/VectorSMBJoinDesc.java
new file mode 100644
index 0000000..ab578cd
--- /dev/null
+++ b/ql/src/java/org/apache/hadoop/hive/ql/plan/VectorSMBJoinDesc.java
@@ -0,0 +1,35 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.hive.ql.plan;
+
+/**
+ * VectorSMBMapJoinDesc.
+ *
+ * Extra parameters beyond SMBMapJoinDesc just for the 
VectorSMBMapJoinOperator.
+ *
+ * We don't extend SMBMapJoinDesc because the base OperatorDesc doesn't support
+ * clone and adding it is a lot work for little gain.
+ */
+public class VectorSMBJoinDesc extends AbstractVectorDesc  {
+
+  private static long serialVersionUID = 1L;
+
+  public VectorSMBJoinDesc() {
+  }
+}

http://git-wip-us.apache.org/repos/asf/hive/blob/f923db0b/ql/src/java/org/apache/hadoop/hive/ql/plan/VectorSelectDesc.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/plan/VectorSelectDesc.java 
b/ql/src/java/org/apache/hadoop/hive/ql/plan/VectorSelectDesc.java
new file mode 100644
index 0000000..c2c9450
--- /dev/null
+++ b/ql/src/java/org/apache/hadoop/hive/ql/plan/VectorSelectDesc.java
@@ -0,0 +1,56 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.hive.ql.plan;
+
+import org.apache.hadoop.hive.ql.exec.vector.expressions.VectorExpression;
+
+/**
+ * VectorSelectDesc.
+ *
+ * Extra parameters beyond SelectDesc just for the VectorSelectOperator.
+ *
+ * We don't extend SelectDesc because the base OperatorDesc doesn't support
+ * clone and adding it is a lot work for little gain.
+ */
+public class VectorSelectDesc extends AbstractVectorDesc  {
+
+  private static long serialVersionUID = 1L;
+
+  private VectorExpression[] selectExpressions;
+  private int[] projectedOutputColumns;
+
+  public VectorSelectDesc() {
+  }
+
+  public void setSelectExpressions(VectorExpression[] selectExpressions) {
+    this.selectExpressions = selectExpressions;
+  }
+
+  public VectorExpression[] getSelectExpressions() {
+    return selectExpressions;
+  }
+
+  public void setProjectedOutputColumns(int[] projectedOutputColumns) {
+    this.projectedOutputColumns = projectedOutputColumns;
+  }
+
+  public int[] getProjectedOutputColumns() {
+    return projectedOutputColumns;
+  }
+}

http://git-wip-us.apache.org/repos/asf/hive/blob/f923db0b/ql/src/java/org/apache/hadoop/hive/ql/plan/VectorSparkHashTableSinkDesc.java
----------------------------------------------------------------------
diff --git 
a/ql/src/java/org/apache/hadoop/hive/ql/plan/VectorSparkHashTableSinkDesc.java 
b/ql/src/java/org/apache/hadoop/hive/ql/plan/VectorSparkHashTableSinkDesc.java
new file mode 100644
index 0000000..7fb59db
--- /dev/null
+++ 
b/ql/src/java/org/apache/hadoop/hive/ql/plan/VectorSparkHashTableSinkDesc.java
@@ -0,0 +1,35 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.hive.ql.plan;
+
+/**
+ * VectorHashTableSinkDesc.
+ *
+ * Extra parameters beyond HashTableSinkDesc just for the 
VectorHashTableSinkOperator.
+ *
+ * We don't extend HashTableSinkDesc because the base OperatorDesc doesn't 
support
+ * clone and adding it is a lot work for little gain.
+ */
+public class VectorSparkHashTableSinkDesc extends AbstractVectorDesc  {
+
+  private static long serialVersionUID = 1L;
+
+  public VectorSparkHashTableSinkDesc() {
+  }
+}

http://git-wip-us.apache.org/repos/asf/hive/blob/f923db0b/ql/src/java/org/apache/hadoop/hive/ql/plan/VectorTableScanDesc.java
----------------------------------------------------------------------
diff --git 
a/ql/src/java/org/apache/hadoop/hive/ql/plan/VectorTableScanDesc.java 
b/ql/src/java/org/apache/hadoop/hive/ql/plan/VectorTableScanDesc.java
new file mode 100644
index 0000000..6e5ebe4
--- /dev/null
+++ b/ql/src/java/org/apache/hadoop/hive/ql/plan/VectorTableScanDesc.java
@@ -0,0 +1,45 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.hive.ql.plan;
+
+/**
+ * VectorTableScanDesc.
+ *
+ * Extra parameters beyond TableScanDesc just for the VectorTableScanOperator.
+ *
+ * We don't extend TableScanDesc because the base OperatorDesc doesn't support
+ * clone and adding it is a lot work for little gain.
+ */
+public class VectorTableScanDesc extends AbstractVectorDesc  {
+
+  private static long serialVersionUID = 1L;
+
+  private int[] projectedOutputColumns;
+
+  public VectorTableScanDesc() {
+  }
+
+  public void setProjectedOutputColumns(int[] projectedOutputColumns) {
+    this.projectedOutputColumns = projectedOutputColumns;
+  }
+
+  public int[] getProjectedOutputColumns() {
+    return projectedOutputColumns;
+  }
+}

http://git-wip-us.apache.org/repos/asf/hive/blob/f923db0b/ql/src/java/org/apache/hadoop/hive/ql/plan/VectorizationCondition.java
----------------------------------------------------------------------
diff --git 
a/ql/src/java/org/apache/hadoop/hive/ql/plan/VectorizationCondition.java 
b/ql/src/java/org/apache/hadoop/hive/ql/plan/VectorizationCondition.java
new file mode 100644
index 0000000..32b62e8
--- /dev/null
+++ b/ql/src/java/org/apache/hadoop/hive/ql/plan/VectorizationCondition.java
@@ -0,0 +1,76 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.hive.ql.plan;
+
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.List;
+
+public class VectorizationCondition {
+
+  private final boolean flag;
+  private final String conditionName;
+
+  public VectorizationCondition(boolean flag, String conditionName) {
+    this.flag = flag;
+    this.conditionName = conditionName;
+  }
+
+  public boolean getFlag() {
+    return flag;
+  }
+
+  public String getConditionName() {
+    return conditionName;
+  }
+
+  public static List<String> getConditionsMet(VectorizationCondition[] 
conditions) {
+    List<String> metList = new ArrayList<String>();
+    for (VectorizationCondition condition : conditions) {
+      if (condition.getFlag()) {
+        metList.add(condition.getConditionName() + " IS true");
+      }
+    }
+    return metList;
+  }
+
+  public static List<String> getConditionsNotMet(VectorizationCondition[] 
conditions) {
+    List<String> notMetList = new ArrayList<String>();
+    for (VectorizationCondition condition : conditions) {
+      if (!condition.getFlag()) {
+        notMetList.add(condition.getConditionName() + " IS false");
+      }
+    }
+    return notMetList;
+  }
+
+  public static List<String> addBooleans(List<String> conditions, boolean 
flag) {
+    ArrayList<String> result = new ArrayList<String>(conditions.size());
+    for (String condition : conditions) {
+      result.add(condition + " IS " + flag);
+    }
+    return result;
+  }
+
+  // Helper method.
+  public static List<String> getConditionsSupported(boolean isSupported) {
+    return Arrays.asList("Supported IS " + isSupported);
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hive/blob/f923db0b/ql/src/test/org/apache/hadoop/hive/ql/exec/vector/TestVectorFilterOperator.java
----------------------------------------------------------------------
diff --git 
a/ql/src/test/org/apache/hadoop/hive/ql/exec/vector/TestVectorFilterOperator.java
 
b/ql/src/test/org/apache/hadoop/hive/ql/exec/vector/TestVectorFilterOperator.java
index d3bb84d..22b845d 100644
--- 
a/ql/src/test/org/apache/hadoop/hive/ql/exec/vector/TestVectorFilterOperator.java
+++ 
b/ql/src/test/org/apache/hadoop/hive/ql/exec/vector/TestVectorFilterOperator.java
@@ -25,13 +25,19 @@ import junit.framework.Assert;
 
 import org.apache.hadoop.hive.conf.HiveConf;
 import org.apache.hadoop.hive.ql.CompilationOpContext;
+import org.apache.hadoop.hive.ql.exec.FilterOperator;
+import org.apache.hadoop.hive.ql.exec.Operator;
+import org.apache.hadoop.hive.ql.exec.OperatorFactory;
 import org.apache.hadoop.hive.ql.exec.vector.expressions.FilterExprAndExpr;
 import org.apache.hadoop.hive.ql.exec.vector.expressions.VectorExpression;
 import 
org.apache.hadoop.hive.ql.exec.vector.expressions.gen.FilterLongColEqualDoubleScalar;
 import 
org.apache.hadoop.hive.ql.exec.vector.expressions.gen.FilterLongColGreaterLongColumn;
 import org.apache.hadoop.hive.ql.metadata.HiveException;
+import org.apache.hadoop.hive.ql.optimizer.physical.Vectorizer;
 import org.apache.hadoop.hive.ql.plan.ExprNodeColumnDesc;
 import org.apache.hadoop.hive.ql.plan.FilterDesc;
+import org.apache.hadoop.hive.ql.plan.OperatorDesc;
+import org.apache.hadoop.hive.ql.plan.VectorFilterDesc;
 import org.junit.Test;
 
 /**
@@ -89,10 +95,15 @@ public class TestVectorFilterOperator {
     ExprNodeColumnDesc col1Expr = new  ExprNodeColumnDesc(Long.class, "col1", 
"table", false);
     List<String> columns = new ArrayList<String>();
     columns.add("col1");
-    VectorizationContext vc = new VectorizationContext("name", columns);
     FilterDesc fdesc = new FilterDesc();
     fdesc.setPredicate(col1Expr);
-    return new VectorFilterOperator(new CompilationOpContext(), vc, fdesc);
+
+    Operator<? extends OperatorDesc> filterOp = 
+        OperatorFactory.get(new CompilationOpContext(), fdesc);
+
+    VectorizationContext vc = new VectorizationContext("name", columns);
+
+    return (VectorFilterOperator) Vectorizer.vectorizeFilterOperator(filterOp, 
vc);
   }
 
   @Test

http://git-wip-us.apache.org/repos/asf/hive/blob/f923db0b/ql/src/test/org/apache/hadoop/hive/ql/exec/vector/TestVectorGroupByOperator.java
----------------------------------------------------------------------
diff --git 
a/ql/src/test/org/apache/hadoop/hive/ql/exec/vector/TestVectorGroupByOperator.java
 
b/ql/src/test/org/apache/hadoop/hive/ql/exec/vector/TestVectorGroupByOperator.java
index f5b5d9d..086c0b7 100644
--- 
a/ql/src/test/org/apache/hadoop/hive/ql/exec/vector/TestVectorGroupByOperator.java
+++ 
b/ql/src/test/org/apache/hadoop/hive/ql/exec/vector/TestVectorGroupByOperator.java
@@ -39,16 +39,20 @@ import java.util.Set;
 import org.apache.hadoop.hive.common.type.HiveDecimal;
 import org.apache.hadoop.hive.conf.HiveConf;
 import org.apache.hadoop.hive.ql.CompilationOpContext;
+import org.apache.hadoop.hive.ql.exec.Operator;
+import org.apache.hadoop.hive.ql.exec.OperatorFactory;
 import org.apache.hadoop.hive.ql.exec.vector.util.FakeCaptureOutputOperator;
 import org.apache.hadoop.hive.ql.exec.vector.util.FakeVectorRowBatchFromConcat;
 import 
org.apache.hadoop.hive.ql.exec.vector.util.FakeVectorRowBatchFromLongIterables;
 import 
org.apache.hadoop.hive.ql.exec.vector.util.FakeVectorRowBatchFromObjectIterables;
 import 
org.apache.hadoop.hive.ql.exec.vector.util.FakeVectorRowBatchFromRepeats;
 import org.apache.hadoop.hive.ql.metadata.HiveException;
+import org.apache.hadoop.hive.ql.optimizer.physical.Vectorizer;
 import org.apache.hadoop.hive.ql.plan.AggregationDesc;
 import org.apache.hadoop.hive.ql.plan.ExprNodeColumnDesc;
 import org.apache.hadoop.hive.ql.plan.ExprNodeDesc;
 import org.apache.hadoop.hive.ql.plan.GroupByDesc;
+import org.apache.hadoop.hive.ql.plan.OperatorDesc;
 import org.apache.hadoop.hive.ql.plan.VectorGroupByDesc;
 import org.apache.hadoop.hive.ql.plan.VectorGroupByDesc.ProcessingMode;
 import org.apache.hadoop.hive.ql.udf.generic.GenericUDAFEvaluator;
@@ -128,9 +132,11 @@ public class TestVectorGroupByOperator {
     outputColumnNames.add("_col0");
 
     GroupByDesc desc = new GroupByDesc();
+    desc.setVectorDesc(new VectorGroupByDesc());
+
     desc.setOutputColumnNames(outputColumnNames);
     desc.setAggregators(aggs);
-    desc.getVectorDesc().setProcessingMode(ProcessingMode.GLOBAL);
+    ((VectorGroupByDesc) 
desc.getVectorDesc()).setProcessingMode(ProcessingMode.GLOBAL);
 
     return desc;
   }
@@ -146,6 +152,8 @@ public class TestVectorGroupByOperator {
     outputColumnNames.add("_col0");
 
     GroupByDesc desc = new GroupByDesc();
+    desc.setVectorDesc(new VectorGroupByDesc());
+
     desc.setOutputColumnNames(outputColumnNames);
     desc.setAggregators(aggs);
 
@@ -162,7 +170,7 @@ public class TestVectorGroupByOperator {
       TypeInfo keyTypeInfo) {
 
     GroupByDesc desc = buildGroupByDescType(ctx, aggregate, 
GenericUDAFEvaluator.Mode.PARTIAL1, column, dataTypeInfo);
-    desc.getVectorDesc().setProcessingMode(ProcessingMode.HASH);
+    ((VectorGroupByDesc) 
desc.getVectorDesc()).setProcessingMode(ProcessingMode.HASH);
 
     ExprNodeDesc keyExp = buildColumnDesc(ctx, key, keyTypeInfo);
     ArrayList<ExprNodeDesc> keys = new ArrayList<ExprNodeDesc>();
@@ -196,7 +204,11 @@ public class TestVectorGroupByOperator {
     desc.setMemoryThreshold(treshold);
 
     CompilationOpContext cCtx = new CompilationOpContext();
-    VectorGroupByOperator vgo = new VectorGroupByOperator(cCtx, ctx, desc);
+
+    Operator<? extends OperatorDesc> groupByOp = OperatorFactory.get(cCtx, 
desc);
+
+    VectorGroupByOperator vgo =
+        (VectorGroupByOperator) Vectorizer.vectorizeGroupByOperator(groupByOp, 
ctx);
 
     FakeCaptureOutputOperator out = 
FakeCaptureOutputOperator.addCaptureOutputChild(cCtx, vgo);
     vgo.initialize(hconf, null);
@@ -1735,13 +1747,19 @@ public class TestVectorGroupByOperator {
     }
 
     GroupByDesc desc = new GroupByDesc();
+    desc.setVectorDesc(new VectorGroupByDesc());
+
     desc.setOutputColumnNames(outputColumnNames);
     desc.setAggregators(aggs);
     desc.setKeys(keysDesc);
-    desc.getVectorDesc().setProcessingMode(ProcessingMode.HASH);
+    ((VectorGroupByDesc) 
desc.getVectorDesc()).setProcessingMode(ProcessingMode.HASH);
 
     CompilationOpContext cCtx = new CompilationOpContext();
-    VectorGroupByOperator vgo = new VectorGroupByOperator(cCtx, ctx, desc);
+
+    Operator<? extends OperatorDesc> groupByOp = OperatorFactory.get(cCtx, 
desc);
+
+    VectorGroupByOperator vgo =
+        (VectorGroupByOperator) Vectorizer.vectorizeGroupByOperator(groupByOp, 
ctx);
 
     FakeCaptureOutputOperator out = 
FakeCaptureOutputOperator.addCaptureOutputChild(cCtx, vgo);
     vgo.initialize(hconf, null);
@@ -1846,9 +1864,11 @@ public class TestVectorGroupByOperator {
     outputColumnNames.add("_col1");
 
     GroupByDesc desc = new GroupByDesc();
+    desc.setVectorDesc(new VectorGroupByDesc());
+
     desc.setOutputColumnNames(outputColumnNames);
     desc.setAggregators(aggs);
-    desc.getVectorDesc().setProcessingMode(ProcessingMode.HASH);
+    ((VectorGroupByDesc) 
desc.getVectorDesc()).setProcessingMode(ProcessingMode.HASH);
 
     ExprNodeDesc keyExp = buildColumnDesc(ctx, "Key",
         TypeInfoFactory.getPrimitiveTypeInfo(data.getTypes()[0]));
@@ -1857,7 +1877,11 @@ public class TestVectorGroupByOperator {
     desc.setKeys(keysDesc);
 
     CompilationOpContext cCtx = new CompilationOpContext();
-    VectorGroupByOperator vgo = new VectorGroupByOperator(cCtx, ctx, desc);
+
+    Operator<? extends OperatorDesc> groupByOp = OperatorFactory.get(cCtx, 
desc);
+
+    VectorGroupByOperator vgo =
+        (VectorGroupByOperator) Vectorizer.vectorizeGroupByOperator(groupByOp, 
ctx);
 
     FakeCaptureOutputOperator out = 
FakeCaptureOutputOperator.addCaptureOutputChild(cCtx, vgo);
     vgo.initialize(hconf, null);
@@ -2252,10 +2276,14 @@ public class TestVectorGroupByOperator {
     VectorizationContext ctx = new VectorizationContext("name", 
mapColumnNames);
 
     GroupByDesc desc = buildGroupByDescCountStar (ctx);
-    desc.getVectorDesc().setProcessingMode(ProcessingMode.HASH);
+    ((VectorGroupByDesc) 
desc.getVectorDesc()).setProcessingMode(ProcessingMode.HASH);
 
     CompilationOpContext cCtx = new CompilationOpContext();
-    VectorGroupByOperator vgo = new VectorGroupByOperator(cCtx, ctx, desc);
+
+    Operator<? extends OperatorDesc> groupByOp = OperatorFactory.get(cCtx, 
desc);
+
+    VectorGroupByOperator vgo =
+        (VectorGroupByOperator) Vectorizer.vectorizeGroupByOperator(groupByOp, 
ctx);
 
     FakeCaptureOutputOperator out = 
FakeCaptureOutputOperator.addCaptureOutputChild(cCtx, vgo);
     vgo.initialize(hconf, null);
@@ -2283,10 +2311,14 @@ public class TestVectorGroupByOperator {
     VectorizationContext ctx = new VectorizationContext("name", 
mapColumnNames);
 
     GroupByDesc desc = buildGroupByDescType(ctx, "count", 
GenericUDAFEvaluator.Mode.FINAL, "A", TypeInfoFactory.longTypeInfo);
-    VectorGroupByDesc vectorDesc = desc.getVectorDesc();
+    VectorGroupByDesc vectorDesc = (VectorGroupByDesc) desc.getVectorDesc();
     vectorDesc.setProcessingMode(ProcessingMode.GLOBAL);  // Use GLOBAL when 
no key for Reduce.
     CompilationOpContext cCtx = new CompilationOpContext();
-    VectorGroupByOperator vgo = new VectorGroupByOperator(cCtx, ctx, desc);
+
+    Operator<? extends OperatorDesc> groupByOp = OperatorFactory.get(cCtx, 
desc);
+
+    VectorGroupByOperator vgo =
+        (VectorGroupByOperator) Vectorizer.vectorizeGroupByOperator(groupByOp, 
ctx);
 
     FakeCaptureOutputOperator out = 
FakeCaptureOutputOperator.addCaptureOutputChild(cCtx, vgo);
     vgo.initialize(hconf, null);
@@ -2318,7 +2350,11 @@ public class TestVectorGroupByOperator {
         TypeInfoFactory.stringTypeInfo);
 
     CompilationOpContext cCtx = new CompilationOpContext();
-    VectorGroupByOperator vgo = new VectorGroupByOperator(cCtx, ctx, desc);
+
+    Operator<? extends OperatorDesc> groupByOp = OperatorFactory.get(cCtx, 
desc);
+
+    VectorGroupByOperator vgo =
+        (VectorGroupByOperator) Vectorizer.vectorizeGroupByOperator(groupByOp, 
ctx);
 
     FakeCaptureOutputOperator out = 
FakeCaptureOutputOperator.addCaptureOutputChild(cCtx, vgo);
     vgo.initialize(hconf, null);
@@ -2350,7 +2386,11 @@ public class TestVectorGroupByOperator {
         buildGroupByDescType(ctx, aggregateName, 
GenericUDAFEvaluator.Mode.PARTIAL1, "A", TypeInfoFactory.getDecimalTypeInfo(30, 
4));
 
     CompilationOpContext cCtx = new CompilationOpContext();
-    VectorGroupByOperator vgo = new VectorGroupByOperator(cCtx, ctx, desc);
+
+    Operator<? extends OperatorDesc> groupByOp = OperatorFactory.get(cCtx, 
desc);
+
+    VectorGroupByOperator vgo =
+        (VectorGroupByOperator) Vectorizer.vectorizeGroupByOperator(groupByOp, 
ctx);
 
     FakeCaptureOutputOperator out = 
FakeCaptureOutputOperator.addCaptureOutputChild(cCtx, vgo);
     vgo.initialize(hconf, null);
@@ -2383,7 +2423,11 @@ public class TestVectorGroupByOperator {
         TypeInfoFactory.doubleTypeInfo);
 
     CompilationOpContext cCtx = new CompilationOpContext();
-    VectorGroupByOperator vgo = new VectorGroupByOperator(cCtx, ctx, desc);
+
+    Operator<? extends OperatorDesc> groupByOp = OperatorFactory.get(cCtx, 
desc);
+
+    VectorGroupByOperator vgo =
+        (VectorGroupByOperator) Vectorizer.vectorizeGroupByOperator(groupByOp, 
ctx);
 
     FakeCaptureOutputOperator out = 
FakeCaptureOutputOperator.addCaptureOutputChild(cCtx, vgo);
     vgo.initialize(hconf, null);
@@ -2414,7 +2458,11 @@ public class TestVectorGroupByOperator {
     GroupByDesc desc = buildGroupByDescType(ctx, aggregateName, 
GenericUDAFEvaluator.Mode.PARTIAL1, "A", TypeInfoFactory.longTypeInfo);
 
     CompilationOpContext cCtx = new CompilationOpContext();
-    VectorGroupByOperator vgo = new VectorGroupByOperator(cCtx, ctx, desc);
+
+    Operator<? extends OperatorDesc> groupByOp = OperatorFactory.get(cCtx, 
desc);
+
+    VectorGroupByOperator vgo =
+        (VectorGroupByOperator) Vectorizer.vectorizeGroupByOperator(groupByOp, 
ctx);
 
     FakeCaptureOutputOperator out = 
FakeCaptureOutputOperator.addCaptureOutputChild(cCtx, vgo);
     vgo.initialize(null, null);
@@ -2449,7 +2497,11 @@ public class TestVectorGroupByOperator {
         TypeInfoFactory.longTypeInfo, "Key", TypeInfoFactory.longTypeInfo);
 
     CompilationOpContext cCtx = new CompilationOpContext();
-    VectorGroupByOperator vgo = new VectorGroupByOperator(cCtx, ctx, desc);
+
+    Operator<? extends OperatorDesc> groupByOp = OperatorFactory.get(cCtx, 
desc);
+
+    VectorGroupByOperator vgo =
+        (VectorGroupByOperator) Vectorizer.vectorizeGroupByOperator(groupByOp, 
ctx);
 
     FakeCaptureOutputOperator out = 
FakeCaptureOutputOperator.addCaptureOutputChild(cCtx, vgo);
     vgo.initialize(hconf, null);
@@ -2515,7 +2567,11 @@ public class TestVectorGroupByOperator {
        dataTypeInfo, "Key", TypeInfoFactory.stringTypeInfo);
 
     CompilationOpContext cCtx = new CompilationOpContext();
-    VectorGroupByOperator vgo = new VectorGroupByOperator(cCtx, ctx, desc);
+
+    Operator<? extends OperatorDesc> groupByOp = OperatorFactory.get(cCtx, 
desc);
+
+    VectorGroupByOperator vgo =
+        (VectorGroupByOperator) Vectorizer.vectorizeGroupByOperator(groupByOp, 
ctx);
 
     FakeCaptureOutputOperator out = 
FakeCaptureOutputOperator.addCaptureOutputChild(cCtx, vgo);
     vgo.initialize(hconf, null);

http://git-wip-us.apache.org/repos/asf/hive/blob/f923db0b/ql/src/test/org/apache/hadoop/hive/ql/exec/vector/TestVectorSelectOperator.java
----------------------------------------------------------------------
diff --git 
a/ql/src/test/org/apache/hadoop/hive/ql/exec/vector/TestVectorSelectOperator.java
 
b/ql/src/test/org/apache/hadoop/hive/ql/exec/vector/TestVectorSelectOperator.java
index 779177a..614b1d1 100644
--- 
a/ql/src/test/org/apache/hadoop/hive/ql/exec/vector/TestVectorSelectOperator.java
+++ 
b/ql/src/test/org/apache/hadoop/hive/ql/exec/vector/TestVectorSelectOperator.java
@@ -26,6 +26,7 @@ import java.util.List;
 import java.util.Map;
 
 import org.apache.hadoop.hive.ql.CompilationOpContext;
+import org.apache.hadoop.hive.ql.exec.vector.expressions.VectorExpression;
 import org.apache.hadoop.hive.ql.exec.vector.util.VectorizedRowGroupGenUtil;
 import org.apache.hadoop.hive.ql.metadata.HiveException;
 import org.apache.hadoop.hive.ql.plan.ExprNodeColumnDesc;
@@ -33,6 +34,7 @@ import org.apache.hadoop.hive.ql.plan.ExprNodeDesc;
 import org.apache.hadoop.hive.ql.plan.ExprNodeGenericFuncDesc;
 import org.apache.hadoop.hive.ql.plan.OperatorDesc;
 import org.apache.hadoop.hive.ql.plan.SelectDesc;
+import org.apache.hadoop.hive.ql.plan.VectorSelectDesc;
 import org.apache.hadoop.hive.ql.udf.generic.GenericUDF;
 import org.apache.hadoop.hive.ql.udf.generic.GenericUDFOPPlus;
 import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspector;
@@ -51,6 +53,7 @@ public class TestVectorSelectOperator {
     public ValidatorVectorSelectOperator(CompilationOpContext ctx,
         VectorizationContext ctxt, OperatorDesc conf) throws HiveException {
       super(ctx, ctxt, conf);
+
       initializeOp(null);
     }
 
@@ -115,6 +118,19 @@ public class TestVectorSelectOperator {
     outputColNames.add("_col1");
     selDesc.setOutputColumnNames(outputColNames);
 
+    // CONSIDER unwinding ValidatorVectorSelectOperator as a subclass of 
VectorSelectOperator.
+    VectorSelectDesc vectorSelectDesc = new VectorSelectDesc();
+    selDesc.setVectorDesc(vectorSelectDesc);
+    List<ExprNodeDesc> selectColList = selDesc.getColList();
+    VectorExpression[] vectorSelectExprs = new 
VectorExpression[selectColList.size()];
+    for (int i = 0; i < selectColList.size(); i++) {
+      ExprNodeDesc expr = selectColList.get(i);
+      VectorExpression ve = vc.getVectorExpression(expr);
+      vectorSelectExprs[i] = ve;
+    }
+    vectorSelectDesc.setSelectExpressions(vectorSelectExprs);
+    vectorSelectDesc.setProjectedOutputColumns(new int[] {3, 2});
+
     ValidatorVectorSelectOperator vso = new ValidatorVectorSelectOperator(
         new CompilationOpContext(), vc, selDesc);
 

http://git-wip-us.apache.org/repos/asf/hive/blob/f923db0b/ql/src/test/org/apache/hadoop/hive/ql/optimizer/physical/TestVectorizer.java
----------------------------------------------------------------------
diff --git 
a/ql/src/test/org/apache/hadoop/hive/ql/optimizer/physical/TestVectorizer.java 
b/ql/src/test/org/apache/hadoop/hive/ql/optimizer/physical/TestVectorizer.java
index 3295372..ccd1059 100644
--- 
a/ql/src/test/org/apache/hadoop/hive/ql/optimizer/physical/TestVectorizer.java
+++ 
b/ql/src/test/org/apache/hadoop/hive/ql/optimizer/physical/TestVectorizer.java
@@ -96,6 +96,8 @@ public class TestVectorizer {
     outputColumnNames.add("_col0");
 
     GroupByDesc desc = new GroupByDesc();
+    desc.setVectorDesc(new VectorGroupByDesc());
+
     desc.setOutputColumnNames(outputColumnNames);
     ArrayList<AggregationDesc> aggDescList = new ArrayList<AggregationDesc>();
     aggDescList.add(aggDesc);
@@ -106,13 +108,14 @@ public class TestVectorizer {
     grpByKeys.add(colExprB);
     desc.setKeys(grpByKeys);
 
-    GroupByOperator gbyOp = new GroupByOperator(new CompilationOpContext());
-    gbyOp.setConf(desc);
+    Operator<? extends OperatorDesc> gbyOp = OperatorFactory.get(new 
CompilationOpContext(), desc);
+
     desc.setMode(GroupByDesc.Mode.HASH);
 
     Vectorizer v = new Vectorizer();
+    v.testSetCurrentBaseWork(new MapWork());
     Assert.assertTrue(v.validateMapWorkOperator(gbyOp, null, false));
-    VectorGroupByOperator vectorOp = (VectorGroupByOperator) 
v.vectorizeOperator(gbyOp, vContext, false);
+    VectorGroupByOperator vectorOp = (VectorGroupByOperator) 
v.vectorizeOperator(gbyOp, vContext, false, null);
     Assert.assertEquals(VectorUDAFSumLong.class, 
vectorOp.getAggregators()[0].getClass());
     VectorUDAFSumLong udaf = (VectorUDAFSumLong) vectorOp.getAggregators()[0];
     Assert.assertEquals(FuncAbsLongToLong.class, 
udaf.getInputExpression().getClass());
@@ -147,8 +150,9 @@ public class TestVectorizer {
     andExprDesc.setChildren(children3);
 
     Vectorizer v = new Vectorizer();
-    Assert.assertFalse(v.validateExprNodeDesc(andExprDesc, 
VectorExpressionDescriptor.Mode.FILTER));
-    Assert.assertFalse(v.validateExprNodeDesc(andExprDesc, 
VectorExpressionDescriptor.Mode.PROJECTION));
+    v.testSetCurrentBaseWork(new MapWork());
+    Assert.assertFalse(v.validateExprNodeDesc(andExprDesc, "test", 
VectorExpressionDescriptor.Mode.FILTER));
+    Assert.assertFalse(v.validateExprNodeDesc(andExprDesc, "test", 
VectorExpressionDescriptor.Mode.PROJECTION));
   }
 
   /**
@@ -196,6 +200,7 @@ public class TestVectorizer {
     map.setConf(mjdesc);
 
     Vectorizer vectorizer = new Vectorizer();
+    vectorizer.testSetCurrentBaseWork(new MapWork());
     Assert.assertTrue(vectorizer.validateMapWorkOperator(map, null, false));
   }
 
@@ -212,6 +217,7 @@ public class TestVectorizer {
       map.setConf(mjdesc);
 
       Vectorizer vectorizer = new Vectorizer();
+      vectorizer.testSetCurrentBaseWork(new MapWork());
       Assert.assertTrue(vectorizer.validateMapWorkOperator(map, null, false));
   }
 }

http://git-wip-us.apache.org/repos/asf/hive/blob/f923db0b/ql/src/test/queries/clientpositive/schema_evol_orc_vec_part.q
----------------------------------------------------------------------
diff --git a/ql/src/test/queries/clientpositive/schema_evol_orc_vec_part.q 
b/ql/src/test/queries/clientpositive/schema_evol_orc_vec_part.q
index 7e66cbc..48903d2 100644
--- a/ql/src/test/queries/clientpositive/schema_evol_orc_vec_part.q
+++ b/ql/src/test/queries/clientpositive/schema_evol_orc_vec_part.q
@@ -1,4 +1,4 @@
-set hive.explain.user=true;
+set hive.explain.user=false;
 set hive.mapred.mode=nonstrict;
 set hive.cli.print.header=true;
 SET hive.exec.schema.evolution=true;
@@ -38,7 +38,7 @@ alter table part_add_int_permute_select add columns(c int);
 
 insert into table part_add_int_permute_select partition(part=1) VALUES (2, 
2222, 'new', 3333);
 
-explain
+explain vectorization detail
 select insert_num,part,a,b from part_add_int_permute_select;
 
 -- SELECT permutation columns to make sure NULL defaulting works right
@@ -61,7 +61,7 @@ alter table part_add_int_string_permute_select add columns(c 
int, d string);
 
 insert into table part_add_int_string_permute_select partition(part=1) VALUES 
(2, 2222, 'new', 3333, '4444');
 
-explain
+explain vectorization detail
 select insert_num,part,a,b from part_add_int_string_permute_select;
 
 -- SELECT permutation columns to make sure NULL defaulting works right
@@ -93,7 +93,7 @@ alter table part_change_string_group_double replace columns 
(insert_num int, c1
 
 insert into table part_change_string_group_double partition(part=1) SELECT 
insert_num, double1, double1, double1, 'new' FROM schema_evolution_data WHERE 
insert_num = 111;
 
-explain
+explain vectorization detail
 select insert_num,part,c1,c2,c3,b from part_change_string_group_double;
 
 select insert_num,part,c1,c2,c3,b from part_change_string_group_double;
@@ -116,7 +116,7 @@ alter table 
part_change_date_group_string_group_date_timestamp replace columns(i
 
 insert into table part_change_date_group_string_group_date_timestamp 
partition(part=1) VALUES (111, 'filler', 'filler', 'filler', 'filler', 
'filler', 'filler', 'filler', 'filler', 'filler', 'filler', 'new');
 
-explain
+explain vectorization detail
 select insert_num,part,c1,c2,c3,c4,c5,c6,c7,c8,c9,c10,b from 
part_change_date_group_string_group_date_timestamp;
 
 select insert_num,part,c1,c2,c3,c4,c5,c6,c7,c8,c9,c10,b from 
part_change_date_group_string_group_date_timestamp;
@@ -164,7 +164,7 @@ insert into table 
part_change_numeric_group_string_group_multi_ints_string_group
             'filler', 'filler', 'filler', 'filler', 'filler', 'filler', 
'filler', 'filler',
             'new');
 
-explain
+explain vectorization detail
 select 
insert_num,part,c1,c2,c3,c4,c5,c6,c7,c8,c9,c10,c11,c12,c13,c14,c15,c16,c17,c18,c19,c20,b
 from part_change_numeric_group_string_group_multi_ints_string_group;
 
 select 
insert_num,part,c1,c2,c3,c4,c5,c6,c7,c8,c9,c10,c11,c12,c13,c14,c15,c16,c17,c18,c19,c20,b
 from part_change_numeric_group_string_group_multi_ints_string_group;
@@ -207,7 +207,7 @@ insert into table 
part_change_numeric_group_string_group_floating_string_group p
              'filler', 'filler', 'filler', 'filler', 'filler', 'filler',
              'new');
 
-explain
+explain vectorization detail
 select insert_num,part,c1,c2,c3,c4,c5,c6,c7,c8,c9,c10,c11,c12,c13,c14,c15,b 
from part_change_numeric_group_string_group_floating_string_group;
 
 select insert_num,part,c1,c2,c3,c4,c5,c6,c7,c8,c9,c10,c11,c12,c13,c14,c15,b 
from part_change_numeric_group_string_group_floating_string_group;
@@ -249,7 +249,7 @@ insert into table 
part_change_string_group_string_group_string partition(part=1)
           'filler', 'filler', 'filler',
           'new');
 
-explain
+explain vectorization detail
 select insert_num,part,c1,c2,c3,c4,c5,c6,c7,c8,c9,c10,b from 
part_change_string_group_string_group_string;
 
 select insert_num,part,c1,c2,c3,c4,c5,c6,c7,c8,c9,c10,b from 
part_change_string_group_string_group_string;
@@ -299,7 +299,7 @@ insert into table 
part_change_lower_to_higher_numeric_group_tinyint_to_bigint pa
             1234.5678, 9876.543, 789.321,
            'new');
 
-explain
+explain vectorization detail
 select 
insert_num,part,c1,c2,c3,c4,c5,c6,c7,c8,c9,c10,c11,c12,c13,c14,c15,c16,c17,c18,b
 from part_change_lower_to_higher_numeric_group_tinyint_to_bigint;
 
 select 
insert_num,part,c1,c2,c3,c4,c5,c6,c7,c8,c9,c10,c11,c12,c13,c14,c15,c16,c17,c18,b
 from part_change_lower_to_higher_numeric_group_tinyint_to_bigint;
@@ -330,7 +330,7 @@ alter table 
part_change_lower_to_higher_numeric_group_decimal_to_float replace c
 
 insert into table part_change_lower_to_higher_numeric_group_decimal_to_float 
partition(part=1) VALUES (111, 1234.5678, 9876.543, 1234.5678, 'new');
 
-explain
+explain vectorization detail
 select insert_num,part,c1,c2,c3,b from 
part_change_lower_to_higher_numeric_group_decimal_to_float;
 
 select insert_num,part,c1,c2,c3,b from 
part_change_lower_to_higher_numeric_group_decimal_to_float;

http://git-wip-us.apache.org/repos/asf/hive/blob/f923db0b/ql/src/test/queries/clientpositive/schema_evol_orc_vec_part_all_complex.q
----------------------------------------------------------------------
diff --git 
a/ql/src/test/queries/clientpositive/schema_evol_orc_vec_part_all_complex.q 
b/ql/src/test/queries/clientpositive/schema_evol_orc_vec_part_all_complex.q
index ac747e6..45afd9d 100644
--- a/ql/src/test/queries/clientpositive/schema_evol_orc_vec_part_all_complex.q
+++ b/ql/src/test/queries/clientpositive/schema_evol_orc_vec_part_all_complex.q
@@ -1,4 +1,4 @@
-set hive.explain.user=true;
+set hive.explain.user=false;
 set hive.mapred.mode=nonstrict;
 set hive.cli.print.header=true;
 SET hive.exec.schema.evolution=true;
@@ -51,7 +51,7 @@ load data local inpath 
'../../data/files/schema_evolution/complex_struct1_c.txt'
 
 insert into table part_change_various_various_struct1 partition(part=1) select 
* from complex_struct1_c_txt;
 
-explain
+ explain vectorization detail
 select insert_num,part,s1,b from part_change_various_various_struct1;
 
 select insert_num,part,s1,b from part_change_various_various_struct1;
@@ -111,7 +111,7 @@ load data local inpath 
'../../data/files/schema_evolution/complex_struct2_d.txt'
 
 insert into table part_add_various_various_struct2 partition(part=1) select * 
from complex_struct2_d_txt;
 
-explain
+explain vectorization detail
 select insert_num,part,b,s2 from part_add_various_various_struct2;
 
 select insert_num,part,b,s2 from part_add_various_various_struct2;
@@ -155,7 +155,7 @@ load data local inpath 
'../../data/files/schema_evolution/complex_struct4_c.txt'
 
 insert into table part_add_to_various_various_struct4 partition(part=1) select 
* from complex_struct4_c_txt;
 
-explain
+explain vectorization detail
 select insert_num,part,b,s3 from part_add_to_various_various_struct4;
 
 select insert_num,part,b,s3 from part_add_to_various_various_struct4;

http://git-wip-us.apache.org/repos/asf/hive/blob/f923db0b/ql/src/test/queries/clientpositive/schema_evol_orc_vec_part_all_primitive.q
----------------------------------------------------------------------
diff --git 
a/ql/src/test/queries/clientpositive/schema_evol_orc_vec_part_all_primitive.q 
b/ql/src/test/queries/clientpositive/schema_evol_orc_vec_part_all_primitive.q
index d3898a8..b266a67 100644
--- 
a/ql/src/test/queries/clientpositive/schema_evol_orc_vec_part_all_primitive.q
+++ 
b/ql/src/test/queries/clientpositive/schema_evol_orc_vec_part_all_primitive.q
@@ -1,4 +1,4 @@
-set hive.explain.user=true;
+set hive.explain.user=false;
 set hive.mapred.mode=nonstrict;
 set hive.cli.print.header=true;
 SET hive.exec.schema.evolution=true;
@@ -72,7 +72,7 @@ insert into table 
part_change_various_various_boolean_to_bigint partition(part=1
              bigint1, bigint1, bigint1, bigint1, bigint1, bigint1, bigint1, 
bigint1, bigint1, bigint1, bigint1, 
               'new' FROM schema_evolution_data;
 
-explain
+explain vectorization detail
 select 
insert_num,part,c1,c2,c3,c4,c5,c6,c7,c8,c9,c10,c11,c12,c13,c14,c15,c16,c17,c18,c19,c20,c21,c22,c23,c24,c25,c26,c27,c28,c29,c30,c31,c32,c33,c34,c35,c36,c37,c38,c39,c40,c41,c42,c43,c44,c45,c46,c47,c48,c49,c50,c51,c52,c53,b
 from part_change_various_various_boolean_to_bigint;
 
 select 
insert_num,part,c1,c2,c3,c4,c5,c6,c7,c8,c9,c10,c11,c12,c13,c14,c15,c16,c17,c18,c19,c20,c21,c22,c23,c24,c25,c26,c27,c28,c29,c30,c31,c32,c33,c34,c35,c36,c37,c38,c39,c40,c41,c42,c43,c44,c45,c46,c47,c48,c49,c50,c51,c52,c53,b
 from part_change_various_various_boolean_to_bigint;
@@ -114,7 +114,7 @@ insert into table 
part_change_various_various_decimal_to_double partition(part=1
              double1, double1, double1, double1, double1, double1, double1, 
double1, double1, double1, double1,
              'new' FROM schema_evolution_data_2 WHERE insert_num=111;
 
-explain
+explain vectorization detail
 select 
insert_num,part,c1,c2,c3,c4,c5,c6,c7,c8,c9,c10,c11,c12,c13,c14,c15,c16,c17,c18,c19,c20,c21,c22,c23,c24,c25,c26,c27,c28,c29,c30,c31,c32,c33,b
 from part_change_various_various_decimal_to_double;
 
 select 
insert_num,part,c1,c2,c3,c4,c5,c6,c7,c8,c9,c10,c11,c12,c13,c14,c15,c16,c17,c18,c19,c20,c21,c22,c23,c24,c25,c26,c27,c28,c29,c30,c31,c32,c33,b
 from part_change_various_various_decimal_to_double;
@@ -138,7 +138,7 @@ alter table part_change_various_various_timestamp replace 
columns (insert_num in
 
 insert into table part_change_various_various_timestamp partition(part=1) 
SELECT insert_num, timestamp1, timestamp1, timestamp1, timestamp1, timestamp1, 
timestamp1, timestamp1, timestamp1, timestamp1, timestamp1, timestamp1, 
timestamp1, 'new' FROM schema_evolution_data_2 WHERE insert_num=111;
 
-explain
+explain vectorization detail
 select insert_num,part,c1,c2,c3,c4,c5,c6,c7,c8,c9,c10,c11,c12,b from 
part_change_various_various_timestamp;
 
 select insert_num,part,c1,c2,c3,c4,c5,c6,c7,c8,c9,c10,c11,c12,b from 
part_change_various_various_timestamp;
@@ -159,7 +159,7 @@ alter table part_change_various_various_date replace 
columns (insert_num int, c1
 
 insert into table part_change_various_various_date partition(part=1) SELECT 
insert_num, date1, date1, date1, date1, 'new' FROM schema_evolution_data_2 
WHERE insert_num=111;
 
-explain
+explain vectorization detail
 select insert_num,part,c1,c2,c3,c4,b from part_change_various_various_date;
 
 select insert_num,part,c1,c2,c3,c4,b from part_change_various_various_date;
@@ -198,7 +198,7 @@ load data local inpath 
'../../data/files/schema_evolution/same_type1_c.txt' over
 
 insert into table part_change_same_type_different_params partition(part=2) 
select * from same_type1_c_txt;
 
-explain
+explain vectorization detail
 select insert_num,part,c1,c2,c3,c4,c5,c6,b from 
part_change_same_type_different_params;
 
 select insert_num,part,c1,c2,c3,c4,c5,c6,b from 
part_change_same_type_different_params;

http://git-wip-us.apache.org/repos/asf/hive/blob/f923db0b/ql/src/test/queries/clientpositive/schema_evol_orc_vec_table.q
----------------------------------------------------------------------
diff --git a/ql/src/test/queries/clientpositive/schema_evol_orc_vec_table.q 
b/ql/src/test/queries/clientpositive/schema_evol_orc_vec_table.q
index ffaa07b..866942e 100644
--- a/ql/src/test/queries/clientpositive/schema_evol_orc_vec_table.q
+++ b/ql/src/test/queries/clientpositive/schema_evol_orc_vec_table.q
@@ -1,4 +1,4 @@
-set hive.explain.user=true;
+set hive.explain.user=false;
 set hive.cli.print.header=true;
 SET hive.exec.schema.evolution=true;
 SET hive.vectorized.use.vectorized.input.format=true;
@@ -36,7 +36,7 @@ alter table table_add_int_permute_select add columns(c int);
 
 insert into table table_add_int_permute_select VALUES (111, 80000, 'new', 
80000);
 
-explain
+explain vectorization detail
 select insert_num,a,b from table_add_int_permute_select;
 
 -- SELECT permutation columns to make sure NULL defaulting works right
@@ -59,7 +59,7 @@ alter table table_add_int_string_permute_select add columns(c 
int, d string);
 
 insert into table table_add_int_string_permute_select VALUES (111, 80000, 
'new', 80000, 'filler');
 
-explain
+explain vectorization detail
 select insert_num,a,b from table_add_int_string_permute_select;
 
 -- SELECT permutation columns to make sure NULL defaulting works right
@@ -91,7 +91,7 @@ alter table table_change_string_group_double replace columns 
(insert_num int, c1
 
 insert into table table_change_string_group_double VALUES (111, 789.321, 
789.321, 789.321, 'new');
 
-explain
+explain vectorization detail
 select insert_num,c1,c2,c3,b from table_change_string_group_double;
 
 select insert_num,c1,c2,c3,b from table_change_string_group_double;
@@ -158,7 +158,7 @@ insert into table 
table_change_numeric_group_string_group_multi_ints_string_grou
             'filler', 'filler', 'filler', 'filler', 'filler', 'filler', 
'filler', 'filler',
             'new');
 
-explain
+explain vectorization detail
 select 
insert_num,c1,c2,c3,c4,c5,c6,c7,c8,c9,c10,c11,c12,c13,c14,c15,c16,c17,c18,c19,c20,b
 from table_change_numeric_group_string_group_multi_ints_string_group;
 
 select 
insert_num,c1,c2,c3,c4,c5,c6,c7,c8,c9,c10,c11,c12,c13,c14,c15,c16,c17,c18,c19,c20,b
 from table_change_numeric_group_string_group_multi_ints_string_group;
@@ -201,7 +201,7 @@ insert into table 
table_change_numeric_group_string_group_floating_string_group
              'filler', 'filler', 'filler', 'filler', 'filler', 'filler',
              'new');
 
-explain
+explain vectorization detail
 select insert_num,c1,c2,c3,c4,c5,c6,c7,c8,c9,c10,c11,c12,c13,c14,c15,b from 
table_change_numeric_group_string_group_floating_string_group;
 
 select insert_num,c1,c2,c3,c4,c5,c6,c7,c8,c9,c10,c11,c12,c13,c14,c15,b from 
table_change_numeric_group_string_group_floating_string_group;

http://git-wip-us.apache.org/repos/asf/hive/blob/f923db0b/ql/src/test/queries/clientpositive/schema_evol_text_vec_part.q
----------------------------------------------------------------------
diff --git a/ql/src/test/queries/clientpositive/schema_evol_text_vec_part.q 
b/ql/src/test/queries/clientpositive/schema_evol_text_vec_part.q
index 6582035..77c863a 100644
--- a/ql/src/test/queries/clientpositive/schema_evol_text_vec_part.q
+++ b/ql/src/test/queries/clientpositive/schema_evol_text_vec_part.q
@@ -1,4 +1,4 @@
-set hive.explain.user=true;
+set hive.explain.user=false;
 set hive.mapred.mode=nonstrict;
 set hive.cli.print.header=true;
 SET hive.exec.schema.evolution=true;
@@ -39,7 +39,7 @@ alter table part_add_int_permute_select add columns(c int);
 
 insert into table part_add_int_permute_select partition(part=1) VALUES (2, 
2222, 'new', 3333);
 
-explain
+explain vectorization detail
 select insert_num,part,a,b from part_add_int_permute_select;
 
 -- SELECT permutation columns to make sure NULL defaulting works right
@@ -62,7 +62,7 @@ alter table part_add_int_string_permute_select add columns(c 
int, d string);
 
 insert into table part_add_int_string_permute_select partition(part=1) VALUES 
(2, 2222, 'new', 3333, '4444');
 
-explain
+explain vectorization detail
 select insert_num,part,a,b from part_add_int_string_permute_select;
 
 -- SELECT permutation columns to make sure NULL defaulting works right
@@ -94,7 +94,7 @@ alter table part_change_string_group_double replace columns 
(insert_num int, c1
 
 insert into table part_change_string_group_double partition(part=1) SELECT 
insert_num, double1, double1, double1, 'new' FROM schema_evolution_data WHERE 
insert_num = 111;
 
-explain
+explain vectorization detail
 select insert_num,part,c1,c2,c3,b from part_change_string_group_double;
 
 select insert_num,part,c1,c2,c3,b from part_change_string_group_double;
@@ -117,7 +117,7 @@ alter table 
part_change_date_group_string_group_date_timestamp replace columns(i
 
 insert into table part_change_date_group_string_group_date_timestamp 
partition(part=1) VALUES (111, 'filler', 'filler', 'filler', 'filler', 
'filler', 'filler', 'filler', 'filler', 'filler', 'filler', 'new');
 
-explain
+explain vectorization detail
 select insert_num,part,c1,c2,c3,c4,c5,c6,c7,c8,c9,c10,b from 
part_change_date_group_string_group_date_timestamp;
 
 select insert_num,part,c1,c2,c3,c4,c5,c6,c7,c8,c9,c10,b from 
part_change_date_group_string_group_date_timestamp;
@@ -165,7 +165,7 @@ insert into table 
part_change_numeric_group_string_group_multi_ints_string_group
             'filler', 'filler', 'filler', 'filler', 'filler', 'filler', 
'filler', 'filler',
             'new');
 
-explain
+explain vectorization detail
 select 
insert_num,part,c1,c2,c3,c4,c5,c6,c7,c8,c9,c10,c11,c12,c13,c14,c15,c16,c17,c18,c19,c20,b
 from part_change_numeric_group_string_group_multi_ints_string_group;
 
 select 
insert_num,part,c1,c2,c3,c4,c5,c6,c7,c8,c9,c10,c11,c12,c13,c14,c15,c16,c17,c18,c19,c20,b
 from part_change_numeric_group_string_group_multi_ints_string_group;
@@ -208,7 +208,7 @@ insert into table 
part_change_numeric_group_string_group_floating_string_group p
              'filler', 'filler', 'filler', 'filler', 'filler', 'filler',
              'new');
 
-explain
+explain vectorization detail
 select insert_num,part,c1,c2,c3,c4,c5,c6,c7,c8,c9,c10,c11,c12,c13,c14,c15,b 
from part_change_numeric_group_string_group_floating_string_group;
 
 select insert_num,part,c1,c2,c3,c4,c5,c6,c7,c8,c9,c10,c11,c12,c13,c14,c15,b 
from part_change_numeric_group_string_group_floating_string_group;
@@ -250,7 +250,7 @@ insert into table 
part_change_string_group_string_group_string partition(part=1)
           'filler', 'filler', 'filler',
           'new');
 
-explain
+explain vectorization detail
 select insert_num,part,c1,c2,c3,c4,c5,c6,c7,c8,c9,c10,b from 
part_change_string_group_string_group_string;
 
 select insert_num,part,c1,c2,c3,c4,c5,c6,c7,c8,c9,c10,b from 
part_change_string_group_string_group_string;
@@ -300,7 +300,7 @@ insert into table 
part_change_lower_to_higher_numeric_group_tinyint_to_bigint pa
             1234.5678, 9876.543, 789.321,
            'new');
 
-explain
+explain vectorization detail
 select 
insert_num,part,c1,c2,c3,c4,c5,c6,c7,c8,c9,c10,c11,c12,c13,c14,c15,c16,c17,c18,b
 from part_change_lower_to_higher_numeric_group_tinyint_to_bigint;
 
 select 
insert_num,part,c1,c2,c3,c4,c5,c6,c7,c8,c9,c10,c11,c12,c13,c14,c15,c16,c17,c18,b
 from part_change_lower_to_higher_numeric_group_tinyint_to_bigint;
@@ -331,7 +331,7 @@ alter table 
part_change_lower_to_higher_numeric_group_decimal_to_float replace c
 
 insert into table part_change_lower_to_higher_numeric_group_decimal_to_float 
partition(part=1) VALUES (111, 1234.5678, 9876.543, 1234.5678, 'new');
 
-explain
+explain vectorization detail
 select insert_num,part,c1,c2,c3,b from 
part_change_lower_to_higher_numeric_group_decimal_to_float;
 
 select insert_num,part,c1,c2,c3,b from 
part_change_lower_to_higher_numeric_group_decimal_to_float;

http://git-wip-us.apache.org/repos/asf/hive/blob/f923db0b/ql/src/test/queries/clientpositive/schema_evol_text_vec_part_all_complex.q
----------------------------------------------------------------------
diff --git 
a/ql/src/test/queries/clientpositive/schema_evol_text_vec_part_all_complex.q 
b/ql/src/test/queries/clientpositive/schema_evol_text_vec_part_all_complex.q
index e38a01e..7eb72e0 100644
--- a/ql/src/test/queries/clientpositive/schema_evol_text_vec_part_all_complex.q
+++ b/ql/src/test/queries/clientpositive/schema_evol_text_vec_part_all_complex.q
@@ -1,4 +1,4 @@
-set hive.explain.user=true;
+set hive.explain.user=false;
 set hive.mapred.mode=nonstrict;
 set hive.cli.print.header=true;
 SET hive.exec.schema.evolution=true;
@@ -53,7 +53,7 @@ load data local inpath 
'../../data/files/schema_evolution/complex_struct1_c.txt'
 
 insert into table part_change_various_various_struct1 partition(part=1) select 
* from complex_struct1_c_txt;
 
-explain
+explain vectorization detail
 select insert_num,part,s1,b from part_change_various_various_struct1;
 
 select insert_num,part,s1,b from part_change_various_various_struct1;
@@ -113,7 +113,7 @@ load data local inpath 
'../../data/files/schema_evolution/complex_struct2_d.txt'
 
 insert into table part_add_various_various_struct2 partition(part=1) select * 
from complex_struct2_d_txt;
 
-explain
+explain vectorization detail
 select insert_num,part,b,s2 from part_add_various_various_struct2;
 
 select insert_num,part,b,s2 from part_add_various_various_struct2;
@@ -157,7 +157,7 @@ load data local inpath 
'../../data/files/schema_evolution/complex_struct4_c.txt'
 
 insert into table part_add_to_various_various_struct4 partition(part=1) select 
* from complex_struct4_c_txt;
 
-explain
+explain vectorization detail
 select insert_num,part,b,s3 from part_add_to_various_various_struct4;
 
 select insert_num,part,b,s3 from part_add_to_various_various_struct4;

http://git-wip-us.apache.org/repos/asf/hive/blob/f923db0b/ql/src/test/queries/clientpositive/schema_evol_text_vec_part_all_primitive.q
----------------------------------------------------------------------
diff --git 
a/ql/src/test/queries/clientpositive/schema_evol_text_vec_part_all_primitive.q 
b/ql/src/test/queries/clientpositive/schema_evol_text_vec_part_all_primitive.q
index c9d90c3..d5c01cd 100644
--- 
a/ql/src/test/queries/clientpositive/schema_evol_text_vec_part_all_primitive.q
+++ 
b/ql/src/test/queries/clientpositive/schema_evol_text_vec_part_all_primitive.q
@@ -1,4 +1,4 @@
-set hive.explain.user=true;
+set hive.explain.user=false;
 set hive.mapred.mode=nonstrict;
 set hive.cli.print.header=true;
 SET hive.exec.schema.evolution=true;
@@ -74,7 +74,7 @@ insert into table 
part_change_various_various_boolean_to_bigint partition(part=1
              bigint1, bigint1, bigint1, bigint1, bigint1, bigint1, bigint1, 
bigint1, bigint1, bigint1, bigint1, 
               'new' FROM schema_evolution_data;
 
-explain
+explain vectorization detail
 select 
insert_num,part,c1,c2,c3,c4,c5,c6,c7,c8,c9,c10,c11,c12,c13,c14,c15,c16,c17,c18,c19,c20,c21,c22,c23,c24,c25,c26,c27,c28,c29,c30,c31,c32,c33,c34,c35,c36,c37,c38,c39,c40,c41,c42,c43,c44,c45,c46,c47,c48,c49,c50,c51,c52,c53,b
 from part_change_various_various_boolean_to_bigint;
 
 select 
insert_num,part,c1,c2,c3,c4,c5,c6,c7,c8,c9,c10,c11,c12,c13,c14,c15,c16,c17,c18,c19,c20,c21,c22,c23,c24,c25,c26,c27,c28,c29,c30,c31,c32,c33,c34,c35,c36,c37,c38,c39,c40,c41,c42,c43,c44,c45,c46,c47,c48,c49,c50,c51,c52,c53,b
 from part_change_various_various_boolean_to_bigint;
@@ -116,7 +116,7 @@ insert into table 
part_change_various_various_decimal_to_double partition(part=1
              double1, double1, double1, double1, double1, double1, double1, 
double1, double1, double1, double1,
              'new' FROM schema_evolution_data_2 WHERE insert_num=111;
 
-explain
+explain vectorization detail
 select 
insert_num,part,c1,c2,c3,c4,c5,c6,c7,c8,c9,c10,c11,c12,c13,c14,c15,c16,c17,c18,c19,c20,c21,c22,c23,c24,c25,c26,c27,c28,c29,c30,c31,c32,c33,b
 from part_change_various_various_decimal_to_double;
 
 select 
insert_num,part,c1,c2,c3,c4,c5,c6,c7,c8,c9,c10,c11,c12,c13,c14,c15,c16,c17,c18,c19,c20,c21,c22,c23,c24,c25,c26,c27,c28,c29,c30,c31,c32,c33,b
 from part_change_various_various_decimal_to_double;
@@ -140,7 +140,7 @@ alter table part_change_various_various_timestamp replace 
columns (insert_num in
 
 insert into table part_change_various_various_timestamp partition(part=1) 
SELECT insert_num, timestamp1, timestamp1, timestamp1, timestamp1, timestamp1, 
timestamp1, timestamp1, timestamp1, timestamp1, timestamp1, timestamp1, 
timestamp1, 'new' FROM schema_evolution_data_2 WHERE insert_num=111;
 
-explain
+explain vectorization detail
 select insert_num,part,c1,c2,c3,c4,c5,c6,c7,c8,c9,c10,c11,c12,b from 
part_change_various_various_timestamp;
 
 select insert_num,part,c1,c2,c3,c4,c5,c6,c7,c8,c9,c10,c11,c12,b from 
part_change_various_various_timestamp;
@@ -161,7 +161,7 @@ alter table part_change_various_various_date replace 
columns (insert_num int, c1
 
 insert into table part_change_various_various_date partition(part=1) SELECT 
insert_num, date1, date1, date1, date1, 'new' FROM schema_evolution_data_2 
WHERE insert_num=111;
 
-explain
+explain vectorization detail
 select insert_num,part,c1,c2,c3,c4,b from part_change_various_various_date;
 
 select insert_num,part,c1,c2,c3,c4,b from part_change_various_various_date;
@@ -200,7 +200,7 @@ load data local inpath 
'../../data/files/schema_evolution/same_type1_c.txt' over
 
 insert into table part_change_same_type_different_params partition(part=2) 
select * from same_type1_c_txt;
 
-explain
+explain vectorization detail
 select insert_num,part,c1,c2,c3,c4,c5,c6,b from 
part_change_same_type_different_params;
 
 select insert_num,part,c1,c2,c3,c4,c5,c6,b from 
part_change_same_type_different_params;

http://git-wip-us.apache.org/repos/asf/hive/blob/f923db0b/ql/src/test/queries/clientpositive/schema_evol_text_vec_table.q
----------------------------------------------------------------------
diff --git a/ql/src/test/queries/clientpositive/schema_evol_text_vec_table.q 
b/ql/src/test/queries/clientpositive/schema_evol_text_vec_table.q
index 7785f87..bbf03af 100644
--- a/ql/src/test/queries/clientpositive/schema_evol_text_vec_table.q
+++ b/ql/src/test/queries/clientpositive/schema_evol_text_vec_table.q
@@ -1,4 +1,4 @@
-set hive.explain.user=true;
+set hive.explain.user=false;
 set hive.cli.print.header=true;
 SET hive.exec.schema.evolution=true;
 SET hive.vectorized.use.vectorized.input.format=false;
@@ -38,7 +38,7 @@ alter table table_add_int_permute_select add columns(c int);
 
 insert into table table_add_int_permute_select VALUES (111, 80000, 'new', 
80000);
 
-explain
+explain vectorization detail
 select insert_num,a,b from table_add_int_permute_select;
 
 -- SELECT permutation columns to make sure NULL defaulting works right
@@ -61,7 +61,7 @@ alter table table_add_int_string_permute_select add columns(c 
int, d string);
 
 insert into table table_add_int_string_permute_select VALUES (111, 80000, 
'new', 80000, 'filler');
 
-explain
+explain vectorization detail
 select insert_num,a,b from table_add_int_string_permute_select;
 
 -- SELECT permutation columns to make sure NULL defaulting works right
@@ -93,7 +93,7 @@ alter table table_change_string_group_double replace columns 
(insert_num int, c1
 
 insert into table table_change_string_group_double VALUES (111, 789.321, 
789.321, 789.321, 'new');
 
-explain
+explain vectorization detail
 select insert_num,c1,c2,c3,b from table_change_string_group_double;
 
 select insert_num,c1,c2,c3,b from table_change_string_group_double;
@@ -160,7 +160,7 @@ insert into table 
table_change_numeric_group_string_group_multi_ints_string_grou
             'filler', 'filler', 'filler', 'filler', 'filler', 'filler', 
'filler', 'filler',
             'new');
 
-explain
+explain vectorization detail
 select 
insert_num,c1,c2,c3,c4,c5,c6,c7,c8,c9,c10,c11,c12,c13,c14,c15,c16,c17,c18,c19,c20,b
 from table_change_numeric_group_string_group_multi_ints_string_group;
 
 select 
insert_num,c1,c2,c3,c4,c5,c6,c7,c8,c9,c10,c11,c12,c13,c14,c15,c16,c17,c18,c19,c20,b
 from table_change_numeric_group_string_group_multi_ints_string_group;
@@ -203,7 +203,7 @@ insert into table 
table_change_numeric_group_string_group_floating_string_group
              'filler', 'filler', 'filler', 'filler', 'filler', 'filler',
              'new');
 
-explain
+explain vectorization detail
 select insert_num,c1,c2,c3,c4,c5,c6,c7,c8,c9,c10,c11,c12,c13,c14,c15,b from 
table_change_numeric_group_string_group_floating_string_group;
 
 select insert_num,c1,c2,c3,c4,c5,c6,c7,c8,c9,c10,c11,c12,c13,c14,c15,b from 
table_change_numeric_group_string_group_floating_string_group;

Reply via email to