You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hive.apache.org by se...@apache.org on 2018/06/22 18:11:30 UTC

[5/7] hive git commit: HIVE-19016: Vectorization and Parquet: Disable vectorization for nested complex types (Matt McCline, reviewed by Vihang Karajgaonkar and Teddy Choi)

HIVE-19016: Vectorization and Parquet: Disable vectorization for nested complex types (Matt McCline, reviewed by Vihang Karajgaonkar and Teddy Choi)


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

Branch: refs/heads/master-txnstats
Commit: 6d532e7c4396a81b0afd16b66b4873c5fe9398ee
Parents: e36f6e4
Author: Matt McCline <mm...@hortonworks.com>
Authored: Fri Jun 22 10:46:24 2018 -0500
Committer: Matt McCline <mm...@hortonworks.com>
Committed: Fri Jun 22 10:46:24 2018 -0500

----------------------------------------------------------------------
 .../test/resources/testconfiguration.properties |   1 +
 .../hive/ql/optimizer/physical/Vectorizer.java  |  64 ++-
 .../vector_parquet_nested_two_level_complex.q   |  67 +++
 ...ector_parquet_nested_two_level_complex.q.out | 540 +++++++++++++++++++
 4 files changed, 670 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hive/blob/6d532e7c/itests/src/test/resources/testconfiguration.properties
----------------------------------------------------------------------
diff --git a/itests/src/test/resources/testconfiguration.properties b/itests/src/test/resources/testconfiguration.properties
index a3ddbda..93e2a44 100644
--- a/itests/src/test/resources/testconfiguration.properties
+++ b/itests/src/test/resources/testconfiguration.properties
@@ -759,6 +759,7 @@ minillaplocal.query.files=\
   vector_orc_null_check.q,\
   vector_order_null.q,\
   vector_outer_reference_windowed.q,\
+  vector_parquet_nested_two_level_complex.q,\
   vector_partitioned_date_time.q,\
   vector_ptf_1.q,\
   vector_ptf_part_simple.q,\

http://git-wip-us.apache.org/repos/asf/hive/blob/6d532e7c/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 f4e8207..7afbf04 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
@@ -221,6 +221,8 @@ import org.apache.hadoop.hive.serde2.objectinspector.PrimitiveObjectInspector.Pr
 import org.apache.hadoop.hive.serde2.objectinspector.StructField;
 import org.apache.hadoop.hive.serde2.objectinspector.StructObjectInspector;
 import org.apache.hadoop.hive.serde2.typeinfo.DecimalTypeInfo;
+import org.apache.hadoop.hive.serde2.typeinfo.ListTypeInfo;
+import org.apache.hadoop.hive.serde2.typeinfo.MapTypeInfo;
 import org.apache.hadoop.hive.serde2.typeinfo.PrimitiveTypeInfo;
 import org.apache.hadoop.hive.serde2.typeinfo.StructTypeInfo;
 import org.apache.hadoop.hive.serde2.typeinfo.TypeInfo;
@@ -1288,6 +1290,7 @@ public class Vectorizer implements PhysicalPlanResolver {
      */
     private boolean verifyAndSetVectorPartDesc(
         PartitionDesc pd, boolean isFullAcidTable,
+        List<TypeInfo> allTypeInfoList,
         Set<String> inputFileFormatClassNameSet,
         Map<VectorPartitionDesc, VectorPartitionDesc> vectorPartitionDescMap,
         Set<String> enabledConditionsMetSet, ArrayList<String> enabledConditionsNotMetList,
@@ -1332,8 +1335,13 @@ public class Vectorizer implements PhysicalPlanResolver {
 
       if (useVectorizedInputFileFormat) {
 
-        if (isInputFileFormatVectorized && !isInputFormatExcluded(inputFileFormatClassName,
-            vectorizedInputFormatExcludes)) {
+        if (isInputFileFormatVectorized &&
+            !isInputFormatExcluded(
+                inputFileFormatClassName,
+                vectorizedInputFormatExcludes) &&
+            !hasUnsupportedVectorizedParquetDataType(
+                inputFileFormatClass,
+                allTypeInfoList)) {
 
           addVectorizedInputFileFormatSupport(
               newSupportSet, isInputFileFormatVectorized, inputFileFormatClass);
@@ -1517,6 +1525,57 @@ public class Vectorizer implements PhysicalPlanResolver {
       return false;
     }
 
+    private boolean hasUnsupportedVectorizedParquetDataType(
+        Class<? extends InputFormat> inputFileFormatClass, List<TypeInfo> allTypeInfoList) {
+      if (!inputFileFormatClass.equals(org.apache.hadoop.hive.ql.io.parquet.MapredParquetInputFormat.class)) {
+        return false;
+      }
+
+      /*
+       * Currently, VectorizedParquetRecordReader cannot handle nested complex types.
+       */
+      for (TypeInfo typeInfo : allTypeInfoList) {
+        if (!(typeInfo instanceof PrimitiveTypeInfo)) {
+          switch (typeInfo.getCategory()) {
+          case LIST:
+            if (!(((ListTypeInfo) typeInfo).getListElementTypeInfo() instanceof PrimitiveTypeInfo)) {
+              return true;
+            }
+            break;
+          case MAP:
+            {
+              MapTypeInfo mapTypeInfo = (MapTypeInfo) typeInfo;
+              if (!(mapTypeInfo.getMapKeyTypeInfo() instanceof PrimitiveTypeInfo)) {
+                return true;
+              }
+              if (!(mapTypeInfo.getMapValueTypeInfo() instanceof PrimitiveTypeInfo)) {
+                return true;
+              }
+            }
+            break;
+          case STRUCT:
+            {
+              StructTypeInfo structTypeInfo = (StructTypeInfo) typeInfo;
+              List<TypeInfo> fieldTypeInfos = structTypeInfo.getAllStructFieldTypeInfos();
+              for (TypeInfo fieldTypeInfo : fieldTypeInfos) {
+                if (!(fieldTypeInfo instanceof PrimitiveTypeInfo)) {
+                  return true;
+                }
+              }
+            }
+            break;
+          case UNION:
+            // Not supported at all.
+            return false;
+          default:
+            throw new RuntimeException(
+                "Unsupported complex type category " + typeInfo.getCategory());
+          }
+        }
+      }
+      return false;
+    }
+
     private void setValidateInputFormatAndSchemaEvolutionExplain(MapWork mapWork,
         Set<String> inputFileFormatClassNameSet,
         Map<VectorPartitionDesc, VectorPartitionDesc> vectorPartitionDescMap,
@@ -1594,6 +1653,7 @@ public class Vectorizer implements PhysicalPlanResolver {
         final boolean isVerifiedVectorPartDesc =
             verifyAndSetVectorPartDesc(
               partDesc, isFullAcidTable,
+              allTypeInfoList,
               inputFileFormatClassNameSet,
               vectorPartitionDescMap,
               enabledConditionsMetSet, enabledConditionsNotMetList,

http://git-wip-us.apache.org/repos/asf/hive/blob/6d532e7c/ql/src/test/queries/clientpositive/vector_parquet_nested_two_level_complex.q
----------------------------------------------------------------------
diff --git a/ql/src/test/queries/clientpositive/vector_parquet_nested_two_level_complex.q b/ql/src/test/queries/clientpositive/vector_parquet_nested_two_level_complex.q
new file mode 100644
index 0000000..70480e6
--- /dev/null
+++ b/ql/src/test/queries/clientpositive/vector_parquet_nested_two_level_complex.q
@@ -0,0 +1,67 @@
+set hive.vectorized.execution.enabled=true;
+set hive.explain.user=false;
+set hive.fetch.task.conversion=none;
+
+create table nested_array_array_table (
+nested_array_array  array<array<int>>)
+STORED AS PARQUET;
+
+create table nested_array_map_table (
+nested_array_map  array<map<string,string>>)
+STORED AS PARQUET;
+
+create table nested_array_struct_table (
+nested_array_map  array<struct<latitude: DOUBLE, longitude: DOUBLE>>)
+STORED AS PARQUET;
+
+create table nested_map_array_table (
+nested_map_array  map<string,array<int>>)
+STORED AS PARQUET;
+
+create table nested_map_map_table (
+nested_map_map    map<string,map<string,string>>)
+STORED AS PARQUET;
+
+create table nested_map_struct_table (
+nested_map_struct    map<string,struct<latitude: DOUBLE, longitude: DOUBLE>>)
+STORED AS PARQUET;
+
+create table nested_struct_array_table (
+nested_struct_array struct<s:string, i:bigint, a:array<int>>)
+STORED AS PARQUET;
+
+create table nested_struct_map_table (
+nested_struct_map struct<s:string, i:bigint, m:map<string,string>>)
+STORED AS PARQUET;
+
+create table nested_struct_struct_table (
+nested_struct_struct struct<s:string, i:bigint, s2:struct<latitude: DOUBLE, longitude: DOUBLE>>)
+STORED AS PARQUET;
+
+
+EXPLAIN VECTORIZATION DETAIL
+SELECT * FROM nested_array_array_table;
+
+EXPLAIN VECTORIZATION DETAIL
+SELECT * FROM nested_array_map_table;
+
+EXPLAIN VECTORIZATION DETAIL
+SELECT * FROM nested_array_map_table;
+
+EXPLAIN VECTORIZATION DETAIL
+SELECT * FROM nested_map_array_table;
+
+EXPLAIN VECTORIZATION DETAIL
+SELECT * FROM nested_map_map_table;
+
+EXPLAIN VECTORIZATION DETAIL
+SELECT * FROM nested_map_struct_table;
+
+EXPLAIN VECTORIZATION DETAIL
+SELECT * FROM nested_struct_array_table;
+
+EXPLAIN VECTORIZATION DETAIL
+SELECT * FROM nested_struct_map_table;
+
+EXPLAIN VECTORIZATION DETAIL
+SELECT * FROM nested_struct_struct_table;

http://git-wip-us.apache.org/repos/asf/hive/blob/6d532e7c/ql/src/test/results/clientpositive/llap/vector_parquet_nested_two_level_complex.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/llap/vector_parquet_nested_two_level_complex.q.out b/ql/src/test/results/clientpositive/llap/vector_parquet_nested_two_level_complex.q.out
new file mode 100644
index 0000000..daeca1c
--- /dev/null
+++ b/ql/src/test/results/clientpositive/llap/vector_parquet_nested_two_level_complex.q.out
@@ -0,0 +1,540 @@
+PREHOOK: query: create table nested_array_array_table (
+nested_array_array  array<array<int>>)
+STORED AS PARQUET
+PREHOOK: type: CREATETABLE
+PREHOOK: Output: database:default
+PREHOOK: Output: default@nested_array_array_table
+POSTHOOK: query: create table nested_array_array_table (
+nested_array_array  array<array<int>>)
+STORED AS PARQUET
+POSTHOOK: type: CREATETABLE
+POSTHOOK: Output: database:default
+POSTHOOK: Output: default@nested_array_array_table
+PREHOOK: query: create table nested_array_map_table (
+nested_array_map  array<map<string,string>>)
+STORED AS PARQUET
+PREHOOK: type: CREATETABLE
+PREHOOK: Output: database:default
+PREHOOK: Output: default@nested_array_map_table
+POSTHOOK: query: create table nested_array_map_table (
+nested_array_map  array<map<string,string>>)
+STORED AS PARQUET
+POSTHOOK: type: CREATETABLE
+POSTHOOK: Output: database:default
+POSTHOOK: Output: default@nested_array_map_table
+PREHOOK: query: create table nested_array_struct_table (
+nested_array_map  array<struct<latitude: DOUBLE, longitude: DOUBLE>>)
+STORED AS PARQUET
+PREHOOK: type: CREATETABLE
+PREHOOK: Output: database:default
+PREHOOK: Output: default@nested_array_struct_table
+POSTHOOK: query: create table nested_array_struct_table (
+nested_array_map  array<struct<latitude: DOUBLE, longitude: DOUBLE>>)
+STORED AS PARQUET
+POSTHOOK: type: CREATETABLE
+POSTHOOK: Output: database:default
+POSTHOOK: Output: default@nested_array_struct_table
+PREHOOK: query: create table nested_map_array_table (
+nested_map_array  map<string,array<int>>)
+STORED AS PARQUET
+PREHOOK: type: CREATETABLE
+PREHOOK: Output: database:default
+PREHOOK: Output: default@nested_map_array_table
+POSTHOOK: query: create table nested_map_array_table (
+nested_map_array  map<string,array<int>>)
+STORED AS PARQUET
+POSTHOOK: type: CREATETABLE
+POSTHOOK: Output: database:default
+POSTHOOK: Output: default@nested_map_array_table
+PREHOOK: query: create table nested_map_map_table (
+nested_map_map    map<string,map<string,string>>)
+STORED AS PARQUET
+PREHOOK: type: CREATETABLE
+PREHOOK: Output: database:default
+PREHOOK: Output: default@nested_map_map_table
+POSTHOOK: query: create table nested_map_map_table (
+nested_map_map    map<string,map<string,string>>)
+STORED AS PARQUET
+POSTHOOK: type: CREATETABLE
+POSTHOOK: Output: database:default
+POSTHOOK: Output: default@nested_map_map_table
+PREHOOK: query: create table nested_map_struct_table (
+nested_map_struct    map<string,struct<latitude: DOUBLE, longitude: DOUBLE>>)
+STORED AS PARQUET
+PREHOOK: type: CREATETABLE
+PREHOOK: Output: database:default
+PREHOOK: Output: default@nested_map_struct_table
+POSTHOOK: query: create table nested_map_struct_table (
+nested_map_struct    map<string,struct<latitude: DOUBLE, longitude: DOUBLE>>)
+STORED AS PARQUET
+POSTHOOK: type: CREATETABLE
+POSTHOOK: Output: database:default
+POSTHOOK: Output: default@nested_map_struct_table
+PREHOOK: query: create table nested_struct_array_table (
+nested_struct_array struct<s:string, i:bigint, a:array<int>>)
+STORED AS PARQUET
+PREHOOK: type: CREATETABLE
+PREHOOK: Output: database:default
+PREHOOK: Output: default@nested_struct_array_table
+POSTHOOK: query: create table nested_struct_array_table (
+nested_struct_array struct<s:string, i:bigint, a:array<int>>)
+STORED AS PARQUET
+POSTHOOK: type: CREATETABLE
+POSTHOOK: Output: database:default
+POSTHOOK: Output: default@nested_struct_array_table
+PREHOOK: query: create table nested_struct_map_table (
+nested_struct_map struct<s:string, i:bigint, m:map<string,string>>)
+STORED AS PARQUET
+PREHOOK: type: CREATETABLE
+PREHOOK: Output: database:default
+PREHOOK: Output: default@nested_struct_map_table
+POSTHOOK: query: create table nested_struct_map_table (
+nested_struct_map struct<s:string, i:bigint, m:map<string,string>>)
+STORED AS PARQUET
+POSTHOOK: type: CREATETABLE
+POSTHOOK: Output: database:default
+POSTHOOK: Output: default@nested_struct_map_table
+PREHOOK: query: create table nested_struct_struct_table (
+nested_struct_struct struct<s:string, i:bigint, s2:struct<latitude: DOUBLE, longitude: DOUBLE>>)
+STORED AS PARQUET
+PREHOOK: type: CREATETABLE
+PREHOOK: Output: database:default
+PREHOOK: Output: default@nested_struct_struct_table
+POSTHOOK: query: create table nested_struct_struct_table (
+nested_struct_struct struct<s:string, i:bigint, s2:struct<latitude: DOUBLE, longitude: DOUBLE>>)
+STORED AS PARQUET
+POSTHOOK: type: CREATETABLE
+POSTHOOK: Output: database:default
+POSTHOOK: Output: default@nested_struct_struct_table
+PREHOOK: query: EXPLAIN VECTORIZATION DETAIL
+SELECT * FROM nested_array_array_table
+PREHOOK: type: QUERY
+POSTHOOK: query: EXPLAIN VECTORIZATION DETAIL
+SELECT * FROM nested_array_array_table
+POSTHOOK: type: QUERY
+PLAN VECTORIZATION:
+  enabled: true
+  enabledConditionsMet: [hive.vectorized.execution.enabled IS true]
+
+STAGE DEPENDENCIES:
+  Stage-1 is a root stage
+  Stage-0 depends on stages: Stage-1
+
+STAGE PLANS:
+  Stage: Stage-1
+    Tez
+#### A masked pattern was here ####
+      Vertices:
+        Map 1 
+            Map Operator Tree:
+                TableScan
+                  alias: nested_array_array_table
+                  Statistics: Num rows: 1 Data size: 1280 Basic stats: COMPLETE Column stats: NONE
+                  Select Operator
+                    expressions: nested_array_array (type: array<array<int>>)
+                    outputColumnNames: _col0
+                    Statistics: Num rows: 1 Data size: 1280 Basic stats: COMPLETE Column stats: NONE
+                    File Output Operator
+                      compressed: false
+                      Statistics: Num rows: 1 Data size: 1280 Basic stats: COMPLETE Column stats: NONE
+                      table:
+                          input format: org.apache.hadoop.mapred.SequenceFileInputFormat
+                          output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
+                          serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
+            Execution mode: llap
+            LLAP IO: all inputs (cache only)
+            Map Vectorization:
+                enabled: false
+                enabledConditionsNotMet: Row deserialization of vectorized input format not supported IS false, hive.vectorized.use.vectorized.input.format IS true AND hive.vectorized.input.format.excludes NOT CONTAINS org.apache.hadoop.hive.ql.io.parquet.MapredParquetInputFormat IS false
+                inputFileFormats: org.apache.hadoop.hive.ql.io.parquet.MapredParquetInputFormat
+
+  Stage: Stage-0
+    Fetch Operator
+      limit: -1
+      Processor Tree:
+        ListSink
+
+PREHOOK: query: EXPLAIN VECTORIZATION DETAIL
+SELECT * FROM nested_array_map_table
+PREHOOK: type: QUERY
+POSTHOOK: query: EXPLAIN VECTORIZATION DETAIL
+SELECT * FROM nested_array_map_table
+POSTHOOK: type: QUERY
+PLAN VECTORIZATION:
+  enabled: true
+  enabledConditionsMet: [hive.vectorized.execution.enabled IS true]
+
+STAGE DEPENDENCIES:
+  Stage-1 is a root stage
+  Stage-0 depends on stages: Stage-1
+
+STAGE PLANS:
+  Stage: Stage-1
+    Tez
+#### A masked pattern was here ####
+      Vertices:
+        Map 1 
+            Map Operator Tree:
+                TableScan
+                  alias: nested_array_map_table
+                  Statistics: Num rows: 1 Data size: 9280 Basic stats: COMPLETE Column stats: NONE
+                  Select Operator
+                    expressions: nested_array_map (type: array<map<string,string>>)
+                    outputColumnNames: _col0
+                    Statistics: Num rows: 1 Data size: 9280 Basic stats: COMPLETE Column stats: NONE
+                    File Output Operator
+                      compressed: false
+                      Statistics: Num rows: 1 Data size: 9280 Basic stats: COMPLETE Column stats: NONE
+                      table:
+                          input format: org.apache.hadoop.mapred.SequenceFileInputFormat
+                          output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
+                          serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
+            Execution mode: llap
+            LLAP IO: all inputs (cache only)
+            Map Vectorization:
+                enabled: false
+                enabledConditionsNotMet: Row deserialization of vectorized input format not supported IS false, hive.vectorized.use.vectorized.input.format IS true AND hive.vectorized.input.format.excludes NOT CONTAINS org.apache.hadoop.hive.ql.io.parquet.MapredParquetInputFormat IS false
+                inputFileFormats: org.apache.hadoop.hive.ql.io.parquet.MapredParquetInputFormat
+
+  Stage: Stage-0
+    Fetch Operator
+      limit: -1
+      Processor Tree:
+        ListSink
+
+PREHOOK: query: EXPLAIN VECTORIZATION DETAIL
+SELECT * FROM nested_array_map_table
+PREHOOK: type: QUERY
+POSTHOOK: query: EXPLAIN VECTORIZATION DETAIL
+SELECT * FROM nested_array_map_table
+POSTHOOK: type: QUERY
+PLAN VECTORIZATION:
+  enabled: true
+  enabledConditionsMet: [hive.vectorized.execution.enabled IS true]
+
+STAGE DEPENDENCIES:
+  Stage-1 is a root stage
+  Stage-0 depends on stages: Stage-1
+
+STAGE PLANS:
+  Stage: Stage-1
+    Tez
+#### A masked pattern was here ####
+      Vertices:
+        Map 1 
+            Map Operator Tree:
+                TableScan
+                  alias: nested_array_map_table
+                  Statistics: Num rows: 1 Data size: 9280 Basic stats: COMPLETE Column stats: NONE
+                  Select Operator
+                    expressions: nested_array_map (type: array<map<string,string>>)
+                    outputColumnNames: _col0
+                    Statistics: Num rows: 1 Data size: 9280 Basic stats: COMPLETE Column stats: NONE
+                    File Output Operator
+                      compressed: false
+                      Statistics: Num rows: 1 Data size: 9280 Basic stats: COMPLETE Column stats: NONE
+                      table:
+                          input format: org.apache.hadoop.mapred.SequenceFileInputFormat
+                          output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
+                          serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
+            Execution mode: llap
+            LLAP IO: all inputs (cache only)
+            Map Vectorization:
+                enabled: false
+                enabledConditionsNotMet: Row deserialization of vectorized input format not supported IS false, hive.vectorized.use.vectorized.input.format IS true AND hive.vectorized.input.format.excludes NOT CONTAINS org.apache.hadoop.hive.ql.io.parquet.MapredParquetInputFormat IS false
+                inputFileFormats: org.apache.hadoop.hive.ql.io.parquet.MapredParquetInputFormat
+
+  Stage: Stage-0
+    Fetch Operator
+      limit: -1
+      Processor Tree:
+        ListSink
+
+PREHOOK: query: EXPLAIN VECTORIZATION DETAIL
+SELECT * FROM nested_map_array_table
+PREHOOK: type: QUERY
+POSTHOOK: query: EXPLAIN VECTORIZATION DETAIL
+SELECT * FROM nested_map_array_table
+POSTHOOK: type: QUERY
+PLAN VECTORIZATION:
+  enabled: true
+  enabledConditionsMet: [hive.vectorized.execution.enabled IS true]
+
+STAGE DEPENDENCIES:
+  Stage-1 is a root stage
+  Stage-0 depends on stages: Stage-1
+
+STAGE PLANS:
+  Stage: Stage-1
+    Tez
+#### A masked pattern was here ####
+      Vertices:
+        Map 1 
+            Map Operator Tree:
+                TableScan
+                  alias: nested_map_array_table
+                  Statistics: Num rows: 1 Data size: 856 Basic stats: COMPLETE Column stats: NONE
+                  Select Operator
+                    expressions: nested_map_array (type: map<string,array<int>>)
+                    outputColumnNames: _col0
+                    Statistics: Num rows: 1 Data size: 856 Basic stats: COMPLETE Column stats: NONE
+                    File Output Operator
+                      compressed: false
+                      Statistics: Num rows: 1 Data size: 856 Basic stats: COMPLETE Column stats: NONE
+                      table:
+                          input format: org.apache.hadoop.mapred.SequenceFileInputFormat
+                          output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
+                          serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
+            Execution mode: llap
+            LLAP IO: all inputs (cache only)
+            Map Vectorization:
+                enabled: false
+                enabledConditionsNotMet: Row deserialization of vectorized input format not supported IS false, hive.vectorized.use.vectorized.input.format IS true AND hive.vectorized.input.format.excludes NOT CONTAINS org.apache.hadoop.hive.ql.io.parquet.MapredParquetInputFormat IS false
+                inputFileFormats: org.apache.hadoop.hive.ql.io.parquet.MapredParquetInputFormat
+
+  Stage: Stage-0
+    Fetch Operator
+      limit: -1
+      Processor Tree:
+        ListSink
+
+PREHOOK: query: EXPLAIN VECTORIZATION DETAIL
+SELECT * FROM nested_map_map_table
+PREHOOK: type: QUERY
+POSTHOOK: query: EXPLAIN VECTORIZATION DETAIL
+SELECT * FROM nested_map_map_table
+POSTHOOK: type: QUERY
+PLAN VECTORIZATION:
+  enabled: true
+  enabledConditionsMet: [hive.vectorized.execution.enabled IS true]
+
+STAGE DEPENDENCIES:
+  Stage-1 is a root stage
+  Stage-0 depends on stages: Stage-1
+
+STAGE PLANS:
+  Stage: Stage-1
+    Tez
+#### A masked pattern was here ####
+      Vertices:
+        Map 1 
+            Map Operator Tree:
+                TableScan
+                  alias: nested_map_map_table
+                  Statistics: Num rows: 1 Data size: 1656 Basic stats: COMPLETE Column stats: NONE
+                  Select Operator
+                    expressions: nested_map_map (type: map<string,map<string,string>>)
+                    outputColumnNames: _col0
+                    Statistics: Num rows: 1 Data size: 1656 Basic stats: COMPLETE Column stats: NONE
+                    File Output Operator
+                      compressed: false
+                      Statistics: Num rows: 1 Data size: 1656 Basic stats: COMPLETE Column stats: NONE
+                      table:
+                          input format: org.apache.hadoop.mapred.SequenceFileInputFormat
+                          output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
+                          serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
+            Execution mode: llap
+            LLAP IO: all inputs (cache only)
+            Map Vectorization:
+                enabled: false
+                enabledConditionsNotMet: Row deserialization of vectorized input format not supported IS false, hive.vectorized.use.vectorized.input.format IS true AND hive.vectorized.input.format.excludes NOT CONTAINS org.apache.hadoop.hive.ql.io.parquet.MapredParquetInputFormat IS false
+                inputFileFormats: org.apache.hadoop.hive.ql.io.parquet.MapredParquetInputFormat
+
+  Stage: Stage-0
+    Fetch Operator
+      limit: -1
+      Processor Tree:
+        ListSink
+
+PREHOOK: query: EXPLAIN VECTORIZATION DETAIL
+SELECT * FROM nested_map_struct_table
+PREHOOK: type: QUERY
+POSTHOOK: query: EXPLAIN VECTORIZATION DETAIL
+SELECT * FROM nested_map_struct_table
+POSTHOOK: type: QUERY
+PLAN VECTORIZATION:
+  enabled: true
+  enabledConditionsMet: [hive.vectorized.execution.enabled IS true]
+
+STAGE DEPENDENCIES:
+  Stage-1 is a root stage
+  Stage-0 depends on stages: Stage-1
+
+STAGE PLANS:
+  Stage: Stage-1
+    Tez
+#### A masked pattern was here ####
+      Vertices:
+        Map 1 
+            Map Operator Tree:
+                TableScan
+                  alias: nested_map_struct_table
+                  Statistics: Num rows: 1 Data size: 800 Basic stats: COMPLETE Column stats: NONE
+                  Select Operator
+                    expressions: nested_map_struct (type: map<string,struct<latitude:double,longitude:double>>)
+                    outputColumnNames: _col0
+                    Statistics: Num rows: 1 Data size: 800 Basic stats: COMPLETE Column stats: NONE
+                    File Output Operator
+                      compressed: false
+                      Statistics: Num rows: 1 Data size: 800 Basic stats: COMPLETE Column stats: NONE
+                      table:
+                          input format: org.apache.hadoop.mapred.SequenceFileInputFormat
+                          output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
+                          serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
+            Execution mode: llap
+            LLAP IO: all inputs (cache only)
+            Map Vectorization:
+                enabled: false
+                enabledConditionsNotMet: Row deserialization of vectorized input format not supported IS false, hive.vectorized.use.vectorized.input.format IS true AND hive.vectorized.input.format.excludes NOT CONTAINS org.apache.hadoop.hive.ql.io.parquet.MapredParquetInputFormat IS false
+                inputFileFormats: org.apache.hadoop.hive.ql.io.parquet.MapredParquetInputFormat
+
+  Stage: Stage-0
+    Fetch Operator
+      limit: -1
+      Processor Tree:
+        ListSink
+
+PREHOOK: query: EXPLAIN VECTORIZATION DETAIL
+SELECT * FROM nested_struct_array_table
+PREHOOK: type: QUERY
+POSTHOOK: query: EXPLAIN VECTORIZATION DETAIL
+SELECT * FROM nested_struct_array_table
+POSTHOOK: type: QUERY
+PLAN VECTORIZATION:
+  enabled: true
+  enabledConditionsMet: [hive.vectorized.execution.enabled IS true]
+
+STAGE DEPENDENCIES:
+  Stage-1 is a root stage
+  Stage-0 depends on stages: Stage-1
+
+STAGE PLANS:
+  Stage: Stage-1
+    Tez
+#### A masked pattern was here ####
+      Vertices:
+        Map 1 
+            Map Operator Tree:
+                TableScan
+                  alias: nested_struct_array_table
+                  Statistics: Num rows: 1 Data size: 368 Basic stats: COMPLETE Column stats: NONE
+                  Select Operator
+                    expressions: nested_struct_array (type: struct<s:string,i:bigint,a:array<int>>)
+                    outputColumnNames: _col0
+                    Statistics: Num rows: 1 Data size: 368 Basic stats: COMPLETE Column stats: NONE
+                    File Output Operator
+                      compressed: false
+                      Statistics: Num rows: 1 Data size: 368 Basic stats: COMPLETE Column stats: NONE
+                      table:
+                          input format: org.apache.hadoop.mapred.SequenceFileInputFormat
+                          output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
+                          serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
+            Execution mode: llap
+            LLAP IO: all inputs (cache only)
+            Map Vectorization:
+                enabled: false
+                enabledConditionsNotMet: Row deserialization of vectorized input format not supported IS false, hive.vectorized.use.vectorized.input.format IS true AND hive.vectorized.input.format.excludes NOT CONTAINS org.apache.hadoop.hive.ql.io.parquet.MapredParquetInputFormat IS false
+                inputFileFormats: org.apache.hadoop.hive.ql.io.parquet.MapredParquetInputFormat
+
+  Stage: Stage-0
+    Fetch Operator
+      limit: -1
+      Processor Tree:
+        ListSink
+
+PREHOOK: query: EXPLAIN VECTORIZATION DETAIL
+SELECT * FROM nested_struct_map_table
+PREHOOK: type: QUERY
+POSTHOOK: query: EXPLAIN VECTORIZATION DETAIL
+SELECT * FROM nested_struct_map_table
+POSTHOOK: type: QUERY
+PLAN VECTORIZATION:
+  enabled: true
+  enabledConditionsMet: [hive.vectorized.execution.enabled IS true]
+
+STAGE DEPENDENCIES:
+  Stage-1 is a root stage
+  Stage-0 depends on stages: Stage-1
+
+STAGE PLANS:
+  Stage: Stage-1
+    Tez
+#### A masked pattern was here ####
+      Vertices:
+        Map 1 
+            Map Operator Tree:
+                TableScan
+                  alias: nested_struct_map_table
+                  Statistics: Num rows: 1 Data size: 1168 Basic stats: COMPLETE Column stats: NONE
+                  Select Operator
+                    expressions: nested_struct_map (type: struct<s:string,i:bigint,m:map<string,string>>)
+                    outputColumnNames: _col0
+                    Statistics: Num rows: 1 Data size: 1168 Basic stats: COMPLETE Column stats: NONE
+                    File Output Operator
+                      compressed: false
+                      Statistics: Num rows: 1 Data size: 1168 Basic stats: COMPLETE Column stats: NONE
+                      table:
+                          input format: org.apache.hadoop.mapred.SequenceFileInputFormat
+                          output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
+                          serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
+            Execution mode: llap
+            LLAP IO: all inputs (cache only)
+            Map Vectorization:
+                enabled: false
+                enabledConditionsNotMet: Row deserialization of vectorized input format not supported IS false, hive.vectorized.use.vectorized.input.format IS true AND hive.vectorized.input.format.excludes NOT CONTAINS org.apache.hadoop.hive.ql.io.parquet.MapredParquetInputFormat IS false
+                inputFileFormats: org.apache.hadoop.hive.ql.io.parquet.MapredParquetInputFormat
+
+  Stage: Stage-0
+    Fetch Operator
+      limit: -1
+      Processor Tree:
+        ListSink
+
+PREHOOK: query: EXPLAIN VECTORIZATION DETAIL
+SELECT * FROM nested_struct_struct_table
+PREHOOK: type: QUERY
+POSTHOOK: query: EXPLAIN VECTORIZATION DETAIL
+SELECT * FROM nested_struct_struct_table
+POSTHOOK: type: QUERY
+PLAN VECTORIZATION:
+  enabled: true
+  enabledConditionsMet: [hive.vectorized.execution.enabled IS true]
+
+STAGE DEPENDENCIES:
+  Stage-1 is a root stage
+  Stage-0 depends on stages: Stage-1
+
+STAGE PLANS:
+  Stage: Stage-1
+    Tez
+#### A masked pattern was here ####
+      Vertices:
+        Map 1 
+            Map Operator Tree:
+                TableScan
+                  alias: nested_struct_struct_table
+                  Statistics: Num rows: 1 Data size: 312 Basic stats: COMPLETE Column stats: NONE
+                  Select Operator
+                    expressions: nested_struct_struct (type: struct<s:string,i:bigint,s2:struct<latitude:double,longitude:double>>)
+                    outputColumnNames: _col0
+                    Statistics: Num rows: 1 Data size: 312 Basic stats: COMPLETE Column stats: NONE
+                    File Output Operator
+                      compressed: false
+                      Statistics: Num rows: 1 Data size: 312 Basic stats: COMPLETE Column stats: NONE
+                      table:
+                          input format: org.apache.hadoop.mapred.SequenceFileInputFormat
+                          output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
+                          serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
+            Execution mode: llap
+            LLAP IO: all inputs (cache only)
+            Map Vectorization:
+                enabled: false
+                enabledConditionsNotMet: Row deserialization of vectorized input format not supported IS false, hive.vectorized.use.vectorized.input.format IS true AND hive.vectorized.input.format.excludes NOT CONTAINS org.apache.hadoop.hive.ql.io.parquet.MapredParquetInputFormat IS false
+                inputFileFormats: org.apache.hadoop.hive.ql.io.parquet.MapredParquetInputFormat
+
+  Stage: Stage-0
+    Fetch Operator
+      limit: -1
+      Processor Tree:
+        ListSink
+