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

[9/9] hive git commit: HIVE-19200: Vectorization: Disable vectorization for LLAP I/O when a non-VECTORIZED_INPUT_FILE_FORMAT mode is needed (i.e. rows) and data type conversion is needed (Matt McCline, reviewed by Jason Dere)

HIVE-19200: Vectorization: Disable vectorization for LLAP I/O when a non-VECTORIZED_INPUT_FILE_FORMAT mode is needed (i.e. rows) and data type conversion is needed (Matt McCline, reviewed by Jason Dere)


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

Branch: refs/heads/master
Commit: c46c4c90fcea274d788185003cfc490212d8f21c
Parents: 6bd32a0
Author: Matt McCline <mm...@hortonworks.com>
Authored: Mon Apr 16 10:03:14 2018 -0500
Committer: Matt McCline <mm...@hortonworks.com>
Committed: Mon Apr 16 10:03:14 2018 -0500

----------------------------------------------------------------------
 .../hive/ql/optimizer/physical/Vectorizer.java  | 130 ++-
 ...vol_text_nonvec_part_all_primitive_llap_io.q | 114 +--
 ...ema_evol_text_vec_part_all_complex_llap_io.q | 118 +--
 ...a_evol_text_vec_part_all_primitive_llap_io.q | 114 +--
 .../schema_evol_text_vec_table_llap_io.q        | 144 +--
 ..._evol_text_vecrow_part_all_complex_llap_io.q | 122 +--
 ...vol_text_vecrow_part_all_primitive_llap_io.q | 114 +--
 .../schema_evol_text_vecrow_part_llap_io.q      | 154 ++--
 .../schema_evol_text_vecrow_table_llap_io.q     | 136 +--
 ...text_nonvec_part_all_primitive_llap_io.q.out | 623 +++++++++++++
 ...evol_text_vec_part_all_complex_llap_io.q.out | 526 +++++++++++
 ...ol_text_vec_part_all_primitive_llap_io.q.out | 648 +++++++++++++
 .../schema_evol_text_vec_table_llap_io.q.out    | 732 +++++++++++++--
 ...l_text_vecrow_part_all_complex_llap_io.q.out | 469 +++++++++-
 ...text_vecrow_part_all_primitive_llap_io.q.out | 578 +++++++++++-
 .../schema_evol_text_vecrow_part_llap_io.q.out  | 901 ++++++++++++++++++-
 .../schema_evol_text_vecrow_table_llap_io.q.out | 741 +++++++++++++++
 .../clientpositive/llap/vector_bucket.q.out     |   4 +-
 .../vector_reduce_groupby_duplicate_cols.q.out  |   4 +-
 .../vectorized_insert_into_bucketed_table.q.out |   4 +-
 .../results/clientpositive/vector_bucket.q.out  |   4 +-
 .../vector_tablesample_rows.q.out               |   8 +-
 22 files changed, 5664 insertions(+), 724 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hive/blob/c46c4c90/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 d3fbf07..e15c5b7 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
@@ -1507,6 +1507,18 @@ public class Vectorizer implements PhysicalPlanResolver {
       return false;
     }
 
+    private void setValidateInputFormatAndSchemaEvolutionExplain(MapWork mapWork,
+        Set<String> inputFileFormatClassNameSet,
+        Map<VectorPartitionDesc, VectorPartitionDesc> vectorPartitionDescMap,
+        Set<String> enabledConditionsMetSet, ArrayList<String> enabledConditionsNotMetList) {
+      mapWork.setVectorizationInputFileFormatClassNameSet(inputFileFormatClassNameSet);
+      ArrayList<VectorPartitionDesc> vectorPartitionDescList = new ArrayList<VectorPartitionDesc>();
+      vectorPartitionDescList.addAll(vectorPartitionDescMap.keySet());
+      mapWork.setVectorPartitionDescList(vectorPartitionDescList);
+      mapWork.setVectorizationEnabledConditionsMet(new ArrayList(enabledConditionsMetSet));
+      mapWork.setVectorizationEnabledConditionsNotMet(enabledConditionsNotMetList);
+    }
+
     private ImmutablePair<Boolean, Boolean> validateInputFormatAndSchemaEvolution(MapWork mapWork, String alias,
         TableScanOperator tableScanOperator, VectorTaskColumnInfo vectorTaskColumnInfo)
             throws SemanticException {
@@ -1569,24 +1581,22 @@ public class Vectorizer implements PhysicalPlanResolver {
           continue;
         }
         Set<Support> newSupportSet = new TreeSet<Support>();
-        if (!verifyAndSetVectorPartDesc(
-            partDesc, isFullAcidTable,
-            inputFileFormatClassNameSet,
-            vectorPartitionDescMap,
-            enabledConditionsMetSet, enabledConditionsNotMetList,
-            newSupportSet)) {
+        final boolean isVerifiedVectorPartDesc =
+            verifyAndSetVectorPartDesc(
+              partDesc, isFullAcidTable,
+              inputFileFormatClassNameSet,
+              vectorPartitionDescMap,
+              enabledConditionsMetSet, enabledConditionsNotMetList,
+              newSupportSet);
+
+        if (!isVerifiedVectorPartDesc) {
 
           // Always set these so EXPLAIN can see.
-          mapWork.setVectorizationInputFileFormatClassNameSet(inputFileFormatClassNameSet);
-          ArrayList<VectorPartitionDesc> vectorPartitionDescList = new ArrayList<VectorPartitionDesc>();
-          vectorPartitionDescList.addAll(vectorPartitionDescMap.keySet());
-          mapWork.setVectorPartitionDescList(vectorPartitionDescList);
-          mapWork.setVectorizationEnabledConditionsMet(new ArrayList(enabledConditionsMetSet));
-          mapWork.setVectorizationEnabledConditionsNotMet(enabledConditionsNotMetList);
+          setValidateInputFormatAndSchemaEvolutionExplain(
+              mapWork, inputFileFormatClassNameSet, vectorPartitionDescMap,
+              enabledConditionsMetSet, enabledConditionsNotMetList);
 
           // We consider this an enable issue, not a not vectorized issue.
-          LOG.info("Cannot enable vectorization because input file format(s) " + inputFileFormatClassNameSet +
-              " do not met conditions " + VectorizationCondition.addBooleans(enabledConditionsNotMetList, false));
           return new ImmutablePair<Boolean,Boolean>(false, true);
         }
 
@@ -1596,7 +1606,11 @@ public class Vectorizer implements PhysicalPlanResolver {
 
         if (isFirst) {
 
-          // Determine the data and partition columns using the first partition descriptor.
+          /*
+           * Determine the data and partition columns using the first partition descriptor's
+           * partition count.  In other words, how to split the schema columns -- the
+           * allColumnNameList and allTypeInfoList variables -- into the data and partition columns.
+           */
 
           LinkedHashMap<String, String> partSpec = partDesc.getPartSpec();
           if (partSpec != null && partSpec.size() > 0) {
@@ -1636,13 +1650,17 @@ public class Vectorizer implements PhysicalPlanResolver {
          * implicitly defaulted to null.
          */
         if (nextDataColumnList.size() > tableDataColumnList.size()) {
-          setOperatorIssue(
+          enabledConditionsNotMetList.add(
               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 new ImmutablePair<Boolean,Boolean>(false, false);
+                  "Could not enable vectorization due to " +
+                  "partition column names size %d is greater than the number of table column names size %d",
+                  nextDataColumnList.size(), tableDataColumnList.size()));
+
+          // Always set these so EXPLAIN can see.
+          setValidateInputFormatAndSchemaEvolutionExplain(
+              mapWork, inputFileFormatClassNameSet, vectorPartitionDescMap,
+              enabledConditionsMetSet, enabledConditionsNotMetList);
+          return new ImmutablePair<Boolean,Boolean>(false, true);
         }
         if (!(deserializer instanceof NullStructSerDe)) {
 
@@ -1651,33 +1669,79 @@ public class Vectorizer implements PhysicalPlanResolver {
             String nextColumnName = nextDataColumnList.get(i);
             String tableColumnName = tableDataColumnList.get(i);
             if (!nextColumnName.equals(tableColumnName)) {
-              setOperatorIssue(
+              enabledConditionsNotMetList.add(
                   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 new ImmutablePair<Boolean,Boolean>(false, false);
+                      "Could not enable vectorization due to " +
+                      "partition column name %s does not match table column name %s",
+                      nextColumnName, tableColumnName));
+
+              // Always set these so EXPLAIN can see.
+              setValidateInputFormatAndSchemaEvolutionExplain(
+                  mapWork, inputFileFormatClassNameSet, vectorPartitionDescMap,
+                  enabledConditionsMetSet, enabledConditionsNotMetList);
+              return new ImmutablePair<Boolean,Boolean>(false, true);
             }
           }
         }
 
+        boolean isPartitionRowConversion = false;
         List<TypeInfo> nextDataTypeInfoList;
         if (vectorPartDesc.getIsInputFileFormatSelfDescribing()) {
 
           /*
-           * Self-Describing Input Format will convert its data to the table schema.
+           * Self-Describing Input Format will convert its data to the table schema. So, there
+           * will be no VectorMapOperator conversion needed.
            */
           nextDataTypeInfoList = tableDataTypeInfoList;
 
         } else {
           String nextDataTypesString = ObjectInspectorUtils.getFieldTypes(partObjectInspector);
 
-          // 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.
+          /*
+           * 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);
+
+          final int nextDataTypeInfoSize = nextDataTypeInfoList.size();
+          if (nextDataTypeInfoSize > tableDataTypeInfoList.size()) {
+            enabledConditionsNotMetList.add(
+                String.format(
+                    "Could not enable vectorization due to " +
+                    "partition column types size %d is greater than the number of table column types size %d",
+                    nextDataTypeInfoSize, tableDataTypeInfoList.size()));
+
+            // Always set these so EXPLAIN can see.
+            setValidateInputFormatAndSchemaEvolutionExplain(
+                mapWork, inputFileFormatClassNameSet, vectorPartitionDescMap,
+                enabledConditionsMetSet, enabledConditionsNotMetList);
+            return new ImmutablePair<Boolean,Boolean>(false, true);
+          }
+          for (int i = 0; i < nextDataTypeInfoSize; i++) {
+            TypeInfo tableDataTypeInfo = tableDataTypeInfoList.get(i);
+            TypeInfo nextDataTypeInfo = nextDataTypeInfoList.get(i);
+
+            // FUTURE: We be more sophisticated in our conversion check.
+            if (!tableDataTypeInfo.equals(nextDataTypeInfo)) {
+              isPartitionRowConversion = true;
+              break;
+            }
+          }
+        }
+        if (isPartitionRowConversion && isLlapIoEnabled) {
+          enabledConditionsNotMetList.add(
+              "Could not enable vectorization. " +
+              "LLAP I/O is enabled wbich automatically deserializes into " +
+              "VECTORIZED_INPUT_FILE_FORMAT. " +
+              "A partition requires data type conversion and that is not supported");
+
+          // Always set these so EXPLAIN can see.
+          setValidateInputFormatAndSchemaEvolutionExplain(
+              mapWork, inputFileFormatClassNameSet, vectorPartitionDescMap,
+              enabledConditionsMetSet, enabledConditionsNotMetList);
+          return new ImmutablePair<Boolean,Boolean>(false, true);
         }
 
         vectorPartDesc.setDataTypeInfos(nextDataTypeInfoList);
@@ -2283,6 +2347,10 @@ public class Vectorizer implements PhysicalPlanResolver {
       }
     }
 
+    /*
+     * Notice the default value for LLAP_IO_ENABLED is overridden to be whether we are
+     * executing under LLAP.
+     */
     isLlapIoEnabled =
         HiveConf.getBoolVar(hiveConf,
             HiveConf.ConfVars.LLAP_IO_ENABLED,

http://git-wip-us.apache.org/repos/asf/hive/blob/c46c4c90/ql/src/test/queries/clientpositive/schema_evol_text_nonvec_part_all_primitive_llap_io.q
----------------------------------------------------------------------
diff --git a/ql/src/test/queries/clientpositive/schema_evol_text_nonvec_part_all_primitive_llap_io.q b/ql/src/test/queries/clientpositive/schema_evol_text_nonvec_part_all_primitive_llap_io.q
index 93494c0..6298c85 100644
--- a/ql/src/test/queries/clientpositive/schema_evol_text_nonvec_part_all_primitive_llap_io.q
+++ b/ql/src/test/queries/clientpositive/schema_evol_text_nonvec_part_all_primitive_llap_io.q
@@ -61,26 +61,26 @@ select insert_num,part,c1,c2,c3,c4,c5,c6,c7,c8,c9,c10,c11,c12,c13,c14,c15,c16,c1
 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;
 
 -- Table-Non-Cascade CHANGE COLUMNS ...
---** alter table part_change_various_various_boolean_to_bigint replace columns (insert_num int,
---**              c1 BOOLEAN, c2 BOOLEAN, c3 BOOLEAN, c4 BOOLEAN, c5 BOOLEAN, c6 BOOLEAN, c7 BOOLEAN, c8 BOOLEAN, c9 BOOLEAN,
---**              c10 TINYINT, c11 TINYINT, c12 TINYINT, c13 TINYINT, c14 TINYINT, c15 TINYINT, c16 TINYINT, c17 TINYINT, c18 TINYINT, c19 TINYINT, c20 TINYINT,
---**              c21 SMALLINT, c22 SMALLINT, c23 SMALLINT, c24 SMALLINT, c25 SMALLINT, c26 SMALLINT, c27 SMALLINT, c28 SMALLINT, c29 SMALLINT, c30 SMALLINT, c31 SMALLINT,
---**              c32 INT, c33 INT, c34 INT, c35 INT, c36 INT, c37 INT, c38 INT, c39 INT, c40 INT, c41 INT, c42 INT,
---**              c43 BIGINT, c44 BIGINT, c45 BIGINT, c46 BIGINT, c47 BIGINT, c48 BIGINT, c49 BIGINT, c50 BIGINT, c51 BIGINT, c52 BIGINT, c53 BIGINT,
---**              b STRING);
-
---** insert into table part_change_various_various_boolean_to_bigint partition(part=1) SELECT insert_num,
---**              boolean1, boolean1, boolean1, boolean1, boolean1, boolean1, boolean1, boolean1, boolean1,
---**              tinyint1, tinyint1, tinyint1, tinyint1, tinyint1, tinyint1, tinyint1, tinyint1, tinyint1, tinyint1, tinyint1,
---**              smallint1, smallint1, smallint1, smallint1, smallint1, smallint1, smallint1, smallint1, smallint1, smallint1, smallint1,
---**              int1, int1, int1, int1, int1, int1, int1, int1, int1, int1, int1,
---**              bigint1, bigint1, bigint1, bigint1, bigint1, bigint1, bigint1, bigint1, bigint1, bigint1, bigint1, 
---**               'new' FROM schema_evolution_data;
-
---** 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;
+alter table part_change_various_various_boolean_to_bigint replace columns (insert_num int,
+             c1 BOOLEAN, c2 BOOLEAN, c3 BOOLEAN, c4 BOOLEAN, c5 BOOLEAN, c6 BOOLEAN, c7 BOOLEAN, c8 BOOLEAN, c9 BOOLEAN,
+             c10 TINYINT, c11 TINYINT, c12 TINYINT, c13 TINYINT, c14 TINYINT, c15 TINYINT, c16 TINYINT, c17 TINYINT, c18 TINYINT, c19 TINYINT, c20 TINYINT,
+             c21 SMALLINT, c22 SMALLINT, c23 SMALLINT, c24 SMALLINT, c25 SMALLINT, c26 SMALLINT, c27 SMALLINT, c28 SMALLINT, c29 SMALLINT, c30 SMALLINT, c31 SMALLINT,
+             c32 INT, c33 INT, c34 INT, c35 INT, c36 INT, c37 INT, c38 INT, c39 INT, c40 INT, c41 INT, c42 INT,
+             c43 BIGINT, c44 BIGINT, c45 BIGINT, c46 BIGINT, c47 BIGINT, c48 BIGINT, c49 BIGINT, c50 BIGINT, c51 BIGINT, c52 BIGINT, c53 BIGINT,
+             b STRING);
+
+insert into table part_change_various_various_boolean_to_bigint partition(part=1) SELECT insert_num,
+             boolean1, boolean1, boolean1, boolean1, boolean1, boolean1, boolean1, boolean1, boolean1,
+             tinyint1, tinyint1, tinyint1, tinyint1, tinyint1, tinyint1, tinyint1, tinyint1, tinyint1, tinyint1, tinyint1,
+             smallint1, smallint1, smallint1, smallint1, smallint1, smallint1, smallint1, smallint1, smallint1, smallint1, smallint1,
+             int1, int1, int1, int1, int1, int1, int1, int1, int1, int1, int1,
+             bigint1, bigint1, bigint1, bigint1, bigint1, bigint1, bigint1, bigint1, bigint1, bigint1, bigint1, 
+              'new' FROM schema_evolution_data;
+
+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;
 
 drop table part_change_various_various_boolean_to_bigint;
 
@@ -110,22 +110,22 @@ select insert_num,part,c1,c2,c3,c4,c5,c6,c7,c8,c9,c10,c11,c12,c13,c14,c15,c16,c1
 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;
 
 -- Table-Non-Cascade CHANGE COLUMNS ...
---** alter table part_change_various_various_decimal_to_double replace columns (insert_num int,
---**              c1 DECIMAL(38,18), c2 DECIMAL(38,18), c3 DECIMAL(38,18), c4 DECIMAL(38,18), c5 DECIMAL(38,18), c6 DECIMAL(38,18), c7 DECIMAL(38,18), c8 DECIMAL(38,18), c9 DECIMAL(38,18), c10 DECIMAL(38,18), c11 DECIMAL(38,18), 
---**              c12 FLOAT, c13 FLOAT, c14 FLOAT, c15 FLOAT, c16 FLOAT, c17 FLOAT, c18 FLOAT, c19 FLOAT, c20 FLOAT, c21 FLOAT, c22 FLOAT,
---**              c23 DOUBLE, c24 DOUBLE, c25 DOUBLE, c26 DOUBLE, c27 DOUBLE, c28 DOUBLE, c29 DOUBLE, c30 DOUBLE, c31 DOUBLE, c32 DOUBLE, c33 DOUBLE,
---**              b STRING);
+alter table part_change_various_various_decimal_to_double replace columns (insert_num int,
+             c1 DECIMAL(38,18), c2 DECIMAL(38,18), c3 DECIMAL(38,18), c4 DECIMAL(38,18), c5 DECIMAL(38,18), c6 DECIMAL(38,18), c7 DECIMAL(38,18), c8 DECIMAL(38,18), c9 DECIMAL(38,18), c10 DECIMAL(38,18), c11 DECIMAL(38,18), 
+             c12 FLOAT, c13 FLOAT, c14 FLOAT, c15 FLOAT, c16 FLOAT, c17 FLOAT, c18 FLOAT, c19 FLOAT, c20 FLOAT, c21 FLOAT, c22 FLOAT,
+             c23 DOUBLE, c24 DOUBLE, c25 DOUBLE, c26 DOUBLE, c27 DOUBLE, c28 DOUBLE, c29 DOUBLE, c30 DOUBLE, c31 DOUBLE, c32 DOUBLE, c33 DOUBLE,
+             b STRING);
 
---** insert into table part_change_various_various_decimal_to_double partition(part=1) SELECT insert_num,
---**              decimal1, decimal1, decimal1, decimal1, decimal1, decimal1, decimal1, decimal1, decimal1, decimal1, decimal1,
---**              float1, float1, float1, float1, float1, float1, float1, float1, float1, float1, float1,
---**              double1, double1, double1, double1, double1, double1, double1, double1, double1, double1, double1,
---**              'new' FROM schema_evolution_data_2 WHERE insert_num=111;
+insert into table part_change_various_various_decimal_to_double partition(part=1) SELECT insert_num,
+             decimal1, decimal1, decimal1, decimal1, decimal1, decimal1, decimal1, decimal1, decimal1, decimal1, decimal1,
+             float1, float1, float1, float1, float1, float1, float1, float1, float1, float1, float1,
+             double1, double1, double1, double1, double1, double1, double1, double1, double1, double1, double1,
+             'new' FROM schema_evolution_data_2 WHERE insert_num=111;
 
---** 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;
+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;
+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;
 
 drop table part_change_various_various_decimal_to_double;
 
@@ -145,14 +145,14 @@ select insert_num,part,c1,c2,c3,c4,c5,c6,c7,c8,c9,c10,c11,c12,b from part_change
 select insert_num,part,c1,c2,c3,c4,c5,c6,c7,c8,c9,c10,c11,c12,b from part_change_various_various_timestamp;
 
 -- Table-Non-Cascade CHANGE COLUMNS ...
---** alter table part_change_various_various_timestamp replace columns (insert_num int, c1 TIMESTAMP, c2 TIMESTAMP, c3 TIMESTAMP, c4 TIMESTAMP, c5 TIMESTAMP, c6 TIMESTAMP, c7 TIMESTAMP, c8 TIMESTAMP, c9 TIMESTAMP, c10 TIMESTAMP, c11 TIMESTAMP, c12 TIMESTAMP, b STRING);
+alter table part_change_various_various_timestamp replace columns (insert_num int, c1 TIMESTAMP, c2 TIMESTAMP, c3 TIMESTAMP, c4 TIMESTAMP, c5 TIMESTAMP, c6 TIMESTAMP, c7 TIMESTAMP, c8 TIMESTAMP, c9 TIMESTAMP, c10 TIMESTAMP, c11 TIMESTAMP, c12 TIMESTAMP, b STRING);
 
---** 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;
+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 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;
+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;
+select insert_num,part,c1,c2,c3,c4,c5,c6,c7,c8,c9,c10,c11,c12,b from part_change_various_various_timestamp;
 
 drop table part_change_various_various_timestamp;
 
@@ -169,14 +169,14 @@ 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;
 
 -- Table-Non-Cascade CHANGE COLUMNS ...
---** alter table part_change_various_various_date replace columns (insert_num int, c1 DATE, c2 DATE, c3 DATE, c4 DATE, b STRING);
+alter table part_change_various_various_date replace columns (insert_num int, c1 DATE, c2 DATE, c3 DATE, c4 DATE, b STRING);
 
---** 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;
+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 vectorization detail
---** select insert_num,part,c1,c2,c3,c4,b from part_change_various_various_date;
+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;
+select insert_num,part,c1,c2,c3,c4,b from part_change_various_various_date;
 
 drop table part_change_various_various_date;
 
@@ -199,25 +199,25 @@ select insert_num,part,c1,c2,c3,c4,c5,c6,b from part_change_same_type_different_
 select insert_num,part,c1,c2,c3,c4,c5,c6,b from part_change_same_type_different_params;
 
 -- Table-Non-Cascade CHANGE COLUMNS ...
---** alter table part_change_same_type_different_params replace columns (insert_num int, c1 CHAR(8), c2 CHAR(32), c3 VARCHAR(15), c4 VARCHAR(18), c5 DECIMAL(10,2), c6 DECIMAL(25,15), b STRING);
+alter table part_change_same_type_different_params replace columns (insert_num int, c1 CHAR(8), c2 CHAR(32), c3 VARCHAR(15), c4 VARCHAR(18), c5 DECIMAL(10,2), c6 DECIMAL(25,15), b STRING);
 
---** CREATE TABLE same_type1_b_txt(insert_num int, c1 CHAR(8), c2 CHAR(32), c3 VARCHAR(15), c4 VARCHAR(18), c5 DECIMAL(10,2), c6 DECIMAL(25,15), b STRING)
---** row format delimited fields terminated by '|'
---** stored as textfile;
---** load data local inpath '../../data/files/schema_evolution/same_type1_b.txt' overwrite into table same_type1_b_txt;
+CREATE TABLE same_type1_b_txt(insert_num int, c1 CHAR(8), c2 CHAR(32), c3 VARCHAR(15), c4 VARCHAR(18), c5 DECIMAL(10,2), c6 DECIMAL(25,15), b STRING)
+row format delimited fields terminated by '|'
+stored as textfile;
+load data local inpath '../../data/files/schema_evolution/same_type1_b.txt' overwrite into table same_type1_b_txt;
 
---** insert into table part_change_same_type_different_params partition(part=1) select * from same_type1_b_txt;
+insert into table part_change_same_type_different_params partition(part=1) select * from same_type1_b_txt;
 
---** CREATE TABLE same_type1_c_txt(insert_num int, c1 CHAR(8), c2 CHAR(32), c3 VARCHAR(15), c4 VARCHAR(18), c5 DECIMAL(10,2), c6 DECIMAL(25,15), b STRING)
---** row format delimited fields terminated by '|'
---** stored as textfile;
---** load data local inpath '../../data/files/schema_evolution/same_type1_c.txt' overwrite into table same_type1_c_txt;
+CREATE TABLE same_type1_c_txt(insert_num int, c1 CHAR(8), c2 CHAR(32), c3 VARCHAR(15), c4 VARCHAR(18), c5 DECIMAL(10,2), c6 DECIMAL(25,15), b STRING)
+row format delimited fields terminated by '|'
+stored as textfile;
+load data local inpath '../../data/files/schema_evolution/same_type1_c.txt' overwrite into table same_type1_c_txt;
 
---** insert into table part_change_same_type_different_params partition(part=2) select * from same_type1_c_txt;
+insert into table part_change_same_type_different_params partition(part=2) select * from same_type1_c_txt;
 
---** explain vectorization detail
---** select insert_num,part,c1,c2,c3,c4,c5,c6,b from part_change_same_type_different_params;
+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;
+select insert_num,part,c1,c2,c3,c4,c5,c6,b from part_change_same_type_different_params;
 
 drop table part_change_same_type_different_params;

http://git-wip-us.apache.org/repos/asf/hive/blob/c46c4c90/ql/src/test/queries/clientpositive/schema_evol_text_vec_part_all_complex_llap_io.q
----------------------------------------------------------------------
diff --git a/ql/src/test/queries/clientpositive/schema_evol_text_vec_part_all_complex_llap_io.q b/ql/src/test/queries/clientpositive/schema_evol_text_vec_part_all_complex_llap_io.q
index 8e0c780..6425831 100644
--- a/ql/src/test/queries/clientpositive/schema_evol_text_vec_part_all_complex_llap_io.q
+++ b/ql/src/test/queries/clientpositive/schema_evol_text_vec_part_all_complex_llap_io.q
@@ -39,26 +39,26 @@ select insert_num,part,s1,b from part_change_various_various_struct1;
 select insert_num,part,s1,b from part_change_various_various_struct1;
 
 -- Table-Non-Cascade CHANGE COLUMNS ...
---** alter table part_change_various_various_struct1 replace columns (insert_num int, s1 STRUCT<c1:STRING, c2:STRING, c3:STRING, c4:STRING, c5:STRING, c6:STRING, c7:STRING, c8:STRING, c9:STRING, c10:STRING, c11:STRING, c12:STRING, c13:STRING>, b STRING);
+alter table part_change_various_various_struct1 replace columns (insert_num int, s1 STRUCT<c1:STRING, c2:STRING, c3:STRING, c4:STRING, c5:STRING, c6:STRING, c7:STRING, c8:STRING, c9:STRING, c10:STRING, c11:STRING, c12:STRING, c13:STRING>, b STRING);
 
---** CREATE TABLE complex_struct1_b_txt(insert_num int, s1 STRUCT<c1:STRING, c2:STRING, c3:STRING, c4:STRING, c5:STRING, c6:STRING, c7:STRING, c8:STRING, c9:STRING, c10:STRING, c11:STRING, c12:STRING, c13:STRING>, b STRING)
---** row format delimited fields terminated by '|'
---** collection items terminated by ','
---** map keys terminated by ':' stored as textfile;
---** load data local inpath '../../data/files/schema_evolution/complex_struct1_b.txt' overwrite into table complex_struct1_b_txt;
+CREATE TABLE complex_struct1_b_txt(insert_num int, s1 STRUCT<c1:STRING, c2:STRING, c3:STRING, c4:STRING, c5:STRING, c6:STRING, c7:STRING, c8:STRING, c9:STRING, c10:STRING, c11:STRING, c12:STRING, c13:STRING>, b STRING)
+row format delimited fields terminated by '|'
+collection items terminated by ','
+map keys terminated by ':' stored as textfile;
+load data local inpath '../../data/files/schema_evolution/complex_struct1_b.txt' overwrite into table complex_struct1_b_txt;
 
---** insert into table part_change_various_various_struct1 partition(part=2) select * from complex_struct1_b_txt;
+insert into table part_change_various_various_struct1 partition(part=2) select * from complex_struct1_b_txt;
 
---** CREATE TABLE complex_struct1_c_txt(insert_num int, s1 STRUCT<c1:STRING, c2:STRING, c3:STRING, c4:STRING, c5:STRING, c6:STRING, c7:STRING, c8:STRING, c9:STRING, c10:STRING, c11:STRING, c12:STRING, c13:STRING>, b STRING)
---** row format delimited fields terminated by '|'
---** collection items terminated by ','
---** map keys terminated by ':' stored as textfile;
---** load data local inpath '../../data/files/schema_evolution/complex_struct1_c.txt' overwrite into table complex_struct1_c_txt;
+CREATE TABLE complex_struct1_c_txt(insert_num int, s1 STRUCT<c1:STRING, c2:STRING, c3:STRING, c4:STRING, c5:STRING, c6:STRING, c7:STRING, c8:STRING, c9:STRING, c10:STRING, c11:STRING, c12:STRING, c13:STRING>, b STRING)
+row format delimited fields terminated by '|'
+collection items terminated by ','
+map keys terminated by ':' stored as textfile;
+load data local inpath '../../data/files/schema_evolution/complex_struct1_c.txt' overwrite into table complex_struct1_c_txt;
 
---** insert into table part_change_various_various_struct1 partition(part=1) select * from complex_struct1_c_txt;
+insert into table part_change_various_various_struct1 partition(part=1) select * from complex_struct1_c_txt;
 
---** explain vectorization detail
---** select insert_num,part,s1,b from part_change_various_various_struct1;
+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;
 
@@ -81,47 +81,49 @@ select insert_num,part,b from part_add_various_various_struct2;
 select insert_num,part,b from part_add_various_various_struct2;
 
 -- Table-Non-Cascade ADD COLUMN ...
---** alter table part_add_various_various_struct2 ADD columns (s2 STRUCT<c1:BOOLEAN, c2:TINYINT, c3:SMALLINT, c4:INT, c5:BIGINT, c6:FLOAT, c7:DOUBLE, c8:DECIMAL(38,18), c9:CHAR(25), c10:VARCHAR(25), c11:TIMESTAMP, c12:DATE, c13:BINARY>);
+alter table part_add_various_various_struct2 ADD columns (s2 STRUCT<c1:BOOLEAN, c2:TINYINT, c3:SMALLINT, c4:INT, c5:BIGINT, c6:FLOAT, c7:DOUBLE, c8:DECIMAL(38,18), c9:CHAR(25), c10:VARCHAR(25), c11:TIMESTAMP, c12:DATE, c13:BINARY>);
 
---** CREATE TABLE complex_struct2_a_txt(insert_num int, b STRING, s2 STRUCT<c1:BOOLEAN, c2:TINYINT, c3:SMALLINT, c4:INT, c5:BIGINT, c6:FLOAT, c7:DOUBLE, c8:DECIMAL(38,18), c9:CHAR(25), c10:VARCHAR(25), c11:TIMESTAMP, c12:DATE, c13:BINARY>)
---** row format delimited fields terminated by '|'
---** collection items terminated by ','
---** map keys terminated by ':' stored as textfile;
---** load data local inpath '../../data/files/schema_evolution/complex_struct2_a.txt' overwrite into table complex_struct2_a_txt;
+CREATE TABLE complex_struct2_a_txt(insert_num int, b STRING, s2 STRUCT<c1:BOOLEAN, c2:TINYINT, c3:SMALLINT, c4:INT, c5:BIGINT, c6:FLOAT, c7:DOUBLE, c8:DECIMAL(38,18), c9:CHAR(25), c10:VARCHAR(25), c11:TIMESTAMP, c12:DATE, c13:BINARY>)
+row format delimited fields terminated by '|'
+collection items terminated by ','
+map keys terminated by ':' stored as textfile;
+load data local inpath '../../data/files/schema_evolution/complex_struct2_a.txt' overwrite into table complex_struct2_a_txt;
 
---** insert into table part_add_various_various_struct2 partition(part=1) select * from complex_struct2_a_txt;
+insert into table part_add_various_various_struct2 partition(part=1) select * from complex_struct2_a_txt;
 
---** CREATE TABLE complex_struct2_b_txt(insert_num int, b STRING, s2 STRUCT<c1:BOOLEAN, c2:TINYINT, c3:SMALLINT, c4:INT, c5:BIGINT, c6:FLOAT, c7:DOUBLE, c8:DECIMAL(38,18), c9:CHAR(25), c10:VARCHAR(25), c11:TIMESTAMP, c12:DATE, c13:BINARY>)
---** row format delimited fields terminated by '|'
---** collection items terminated by ','
---** map keys terminated by ':' stored as textfile;
---** load data local inpath '../../data/files/schema_evolution/complex_struct2_b.txt' overwrite into table complex_struct2_b_txt;
+CREATE TABLE complex_struct2_b_txt(insert_num int, b STRING, s2 STRUCT<c1:BOOLEAN, c2:TINYINT, c3:SMALLINT, c4:INT, c5:BIGINT, c6:FLOAT, c7:DOUBLE, c8:DECIMAL(38,18), c9:CHAR(25), c10:VARCHAR(25), c11:TIMESTAMP, c12:DATE, c13:BINARY>)
+row format delimited fields terminated by '|'
+collection items terminated by ','
+map keys terminated by ':' stored as textfile;
+load data local inpath '../../data/files/schema_evolution/complex_struct2_b.txt' overwrite into table complex_struct2_b_txt;
 
---** insert into table part_add_various_various_struct2 partition(part=2) select * from complex_struct2_b_txt;
+insert into table part_add_various_various_struct2 partition(part=2) select * from complex_struct2_b_txt;
 
+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;
 
 -- Table-Non-Cascade CHANGE COLUMNS ...
---** alter table part_add_various_various_struct2 REPLACE columns (insert_num int, b STRING, s2 STRUCT<c1:STRING, c2:STRING, c3:STRING, c4:STRING, c5:STRING, c6:STRING, c7:STRING, c8:STRING, c9:STRING, c10:STRING, c11:STRING, c12:STRING, c13:STRING>);
+alter table part_add_various_various_struct2 REPLACE columns (insert_num int, b STRING, s2 STRUCT<c1:STRING, c2:STRING, c3:STRING, c4:STRING, c5:STRING, c6:STRING, c7:STRING, c8:STRING, c9:STRING, c10:STRING, c11:STRING, c12:STRING, c13:STRING>);
 
---** CREATE TABLE complex_struct2_c_txt(insert_num int, b STRING, s2 STRUCT<c1:STRING, c2:STRING, c3:STRING, c4:STRING, c5:STRING, c6:STRING, c7:STRING, c8:STRING, c9:STRING, c10:STRING, c11:STRING, c12:STRING, c13:STRING>)
---** row format delimited fields terminated by '|'
---** collection items terminated by ','
---** map keys terminated by ':' stored as textfile;
---** load data local inpath '../../data/files/schema_evolution/complex_struct2_c.txt' overwrite into table complex_struct2_c_txt;
+CREATE TABLE complex_struct2_c_txt(insert_num int, b STRING, s2 STRUCT<c1:STRING, c2:STRING, c3:STRING, c4:STRING, c5:STRING, c6:STRING, c7:STRING, c8:STRING, c9:STRING, c10:STRING, c11:STRING, c12:STRING, c13:STRING>)
+row format delimited fields terminated by '|'
+collection items terminated by ','
+map keys terminated by ':' stored as textfile;
+load data local inpath '../../data/files/schema_evolution/complex_struct2_c.txt' overwrite into table complex_struct2_c_txt;
 
---** insert into table part_add_various_various_struct2 partition(part=2) select * from complex_struct2_c_txt;
+insert into table part_add_various_various_struct2 partition(part=2) select * from complex_struct2_c_txt;
 
---** CREATE TABLE complex_struct2_d_txt(insert_num int, b STRING, s2 STRUCT<c1:STRING, c2:STRING, c3:STRING, c4:STRING, c5:STRING, c6:STRING, c7:STRING, c8:STRING, c9:STRING, c10:STRING, c11:STRING, c12:STRING, c13:STRING>)
---** row format delimited fields terminated by '|'
---** collection items terminated by ','
---** map keys terminated by ':' stored as textfile;
---** load data local inpath '../../data/files/schema_evolution/complex_struct2_d.txt' overwrite into table complex_struct2_d_txt;
+CREATE TABLE complex_struct2_d_txt(insert_num int, b STRING, s2 STRUCT<c1:STRING, c2:STRING, c3:STRING, c4:STRING, c5:STRING, c6:STRING, c7:STRING, c8:STRING, c9:STRING, c10:STRING, c11:STRING, c12:STRING, c13:STRING>)
+row format delimited fields terminated by '|'
+collection items terminated by ','
+map keys terminated by ':' stored as textfile;
+load data local inpath '../../data/files/schema_evolution/complex_struct2_d.txt' overwrite into table complex_struct2_d_txt;
 
---** insert into table part_add_various_various_struct2 partition(part=1) select * from complex_struct2_d_txt;
+insert into table part_add_various_various_struct2 partition(part=1) select * from complex_struct2_d_txt;
 
---** explain vectorization detail
---** select insert_num,part,b,s2 from part_add_various_various_struct2;
+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;
 
@@ -149,26 +151,26 @@ 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;
 
 -- Table-Non-Cascade CHANGE COLUMNS ...
---** alter table part_add_to_various_various_struct4 replace columns (insert_num int, b STRING, s3 STRUCT<c1:BOOLEAN, c2:TINYINT, c3:SMALLINT, c4:INT, c5:BIGINT, c6:FLOAT, c7:DOUBLE, c8:DECIMAL(38,18), c9:CHAR(25), c10:VARCHAR(25), c11:TIMESTAMP, c12:DATE, c13:BINARY>);
+alter table part_add_to_various_various_struct4 replace columns (insert_num int, b STRING, s3 STRUCT<c1:BOOLEAN, c2:TINYINT, c3:SMALLINT, c4:INT, c5:BIGINT, c6:FLOAT, c7:DOUBLE, c8:DECIMAL(38,18), c9:CHAR(25), c10:VARCHAR(25), c11:TIMESTAMP, c12:DATE, c13:BINARY>);
 
---** CREATE TABLE complex_struct4_b_txt(insert_num int, b STRING, s3 STRUCT<c1:BOOLEAN, c2:TINYINT, c3:SMALLINT, c4:INT, c5:BIGINT, c6:FLOAT, c7:DOUBLE, c8:DECIMAL(38,18), c9:CHAR(25), c10:VARCHAR(25), c11:TIMESTAMP, c12:DATE, c13:BINARY>)
---** row format delimited fields terminated by '|'
---** collection items terminated by ','
---** map keys terminated by ':' stored as textfile;
---** load data local inpath '../../data/files/schema_evolution/complex_struct4_b.txt' overwrite into table complex_struct4_b_txt;
+CREATE TABLE complex_struct4_b_txt(insert_num int, b STRING, s3 STRUCT<c1:BOOLEAN, c2:TINYINT, c3:SMALLINT, c4:INT, c5:BIGINT, c6:FLOAT, c7:DOUBLE, c8:DECIMAL(38,18), c9:CHAR(25), c10:VARCHAR(25), c11:TIMESTAMP, c12:DATE, c13:BINARY>)
+row format delimited fields terminated by '|'
+collection items terminated by ','
+map keys terminated by ':' stored as textfile;
+load data local inpath '../../data/files/schema_evolution/complex_struct4_b.txt' overwrite into table complex_struct4_b_txt;
 
---** insert into table part_add_to_various_various_struct4 partition(part=2) select * from complex_struct4_b_txt;
+insert into table part_add_to_various_various_struct4 partition(part=2) select * from complex_struct4_b_txt;
 
---** CREATE TABLE complex_struct4_c_txt(insert_num int, b STRING, s3 STRUCT<c1:BOOLEAN, c2:TINYINT, c3:SMALLINT, c4:INT, c5:BIGINT, c6:FLOAT, c7:DOUBLE, c8:DECIMAL(38,18), c9:CHAR(25), c10:VARCHAR(25), c11:TIMESTAMP, c12:DATE, c13:BINARY>)
---** row format delimited fields terminated by '|'
---** collection items terminated by ','
---** map keys terminated by ':' stored as textfile;
---** load data local inpath '../../data/files/schema_evolution/complex_struct4_c.txt' overwrite into table complex_struct4_c_txt;
+CREATE TABLE complex_struct4_c_txt(insert_num int, b STRING, s3 STRUCT<c1:BOOLEAN, c2:TINYINT, c3:SMALLINT, c4:INT, c5:BIGINT, c6:FLOAT, c7:DOUBLE, c8:DECIMAL(38,18), c9:CHAR(25), c10:VARCHAR(25), c11:TIMESTAMP, c12:DATE, c13:BINARY>)
+row format delimited fields terminated by '|'
+collection items terminated by ','
+map keys terminated by ':' stored as textfile;
+load data local inpath '../../data/files/schema_evolution/complex_struct4_c.txt' overwrite into table complex_struct4_c_txt;
 
---** insert into table part_add_to_various_various_struct4 partition(part=1) select * from complex_struct4_c_txt;
+insert into table part_add_to_various_various_struct4 partition(part=1) select * from complex_struct4_c_txt;
 
---** explain vectorization detail
---** select insert_num,part,b,s3 from part_add_to_various_various_struct4;
+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/c46c4c90/ql/src/test/queries/clientpositive/schema_evol_text_vec_part_all_primitive_llap_io.q
----------------------------------------------------------------------
diff --git a/ql/src/test/queries/clientpositive/schema_evol_text_vec_part_all_primitive_llap_io.q b/ql/src/test/queries/clientpositive/schema_evol_text_vec_part_all_primitive_llap_io.q
index b64ed2d..63a4df7 100644
--- a/ql/src/test/queries/clientpositive/schema_evol_text_vec_part_all_primitive_llap_io.q
+++ b/ql/src/test/queries/clientpositive/schema_evol_text_vec_part_all_primitive_llap_io.q
@@ -62,26 +62,26 @@ select insert_num,part,c1,c2,c3,c4,c5,c6,c7,c8,c9,c10,c11,c12,c13,c14,c15,c16,c1
 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;
 
 -- Table-Non-Cascade CHANGE COLUMNS ...
---** alter table part_change_various_various_boolean_to_bigint replace columns (insert_num int,
---**              c1 BOOLEAN, c2 BOOLEAN, c3 BOOLEAN, c4 BOOLEAN, c5 BOOLEAN, c6 BOOLEAN, c7 BOOLEAN, c8 BOOLEAN, c9 BOOLEAN,
---**              c10 TINYINT, c11 TINYINT, c12 TINYINT, c13 TINYINT, c14 TINYINT, c15 TINYINT, c16 TINYINT, c17 TINYINT, c18 TINYINT, c19 TINYINT, c20 TINYINT,
---**              c21 SMALLINT, c22 SMALLINT, c23 SMALLINT, c24 SMALLINT, c25 SMALLINT, c26 SMALLINT, c27 SMALLINT, c28 SMALLINT, c29 SMALLINT, c30 SMALLINT, c31 SMALLINT,
---**              c32 INT, c33 INT, c34 INT, c35 INT, c36 INT, c37 INT, c38 INT, c39 INT, c40 INT, c41 INT, c42 INT,
---**              c43 BIGINT, c44 BIGINT, c45 BIGINT, c46 BIGINT, c47 BIGINT, c48 BIGINT, c49 BIGINT, c50 BIGINT, c51 BIGINT, c52 BIGINT, c53 BIGINT,
---**              b STRING);
-
---** insert into table part_change_various_various_boolean_to_bigint partition(part=1) SELECT insert_num,
---**              boolean1, boolean1, boolean1, boolean1, boolean1, boolean1, boolean1, boolean1, boolean1,
---**              tinyint1, tinyint1, tinyint1, tinyint1, tinyint1, tinyint1, tinyint1, tinyint1, tinyint1, tinyint1, tinyint1,
---**              smallint1, smallint1, smallint1, smallint1, smallint1, smallint1, smallint1, smallint1, smallint1, smallint1, smallint1,
---**              int1, int1, int1, int1, int1, int1, int1, int1, int1, int1, int1,
---**              bigint1, bigint1, bigint1, bigint1, bigint1, bigint1, bigint1, bigint1, bigint1, bigint1, bigint1, 
---**               'new' FROM schema_evolution_data;
-
---** 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;
+alter table part_change_various_various_boolean_to_bigint replace columns (insert_num int,
+             c1 BOOLEAN, c2 BOOLEAN, c3 BOOLEAN, c4 BOOLEAN, c5 BOOLEAN, c6 BOOLEAN, c7 BOOLEAN, c8 BOOLEAN, c9 BOOLEAN,
+             c10 TINYINT, c11 TINYINT, c12 TINYINT, c13 TINYINT, c14 TINYINT, c15 TINYINT, c16 TINYINT, c17 TINYINT, c18 TINYINT, c19 TINYINT, c20 TINYINT,
+             c21 SMALLINT, c22 SMALLINT, c23 SMALLINT, c24 SMALLINT, c25 SMALLINT, c26 SMALLINT, c27 SMALLINT, c28 SMALLINT, c29 SMALLINT, c30 SMALLINT, c31 SMALLINT,
+             c32 INT, c33 INT, c34 INT, c35 INT, c36 INT, c37 INT, c38 INT, c39 INT, c40 INT, c41 INT, c42 INT,
+             c43 BIGINT, c44 BIGINT, c45 BIGINT, c46 BIGINT, c47 BIGINT, c48 BIGINT, c49 BIGINT, c50 BIGINT, c51 BIGINT, c52 BIGINT, c53 BIGINT,
+             b STRING);
+
+insert into table part_change_various_various_boolean_to_bigint partition(part=1) SELECT insert_num,
+             boolean1, boolean1, boolean1, boolean1, boolean1, boolean1, boolean1, boolean1, boolean1,
+             tinyint1, tinyint1, tinyint1, tinyint1, tinyint1, tinyint1, tinyint1, tinyint1, tinyint1, tinyint1, tinyint1,
+             smallint1, smallint1, smallint1, smallint1, smallint1, smallint1, smallint1, smallint1, smallint1, smallint1, smallint1,
+             int1, int1, int1, int1, int1, int1, int1, int1, int1, int1, int1,
+             bigint1, bigint1, bigint1, bigint1, bigint1, bigint1, bigint1, bigint1, bigint1, bigint1, bigint1, 
+              'new' FROM schema_evolution_data;
+
+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;
 
 drop table part_change_various_various_boolean_to_bigint;
 
@@ -111,22 +111,22 @@ select insert_num,part,c1,c2,c3,c4,c5,c6,c7,c8,c9,c10,c11,c12,c13,c14,c15,c16,c1
 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;
 
 -- Table-Non-Cascade CHANGE COLUMNS ...
---** alter table part_change_various_various_decimal_to_double replace columns (insert_num int,
---**              c1 DECIMAL(38,18), c2 DECIMAL(38,18), c3 DECIMAL(38,18), c4 DECIMAL(38,18), c5 DECIMAL(38,18), c6 DECIMAL(38,18), c7 DECIMAL(38,18), c8 DECIMAL(38,18), c9 DECIMAL(38,18), c10 DECIMAL(38,18), c11 DECIMAL(38,18), 
---**              c12 FLOAT, c13 FLOAT, c14 FLOAT, c15 FLOAT, c16 FLOAT, c17 FLOAT, c18 FLOAT, c19 FLOAT, c20 FLOAT, c21 FLOAT, c22 FLOAT,
---**              c23 DOUBLE, c24 DOUBLE, c25 DOUBLE, c26 DOUBLE, c27 DOUBLE, c28 DOUBLE, c29 DOUBLE, c30 DOUBLE, c31 DOUBLE, c32 DOUBLE, c33 DOUBLE,
---**              b STRING);
+alter table part_change_various_various_decimal_to_double replace columns (insert_num int,
+             c1 DECIMAL(38,18), c2 DECIMAL(38,18), c3 DECIMAL(38,18), c4 DECIMAL(38,18), c5 DECIMAL(38,18), c6 DECIMAL(38,18), c7 DECIMAL(38,18), c8 DECIMAL(38,18), c9 DECIMAL(38,18), c10 DECIMAL(38,18), c11 DECIMAL(38,18), 
+             c12 FLOAT, c13 FLOAT, c14 FLOAT, c15 FLOAT, c16 FLOAT, c17 FLOAT, c18 FLOAT, c19 FLOAT, c20 FLOAT, c21 FLOAT, c22 FLOAT,
+             c23 DOUBLE, c24 DOUBLE, c25 DOUBLE, c26 DOUBLE, c27 DOUBLE, c28 DOUBLE, c29 DOUBLE, c30 DOUBLE, c31 DOUBLE, c32 DOUBLE, c33 DOUBLE,
+             b STRING);
 
---** insert into table part_change_various_various_decimal_to_double partition(part=1) SELECT insert_num,
---**              decimal1, decimal1, decimal1, decimal1, decimal1, decimal1, decimal1, decimal1, decimal1, decimal1, decimal1,
---**              float1, float1, float1, float1, float1, float1, float1, float1, float1, float1, float1,
---**              double1, double1, double1, double1, double1, double1, double1, double1, double1, double1, double1,
---**              'new' FROM schema_evolution_data_2 WHERE insert_num=111;
+insert into table part_change_various_various_decimal_to_double partition(part=1) SELECT insert_num,
+             decimal1, decimal1, decimal1, decimal1, decimal1, decimal1, decimal1, decimal1, decimal1, decimal1, decimal1,
+             float1, float1, float1, float1, float1, float1, float1, float1, float1, float1, float1,
+             double1, double1, double1, double1, double1, double1, double1, double1, double1, double1, double1,
+             'new' FROM schema_evolution_data_2 WHERE insert_num=111;
 
---** 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;
+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;
+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;
 
 drop table part_change_various_various_decimal_to_double;
 
@@ -146,14 +146,14 @@ select insert_num,part,c1,c2,c3,c4,c5,c6,c7,c8,c9,c10,c11,c12,b from part_change
 select insert_num,part,c1,c2,c3,c4,c5,c6,c7,c8,c9,c10,c11,c12,b from part_change_various_various_timestamp;
 
 -- Table-Non-Cascade CHANGE COLUMNS ...
---** alter table part_change_various_various_timestamp replace columns (insert_num int, c1 TIMESTAMP, c2 TIMESTAMP, c3 TIMESTAMP, c4 TIMESTAMP, c5 TIMESTAMP, c6 TIMESTAMP, c7 TIMESTAMP, c8 TIMESTAMP, c9 TIMESTAMP, c10 TIMESTAMP, c11 TIMESTAMP, c12 TIMESTAMP, b STRING);
+alter table part_change_various_various_timestamp replace columns (insert_num int, c1 TIMESTAMP, c2 TIMESTAMP, c3 TIMESTAMP, c4 TIMESTAMP, c5 TIMESTAMP, c6 TIMESTAMP, c7 TIMESTAMP, c8 TIMESTAMP, c9 TIMESTAMP, c10 TIMESTAMP, c11 TIMESTAMP, c12 TIMESTAMP, b STRING);
 
---** 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;
+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 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;
+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;
+select insert_num,part,c1,c2,c3,c4,c5,c6,c7,c8,c9,c10,c11,c12,b from part_change_various_various_timestamp;
 
 drop table part_change_various_various_timestamp;
 
@@ -170,14 +170,14 @@ 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;
 
 -- Table-Non-Cascade CHANGE COLUMNS ...
---** alter table part_change_various_various_date replace columns (insert_num int, c1 DATE, c2 DATE, c3 DATE, c4 DATE, b STRING);
+alter table part_change_various_various_date replace columns (insert_num int, c1 DATE, c2 DATE, c3 DATE, c4 DATE, b STRING);
 
---** 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;
+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 vectorization detail
---** select insert_num,part,c1,c2,c3,c4,b from part_change_various_various_date;
+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;
+select insert_num,part,c1,c2,c3,c4,b from part_change_various_various_date;
 
 drop table part_change_various_various_date;
 
@@ -200,25 +200,25 @@ select insert_num,part,c1,c2,c3,c4,c5,c6,b from part_change_same_type_different_
 select insert_num,part,c1,c2,c3,c4,c5,c6,b from part_change_same_type_different_params;
 
 -- Table-Non-Cascade CHANGE COLUMNS ...
---** alter table part_change_same_type_different_params replace columns (insert_num int, c1 CHAR(8), c2 CHAR(32), c3 VARCHAR(15), c4 VARCHAR(18), c5 DECIMAL(10,2), c6 DECIMAL(25,15), b STRING);
+alter table part_change_same_type_different_params replace columns (insert_num int, c1 CHAR(8), c2 CHAR(32), c3 VARCHAR(15), c4 VARCHAR(18), c5 DECIMAL(10,2), c6 DECIMAL(25,15), b STRING);
 
---** CREATE TABLE same_type1_b_txt(insert_num int, c1 CHAR(8), c2 CHAR(32), c3 VARCHAR(15), c4 VARCHAR(18), c5 DECIMAL(10,2), c6 DECIMAL(25,15), b STRING)
---** row format delimited fields terminated by '|'
---** stored as textfile;
---** load data local inpath '../../data/files/schema_evolution/same_type1_b.txt' overwrite into table same_type1_b_txt;
+CREATE TABLE same_type1_b_txt(insert_num int, c1 CHAR(8), c2 CHAR(32), c3 VARCHAR(15), c4 VARCHAR(18), c5 DECIMAL(10,2), c6 DECIMAL(25,15), b STRING)
+row format delimited fields terminated by '|'
+stored as textfile;
+load data local inpath '../../data/files/schema_evolution/same_type1_b.txt' overwrite into table same_type1_b_txt;
 
---** insert into table part_change_same_type_different_params partition(part=1) select * from same_type1_b_txt;
+insert into table part_change_same_type_different_params partition(part=1) select * from same_type1_b_txt;
 
---** CREATE TABLE same_type1_c_txt(insert_num int, c1 CHAR(8), c2 CHAR(32), c3 VARCHAR(15), c4 VARCHAR(18), c5 DECIMAL(10,2), c6 DECIMAL(25,15), b STRING)
---** row format delimited fields terminated by '|'
---** stored as textfile;
---** load data local inpath '../../data/files/schema_evolution/same_type1_c.txt' overwrite into table same_type1_c_txt;
+CREATE TABLE same_type1_c_txt(insert_num int, c1 CHAR(8), c2 CHAR(32), c3 VARCHAR(15), c4 VARCHAR(18), c5 DECIMAL(10,2), c6 DECIMAL(25,15), b STRING)
+row format delimited fields terminated by '|'
+stored as textfile;
+load data local inpath '../../data/files/schema_evolution/same_type1_c.txt' overwrite into table same_type1_c_txt;
 
---** insert into table part_change_same_type_different_params partition(part=2) select * from same_type1_c_txt;
+insert into table part_change_same_type_different_params partition(part=2) select * from same_type1_c_txt;
 
---** explain vectorization detail
---** select insert_num,part,c1,c2,c3,c4,c5,c6,b from part_change_same_type_different_params;
+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;
+select insert_num,part,c1,c2,c3,c4,c5,c6,b from part_change_same_type_different_params;
 
 drop table part_change_same_type_different_params;

http://git-wip-us.apache.org/repos/asf/hive/blob/c46c4c90/ql/src/test/queries/clientpositive/schema_evol_text_vec_table_llap_io.q
----------------------------------------------------------------------
diff --git a/ql/src/test/queries/clientpositive/schema_evol_text_vec_table_llap_io.q b/ql/src/test/queries/clientpositive/schema_evol_text_vec_table_llap_io.q
index 33128d3..cdd4dd4 100644
--- a/ql/src/test/queries/clientpositive/schema_evol_text_vec_table_llap_io.q
+++ b/ql/src/test/queries/clientpositive/schema_evol_text_vec_table_llap_io.q
@@ -39,12 +39,12 @@ select insert_num,a,b from table_add_int_permute_select;
 select insert_num,a,b from table_add_int_permute_select;
 
 -- Table-Non-Cascade ADD COLUMNS ...
---** alter table table_add_int_permute_select add columns(c int);
+alter table table_add_int_permute_select add columns(c int);
 
---** insert into table table_add_int_permute_select VALUES (111, 80000, 'new', 80000);
+insert into table table_add_int_permute_select VALUES (111, 80000, 'new', 80000);
 
---** explain vectorization detail
---** select insert_num,a,b from table_add_int_permute_select;
+explain vectorization detail
+select insert_num,a,b from table_add_int_permute_select;
 
 -- SELECT permutation columns to make sure NULL defaulting works right
 --** select insert_num,a,b from table_add_int_permute_select;
@@ -67,12 +67,12 @@ select insert_num,a,b from table_add_int_string_permute_select;
 select insert_num,a,b from table_add_int_string_permute_select;
 
 -- Table-Non-Cascade ADD COLUMNS ...
---** alter table table_add_int_string_permute_select add columns(c int, d string);
+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');
+insert into table table_add_int_string_permute_select VALUES (111, 80000, 'new', 80000, 'filler');
 
---** explain vectorization detail
---** select insert_num,a,b from table_add_int_string_permute_select;
+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
 --** select insert_num,a,b from table_add_int_string_permute_select;
@@ -104,12 +104,12 @@ 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;
 
 -- Table-Non-Cascade CHANGE COLUMNS ...
---** alter table table_change_string_group_double replace columns (insert_num int, c1 DOUBLE, c2 DOUBLE, c3 DOUBLE, b STRING);
+alter table table_change_string_group_double replace columns (insert_num int, c1 DOUBLE, c2 DOUBLE, c3 DOUBLE, b STRING);
 
---** insert into table table_change_string_group_double VALUES (111, 789.321, 789.321, 789.321, 'new');
+insert into table table_change_string_group_double VALUES (111, 789.321, 789.321, 789.321, 'new');
 
---** explain vectorization detail
---** select insert_num,c1,c2,c3,b from table_change_string_group_double;
+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;
 
@@ -129,12 +129,12 @@ insert into table table_change_date_group_string_group_date_group SELECT insert_
 explain vectorization detail
 select insert_num,c1,c2,c3,c4,c5,c6,c7,c8,c9,c10,b from table_change_date_group_string_group_date_group;
 
-select insert_num,c1,c2,c3,c4,c5,c6,c7,c8,c9,c10,b from table_change_date_group_string_group_date_group;
+--** select insert_num,c1,c2,c3,c4,c5,c6,c7,c8,c9,c10,b from table_change_date_group_string_group_date_group;
 
 -- Table-Non-Cascade CHANGE COLUMNS ...
---** alter table table_change_date_group_string_group_date_group replace columns(insert_num int, c1 STRING, c2 CHAR(50), c3 CHAR(15), c4 VARCHAR(50), c5 VARCHAR(15), c6 STRING, c7 CHAR(50), c8 CHAR(15), c9 VARCHAR(50), c10 VARCHAR(15), b STRING);
+alter table table_change_date_group_string_group_date_group replace columns(insert_num int, c1 STRING, c2 CHAR(50), c3 CHAR(15), c4 VARCHAR(50), c5 VARCHAR(15), c6 STRING, c7 CHAR(50), c8 CHAR(15), c9 VARCHAR(50), c10 VARCHAR(15), b STRING);
 
---** insert into table table_change_date_group_string_group_date_group VALUES (111, 'filler', 'filler', 'filler', 'filler', 'filler', 'filler', 'filler', 'filler', 'filler', 'filler', 'new');
+insert into table table_change_date_group_string_group_date_group VALUES (111, 'filler', 'filler', 'filler', 'filler', 'filler', 'filler', 'filler', 'filler', 'filler', 'filler', 'new');
 
 --** select insert_num,c1,c2,c3,c4,c5,c6,c7,c8,c9,c10,b from table_change_date_group_string_group_date_group;
 
@@ -168,23 +168,23 @@ insert into table table_change_numeric_group_string_group_multi_ints_string_grou
 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;
+--** 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;
 
 -- Table-Non-Cascade CHANGE COLUMNS ...
---** alter table table_change_numeric_group_string_group_multi_ints_string_group replace columns (insert_num int,
---**              c1 STRING, c2 STRING, c3 STRING, c4 STRING,
---**              c5 CHAR(50), c6 CHAR(50), c7 CHAR(50), c8 CHAR(50), c9 CHAR(5), c10 CHAR(5), c11 CHAR(5), c12 CHAR(5),
---**              c13 VARCHAR(50), c14 VARCHAR(50), c15 VARCHAR(50), c16 VARCHAR(50), c17 VARCHAR(5), c18 VARCHAR(5), c19 VARCHAR(5), c20 VARCHAR(5),
---**              b STRING) ;
-
---** insert into table table_change_numeric_group_string_group_multi_ints_string_group VALUES (111,
---**             'filler', 'filler', 'filler', 'filler',
---**             'filler', 'filler', 'filler', 'filler', 'filler', 'filler', 'filler', 'filler',
---**             'filler', 'filler', 'filler', 'filler', 'filler', 'filler', 'filler', 'filler',
---**             'new');
-
---** 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;
+alter table table_change_numeric_group_string_group_multi_ints_string_group replace columns (insert_num int,
+             c1 STRING, c2 STRING, c3 STRING, c4 STRING,
+             c5 CHAR(50), c6 CHAR(50), c7 CHAR(50), c8 CHAR(50), c9 CHAR(5), c10 CHAR(5), c11 CHAR(5), c12 CHAR(5),
+             c13 VARCHAR(50), c14 VARCHAR(50), c15 VARCHAR(50), c16 VARCHAR(50), c17 VARCHAR(5), c18 VARCHAR(5), c19 VARCHAR(5), c20 VARCHAR(5),
+             b STRING) ;
+
+insert into table table_change_numeric_group_string_group_multi_ints_string_group VALUES (111,
+            'filler', 'filler', 'filler', 'filler',
+            'filler', 'filler', 'filler', 'filler', 'filler', 'filler', 'filler', 'filler',
+            'filler', 'filler', 'filler', 'filler', 'filler', 'filler', 'filler', 'filler',
+            'new');
+
+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;
 
@@ -214,23 +214,23 @@ insert into table table_change_numeric_group_string_group_floating_string_group
 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;
+--** 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;
 
 -- Table-Non-Cascade CHANGE COLUMNS ...
---** alter table table_change_numeric_group_string_group_floating_string_group replace columns (insert_num int,
---**               c1 STRING, c2 STRING, c3 STRING,
---**               c4 CHAR(50), c5 CHAR(50), c6 CHAR(50), c7 CHAR(7), c8 CHAR(7), c9 CHAR(7),
---**               c10 VARCHAR(50), c11 VARCHAR(50), c12 VARCHAR(50), c13 VARCHAR(7), c14 VARCHAR(7), c15 VARCHAR(7),
---**               b STRING);
-
---** insert into table table_change_numeric_group_string_group_floating_string_group VALUES (111,
---**              'filler', 'filler', 'filler',
---**              'filler', 'filler', 'filler', 'filler', 'filler', 'filler',
---**              'filler', 'filler', 'filler', 'filler', 'filler', 'filler',
---**              'new');
-
---** 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;
+alter table table_change_numeric_group_string_group_floating_string_group replace columns (insert_num int,
+              c1 STRING, c2 STRING, c3 STRING,
+              c4 CHAR(50), c5 CHAR(50), c6 CHAR(50), c7 CHAR(7), c8 CHAR(7), c9 CHAR(7),
+              c10 VARCHAR(50), c11 VARCHAR(50), c12 VARCHAR(50), c13 VARCHAR(7), c14 VARCHAR(7), c15 VARCHAR(7),
+              b STRING);
+
+insert into table table_change_numeric_group_string_group_floating_string_group VALUES (111,
+             'filler', 'filler', 'filler',
+             'filler', 'filler', 'filler', 'filler', 'filler', 'filler',
+             'filler', 'filler', 'filler', 'filler', 'filler', 'filler',
+             'new');
+
+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;
 
@@ -259,19 +259,19 @@ insert into table table_change_string_group_string_group_string SELECT insert_nu
 explain vectorization detail
 select insert_num,c1,c2,c3,c4,b from table_change_string_group_string_group_string;
 
-select insert_num,c1,c2,c3,c4,b from table_change_string_group_string_group_string;
+--** select insert_num,c1,c2,c3,c4,b from table_change_string_group_string_group_string;
 
 -- Table-Non-Cascade CHANGE COLUMNS ...
---** alter table table_change_string_group_string_group_string replace columns (insert_num int,
---**            c1 CHAR(50), c2 CHAR(9), c3 VARCHAR(50), c4 CHAR(9),
---**            c5 VARCHAR(50), c6 VARCHAR(9), c7 STRING,
---**            c8 CHAR(50), c9 CHAR(9), c10 STRING, b STRING) ;
+alter table table_change_string_group_string_group_string replace columns (insert_num int,
+           c1 CHAR(50), c2 CHAR(9), c3 VARCHAR(50), c4 CHAR(9),
+           c5 VARCHAR(50), c6 VARCHAR(9), c7 STRING,
+           c8 CHAR(50), c9 CHAR(9), c10 STRING, b STRING) ;
 
---** insert into table table_change_string_group_string_group_string VALUES (111,
---**           'filler', 'filler', 'filler', 'filler',
---**           'filler', 'filler', 'filler',
---**           'filler', 'filler', 'filler',
---**           'new');
+insert into table table_change_string_group_string_group_string VALUES (111,
+          'filler', 'filler', 'filler', 'filler',
+          'filler', 'filler', 'filler',
+          'filler', 'filler', 'filler',
+          'new');
 
 --** select insert_num,c1,c2,c3,c4,c5,c6,c7,c8,c9,c10,b from table_change_string_group_string_group_string;
 
@@ -307,22 +307,22 @@ insert into table table_change_lower_to_higher_numeric_group_tinyint_to_bigint S
 explain vectorization detail
 select insert_num,c1,c2,c3,c4,c5,c6,c7,c8,c9,c10,c11,c12,c13,c14,c15,c16,c17,c18,b from table_change_lower_to_higher_numeric_group_tinyint_to_bigint;
 
-select insert_num,c1,c2,c3,c4,c5,c6,c7,c8,c9,c10,c11,c12,c13,c14,c15,c16,c17,c18,b from table_change_lower_to_higher_numeric_group_tinyint_to_bigint;
+--** select insert_num,c1,c2,c3,c4,c5,c6,c7,c8,c9,c10,c11,c12,c13,c14,c15,c16,c17,c18,b from table_change_lower_to_higher_numeric_group_tinyint_to_bigint;
 
 -- Table-Non-Cascade CHANGE COLUMNS ...
---** alter table table_change_lower_to_higher_numeric_group_tinyint_to_bigint replace columns (insert_num int,
---**              c1 SMALLINT, c2 INT, c3 BIGINT, c4 decimal(38,18), c5 FLOAT, c6 DOUBLE,
---**              c7 INT, c8 BIGINT, c9 decimal(38,18), c10 FLOAT, c11 DOUBLE,
---**              c12 BIGINT, c13 decimal(38,18), c14 FLOAT, c15 DOUBLE,
---**              c16 decimal(38,18), c17 FLOAT, c18 DOUBLE,
---**              b STRING) ;
-
---** insert into table table_change_lower_to_higher_numeric_group_tinyint_to_bigint VALUES (111,
---**             7000, 80000, 90000000, 1234.5678, 9876.543, 789.321,
---**             80000, 90000000, 1234.5678, 9876.543, 789.321,
---**             90000000, 1234.5678, 9876.543, 789.321,
---**             1234.5678, 9876.543, 789.321,
---**            'new');
+alter table table_change_lower_to_higher_numeric_group_tinyint_to_bigint replace columns (insert_num int,
+             c1 SMALLINT, c2 INT, c3 BIGINT, c4 decimal(38,18), c5 FLOAT, c6 DOUBLE,
+             c7 INT, c8 BIGINT, c9 decimal(38,18), c10 FLOAT, c11 DOUBLE,
+             c12 BIGINT, c13 decimal(38,18), c14 FLOAT, c15 DOUBLE,
+             c16 decimal(38,18), c17 FLOAT, c18 DOUBLE,
+             b STRING) ;
+
+insert into table table_change_lower_to_higher_numeric_group_tinyint_to_bigint VALUES (111,
+            7000, 80000, 90000000, 1234.5678, 9876.543, 789.321,
+            80000, 90000000, 1234.5678, 9876.543, 789.321,
+            90000000, 1234.5678, 9876.543, 789.321,
+            1234.5678, 9876.543, 789.321,
+           'new');
 
 --** select insert_num,c1,c2,c3,c4,c5,c6,c7,c8,c9,c10,c11,c12,c13,c14,c15,c16,c17,c18,b from table_change_lower_to_higher_numeric_group_tinyint_to_bigint;
 
@@ -348,12 +348,12 @@ insert into table table_change_lower_to_higher_numeric_group_decimal_to_float SE
 explain vectorization detail
 select insert_num,c1,c2,c3,b from table_change_lower_to_higher_numeric_group_decimal_to_float;
 
-select insert_num,c1,c2,c3,b from table_change_lower_to_higher_numeric_group_decimal_to_float;
+--** select insert_num,c1,c2,c3,b from table_change_lower_to_higher_numeric_group_decimal_to_float;
 
 -- Table-Non-Cascade CHANGE COLUMNS ...
---** alter table table_change_lower_to_higher_numeric_group_decimal_to_float replace columns (insert_num int, c1 float, c2 double, c3 DOUBLE, b STRING) ;
+alter table table_change_lower_to_higher_numeric_group_decimal_to_float replace columns (insert_num int, c1 float, c2 double, c3 DOUBLE, b STRING) ;
 
---** insert into table table_change_lower_to_higher_numeric_group_decimal_to_float VALUES (111, 1234.5678, 9876.543, 1234.5678, 'new');
+insert into table table_change_lower_to_higher_numeric_group_decimal_to_float VALUES (111, 1234.5678, 9876.543, 1234.5678, 'new');
 
 --** select insert_num,c1,c2,c3,b from table_change_lower_to_higher_numeric_group_decimal_to_float;
 

http://git-wip-us.apache.org/repos/asf/hive/blob/c46c4c90/ql/src/test/queries/clientpositive/schema_evol_text_vecrow_part_all_complex_llap_io.q
----------------------------------------------------------------------
diff --git a/ql/src/test/queries/clientpositive/schema_evol_text_vecrow_part_all_complex_llap_io.q b/ql/src/test/queries/clientpositive/schema_evol_text_vecrow_part_all_complex_llap_io.q
index 09f06d3..f349a8e 100644
--- a/ql/src/test/queries/clientpositive/schema_evol_text_vecrow_part_all_complex_llap_io.q
+++ b/ql/src/test/queries/clientpositive/schema_evol_text_vecrow_part_all_complex_llap_io.q
@@ -37,29 +37,29 @@ insert into table part_change_various_various_struct1 partition(part=1) select *
 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;
+--** select insert_num,part,s1,b from part_change_various_various_struct1;
 
 -- Table-Non-Cascade CHANGE COLUMNS ...
---** alter table part_change_various_various_struct1 replace columns (insert_num int, s1 STRUCT<c1:STRING, c2:STRING, c3:STRING, c4:STRING, c5:STRING, c6:STRING, c7:STRING, c8:STRING, c9:STRING, c10:STRING, c11:STRING, c12:STRING, c13:STRING>, b STRING);
+alter table part_change_various_various_struct1 replace columns (insert_num int, s1 STRUCT<c1:STRING, c2:STRING, c3:STRING, c4:STRING, c5:STRING, c6:STRING, c7:STRING, c8:STRING, c9:STRING, c10:STRING, c11:STRING, c12:STRING, c13:STRING>, b STRING);
 
---** CREATE TABLE complex_struct1_b_txt(insert_num int, s1 STRUCT<c1:STRING, c2:STRING, c3:STRING, c4:STRING, c5:STRING, c6:STRING, c7:STRING, c8:STRING, c9:STRING, c10:STRING, c11:STRING, c12:STRING, c13:STRING>, b STRING)
---** row format delimited fields terminated by '|'
---** collection items terminated by ','
---** map keys terminated by ':' stored as textfile;
---** load data local inpath '../../data/files/schema_evolution/complex_struct1_b.txt' overwrite into table complex_struct1_b_txt;
+CREATE TABLE complex_struct1_b_txt(insert_num int, s1 STRUCT<c1:STRING, c2:STRING, c3:STRING, c4:STRING, c5:STRING, c6:STRING, c7:STRING, c8:STRING, c9:STRING, c10:STRING, c11:STRING, c12:STRING, c13:STRING>, b STRING)
+row format delimited fields terminated by '|'
+collection items terminated by ','
+map keys terminated by ':' stored as textfile;
+load data local inpath '../../data/files/schema_evolution/complex_struct1_b.txt' overwrite into table complex_struct1_b_txt;
 
---** insert into table part_change_various_various_struct1 partition(part=2) select * from complex_struct1_b_txt;
+insert into table part_change_various_various_struct1 partition(part=2) select * from complex_struct1_b_txt;
 
---** CREATE TABLE complex_struct1_c_txt(insert_num int, s1 STRUCT<c1:STRING, c2:STRING, c3:STRING, c4:STRING, c5:STRING, c6:STRING, c7:STRING, c8:STRING, c9:STRING, c10:STRING, c11:STRING, c12:STRING, c13:STRING>, b STRING)
---** row format delimited fields terminated by '|'
---** collection items terminated by ','
---** map keys terminated by ':' stored as textfile;
---** load data local inpath '../../data/files/schema_evolution/complex_struct1_c.txt' overwrite into table complex_struct1_c_txt;
+CREATE TABLE complex_struct1_c_txt(insert_num int, s1 STRUCT<c1:STRING, c2:STRING, c3:STRING, c4:STRING, c5:STRING, c6:STRING, c7:STRING, c8:STRING, c9:STRING, c10:STRING, c11:STRING, c12:STRING, c13:STRING>, b STRING)
+row format delimited fields terminated by '|'
+collection items terminated by ','
+map keys terminated by ':' stored as textfile;
+load data local inpath '../../data/files/schema_evolution/complex_struct1_c.txt' overwrite into table complex_struct1_c_txt;
 
---** insert into table part_change_various_various_struct1 partition(part=1) select * from complex_struct1_c_txt;
+insert into table part_change_various_various_struct1 partition(part=1) select * from complex_struct1_c_txt;
 
---** explain vectorization detail
---** select insert_num,part,s1,b from part_change_various_various_struct1;
+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;
 
@@ -79,50 +79,50 @@ insert into table part_add_various_various_struct2 partition(part=1)
 explain vectorization detail
 select insert_num,part,b from part_add_various_various_struct2;
 
-select insert_num,part,b from part_add_various_various_struct2;
+--** select insert_num,part,b from part_add_various_various_struct2;
 
 -- Table-Non-Cascade ADD COLUMN ...
---** alter table part_add_various_various_struct2 ADD columns (s2 STRUCT<c1:BOOLEAN, c2:TINYINT, c3:SMALLINT, c4:INT, c5:BIGINT, c6:FLOAT, c7:DOUBLE, c8:DECIMAL(38,18), c9:CHAR(25), c10:VARCHAR(25), c11:TIMESTAMP, c12:DATE, c13:BINARY>);
+alter table part_add_various_various_struct2 ADD columns (s2 STRUCT<c1:BOOLEAN, c2:TINYINT, c3:SMALLINT, c4:INT, c5:BIGINT, c6:FLOAT, c7:DOUBLE, c8:DECIMAL(38,18), c9:CHAR(25), c10:VARCHAR(25), c11:TIMESTAMP, c12:DATE, c13:BINARY>);
 
---** CREATE TABLE complex_struct2_a_txt(insert_num int, b STRING, s2 STRUCT<c1:BOOLEAN, c2:TINYINT, c3:SMALLINT, c4:INT, c5:BIGINT, c6:FLOAT, c7:DOUBLE, c8:DECIMAL(38,18), c9:CHAR(25), c10:VARCHAR(25), c11:TIMESTAMP, c12:DATE, c13:BINARY>)
---** row format delimited fields terminated by '|'
---** collection items terminated by ','
---** map keys terminated by ':' stored as textfile;
---** load data local inpath '../../data/files/schema_evolution/complex_struct2_a.txt' overwrite into table complex_struct2_a_txt;
+CREATE TABLE complex_struct2_a_txt(insert_num int, b STRING, s2 STRUCT<c1:BOOLEAN, c2:TINYINT, c3:SMALLINT, c4:INT, c5:BIGINT, c6:FLOAT, c7:DOUBLE, c8:DECIMAL(38,18), c9:CHAR(25), c10:VARCHAR(25), c11:TIMESTAMP, c12:DATE, c13:BINARY>)
+row format delimited fields terminated by '|'
+collection items terminated by ','
+map keys terminated by ':' stored as textfile;
+load data local inpath '../../data/files/schema_evolution/complex_struct2_a.txt' overwrite into table complex_struct2_a_txt;
 
---** insert into table part_add_various_various_struct2 partition(part=1) select * from complex_struct2_a_txt;
+insert into table part_add_various_various_struct2 partition(part=1) select * from complex_struct2_a_txt;
 
---** CREATE TABLE complex_struct2_b_txt(insert_num int, b STRING, s2 STRUCT<c1:BOOLEAN, c2:TINYINT, c3:SMALLINT, c4:INT, c5:BIGINT, c6:FLOAT, c7:DOUBLE, c8:DECIMAL(38,18), c9:CHAR(25), c10:VARCHAR(25), c11:TIMESTAMP, c12:DATE, c13:BINARY>)
---** row format delimited fields terminated by '|'
---** collection items terminated by ','
---** map keys terminated by ':' stored as textfile;
---** load data local inpath '../../data/files/schema_evolution/complex_struct2_b.txt' overwrite into table complex_struct2_b_txt;
+CREATE TABLE complex_struct2_b_txt(insert_num int, b STRING, s2 STRUCT<c1:BOOLEAN, c2:TINYINT, c3:SMALLINT, c4:INT, c5:BIGINT, c6:FLOAT, c7:DOUBLE, c8:DECIMAL(38,18), c9:CHAR(25), c10:VARCHAR(25), c11:TIMESTAMP, c12:DATE, c13:BINARY>)
+row format delimited fields terminated by '|'
+collection items terminated by ','
+map keys terminated by ':' stored as textfile;
+load data local inpath '../../data/files/schema_evolution/complex_struct2_b.txt' overwrite into table complex_struct2_b_txt;
 
---** insert into table part_add_various_various_struct2 partition(part=2) select * from complex_struct2_b_txt;
+insert into table part_add_various_various_struct2 partition(part=2) select * from complex_struct2_b_txt;
 
 --** select insert_num,part,b,s2 from part_add_various_various_struct2;
 
 -- Table-Non-Cascade CHANGE COLUMNS ...
---** alter table part_add_various_various_struct2 REPLACE columns (insert_num int, b STRING, s2 STRUCT<c1:STRING, c2:STRING, c3:STRING, c4:STRING, c5:STRING, c6:STRING, c7:STRING, c8:STRING, c9:STRING, c10:STRING, c11:STRING, c12:STRING, c13:STRING>);
+alter table part_add_various_various_struct2 REPLACE columns (insert_num int, b STRING, s2 STRUCT<c1:STRING, c2:STRING, c3:STRING, c4:STRING, c5:STRING, c6:STRING, c7:STRING, c8:STRING, c9:STRING, c10:STRING, c11:STRING, c12:STRING, c13:STRING>);
 
---** CREATE TABLE complex_struct2_c_txt(insert_num int, b STRING, s2 STRUCT<c1:STRING, c2:STRING, c3:STRING, c4:STRING, c5:STRING, c6:STRING, c7:STRING, c8:STRING, c9:STRING, c10:STRING, c11:STRING, c12:STRING, c13:STRING>)
---** row format delimited fields terminated by '|'
---** collection items terminated by ','
---** map keys terminated by ':' stored as textfile;
---** load data local inpath '../../data/files/schema_evolution/complex_struct2_c.txt' overwrite into table complex_struct2_c_txt;
+CREATE TABLE complex_struct2_c_txt(insert_num int, b STRING, s2 STRUCT<c1:STRING, c2:STRING, c3:STRING, c4:STRING, c5:STRING, c6:STRING, c7:STRING, c8:STRING, c9:STRING, c10:STRING, c11:STRING, c12:STRING, c13:STRING>)
+row format delimited fields terminated by '|'
+collection items terminated by ','
+map keys terminated by ':' stored as textfile;
+load data local inpath '../../data/files/schema_evolution/complex_struct2_c.txt' overwrite into table complex_struct2_c_txt;
 
---** insert into table part_add_various_various_struct2 partition(part=2) select * from complex_struct2_c_txt;
+insert into table part_add_various_various_struct2 partition(part=2) select * from complex_struct2_c_txt;
 
---** CREATE TABLE complex_struct2_d_txt(insert_num int, b STRING, s2 STRUCT<c1:STRING, c2:STRING, c3:STRING, c4:STRING, c5:STRING, c6:STRING, c7:STRING, c8:STRING, c9:STRING, c10:STRING, c11:STRING, c12:STRING, c13:STRING>)
---** row format delimited fields terminated by '|'
---** collection items terminated by ','
---** map keys terminated by ':' stored as textfile;
---** load data local inpath '../../data/files/schema_evolution/complex_struct2_d.txt' overwrite into table complex_struct2_d_txt;
+CREATE TABLE complex_struct2_d_txt(insert_num int, b STRING, s2 STRUCT<c1:STRING, c2:STRING, c3:STRING, c4:STRING, c5:STRING, c6:STRING, c7:STRING, c8:STRING, c9:STRING, c10:STRING, c11:STRING, c12:STRING, c13:STRING>)
+row format delimited fields terminated by '|'
+collection items terminated by ','
+map keys terminated by ':' stored as textfile;
+load data local inpath '../../data/files/schema_evolution/complex_struct2_d.txt' overwrite into table complex_struct2_d_txt;
 
---** insert into table part_add_various_various_struct2 partition(part=1) select * from complex_struct2_d_txt;
+insert into table part_add_various_various_struct2 partition(part=1) select * from complex_struct2_d_txt;
 
---** explain vectorization detail
---** select insert_num,part,b,s2 from part_add_various_various_struct2;
+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;
 
@@ -147,29 +147,29 @@ insert into table part_add_to_various_various_struct4 partition(part=1) select *
 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;
+--** select insert_num,part,b,s3 from part_add_to_various_various_struct4;
 
 -- Table-Non-Cascade CHANGE COLUMNS ...
---** alter table part_add_to_various_various_struct4 replace columns (insert_num int, b STRING, s3 STRUCT<c1:BOOLEAN, c2:TINYINT, c3:SMALLINT, c4:INT, c5:BIGINT, c6:FLOAT, c7:DOUBLE, c8:DECIMAL(38,18), c9:CHAR(25), c10:VARCHAR(25), c11:TIMESTAMP, c12:DATE, c13:BINARY>);
+alter table part_add_to_various_various_struct4 replace columns (insert_num int, b STRING, s3 STRUCT<c1:BOOLEAN, c2:TINYINT, c3:SMALLINT, c4:INT, c5:BIGINT, c6:FLOAT, c7:DOUBLE, c8:DECIMAL(38,18), c9:CHAR(25), c10:VARCHAR(25), c11:TIMESTAMP, c12:DATE, c13:BINARY>);
 
---** CREATE TABLE complex_struct4_b_txt(insert_num int, b STRING, s3 STRUCT<c1:BOOLEAN, c2:TINYINT, c3:SMALLINT, c4:INT, c5:BIGINT, c6:FLOAT, c7:DOUBLE, c8:DECIMAL(38,18), c9:CHAR(25), c10:VARCHAR(25), c11:TIMESTAMP, c12:DATE, c13:BINARY>)
---** row format delimited fields terminated by '|'
---** collection items terminated by ','
---** map keys terminated by ':' stored as textfile;
---** load data local inpath '../../data/files/schema_evolution/complex_struct4_b.txt' overwrite into table complex_struct4_b_txt;
+CREATE TABLE complex_struct4_b_txt(insert_num int, b STRING, s3 STRUCT<c1:BOOLEAN, c2:TINYINT, c3:SMALLINT, c4:INT, c5:BIGINT, c6:FLOAT, c7:DOUBLE, c8:DECIMAL(38,18), c9:CHAR(25), c10:VARCHAR(25), c11:TIMESTAMP, c12:DATE, c13:BINARY>)
+row format delimited fields terminated by '|'
+collection items terminated by ','
+map keys terminated by ':' stored as textfile;
+load data local inpath '../../data/files/schema_evolution/complex_struct4_b.txt' overwrite into table complex_struct4_b_txt;
 
---** insert into table part_add_to_various_various_struct4 partition(part=2) select * from complex_struct4_b_txt;
+insert into table part_add_to_various_various_struct4 partition(part=2) select * from complex_struct4_b_txt;
 
---** CREATE TABLE complex_struct4_c_txt(insert_num int, b STRING, s3 STRUCT<c1:BOOLEAN, c2:TINYINT, c3:SMALLINT, c4:INT, c5:BIGINT, c6:FLOAT, c7:DOUBLE, c8:DECIMAL(38,18), c9:CHAR(25), c10:VARCHAR(25), c11:TIMESTAMP, c12:DATE, c13:BINARY>)
---** row format delimited fields terminated by '|'
---** collection items terminated by ','
---** map keys terminated by ':' stored as textfile;
---** load data local inpath '../../data/files/schema_evolution/complex_struct4_c.txt' overwrite into table complex_struct4_c_txt;
+CREATE TABLE complex_struct4_c_txt(insert_num int, b STRING, s3 STRUCT<c1:BOOLEAN, c2:TINYINT, c3:SMALLINT, c4:INT, c5:BIGINT, c6:FLOAT, c7:DOUBLE, c8:DECIMAL(38,18), c9:CHAR(25), c10:VARCHAR(25), c11:TIMESTAMP, c12:DATE, c13:BINARY>)
+row format delimited fields terminated by '|'
+collection items terminated by ','
+map keys terminated by ':' stored as textfile;
+load data local inpath '../../data/files/schema_evolution/complex_struct4_c.txt' overwrite into table complex_struct4_c_txt;
 
---** insert into table part_add_to_various_various_struct4 partition(part=1) select * from complex_struct4_c_txt;
+insert into table part_add_to_various_various_struct4 partition(part=1) select * from complex_struct4_c_txt;
 
---** explain vectorization detail
---** select insert_num,part,b,s3 from part_add_to_various_various_struct4;
+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;