You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hive.apache.org by ha...@apache.org on 2018/10/01 00:53:42 UTC

hive git commit: HIVE-20637 : Allow any udfs with 0 arguments or with constant arguments as part of default clause (Miklos Gergely via Ashutosh Chauhan)

Repository: hive
Updated Branches:
  refs/heads/master e133ec5c2 -> 62a51b745


HIVE-20637 : Allow any udfs with 0 arguments or with constant arguments as part of default clause (Miklos Gergely via Ashutosh Chauhan)

Signed-off-by: Ashutosh Chauhan <ha...@apache.org>


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

Branch: refs/heads/master
Commit: 62a51b7451af98eb279ae948046ed5d340233e8e
Parents: e133ec5
Author: Miklos Gergely <mg...@hortonworks.com>
Authored: Sun Sep 30 17:52:54 2018 -0700
Committer: Ashutosh Chauhan <ha...@apache.org>
Committed: Sun Sep 30 17:52:54 2018 -0700

----------------------------------------------------------------------
 .../hive/ql/parse/BaseSemanticAnalyzer.java     |  60 ++------
 .../hive/ql/parse/DDLSemanticAnalyzer.java      |   8 +-
 .../hadoop/hive/ql/parse/SemanticAnalyzer.java  |   4 +-
 .../default_constraint_invalid_default_value2.q |   2 -
 ...ault_constraint_invalid_default_value_type.q |   2 -
 .../insert_into_default_keyword.q               |  10 +-
 ...ault_constraint_invalid_default_value2.q.out |   1 -
 ..._constraint_invalid_default_value_type.q.out |   1 -
 .../llap/insert_into_default_keyword.q.out      | 149 +++++++++++++++++++
 9 files changed, 180 insertions(+), 57 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hive/blob/62a51b74/ql/src/java/org/apache/hadoop/hive/ql/parse/BaseSemanticAnalyzer.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/parse/BaseSemanticAnalyzer.java b/ql/src/java/org/apache/hadoop/hive/ql/parse/BaseSemanticAnalyzer.java
index b655ab1..1df5c74 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/parse/BaseSemanticAnalyzer.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/parse/BaseSemanticAnalyzer.java
@@ -90,11 +90,6 @@ import org.apache.hadoop.hive.ql.plan.PlanUtils;
 import org.apache.hadoop.hive.ql.plan.TableDesc;
 import org.apache.hadoop.hive.ql.session.SessionState;
 import org.apache.hadoop.hive.ql.session.SessionState.LogHelper;
-import org.apache.hadoop.hive.ql.udf.generic.GenericUDFCurrentDate;
-import org.apache.hadoop.hive.ql.udf.generic.GenericUDFCurrentTimestamp;
-import org.apache.hadoop.hive.ql.udf.generic.GenericUDFCurrentUser;
-import org.apache.hadoop.hive.ql.udf.generic.GenericUDFOPNull;
-import org.apache.hadoop.hive.ql.udf.generic.GenericUDFSurrogateKey;
 import org.apache.hadoop.hive.serde.serdeConstants;
 import org.apache.hadoop.hive.serde2.io.DateWritableV2;
 import org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe;
@@ -672,15 +667,6 @@ public abstract class BaseSemanticAnalyzer {
     final String defaultValue;
 
     ConstraintInfo(String colName, String constraintName,
-        boolean enable, boolean validate, boolean rely) {
-      this.colName = colName;
-      this.constraintName = constraintName;
-      this.enable = enable;
-      this.validate = validate;
-      this.rely = rely;
-      this.defaultValue = null;
-    }
-    ConstraintInfo(String colName, String constraintName,
                    boolean enable, boolean validate, boolean rely, String defaultValue) {
       this.colName = colName;
       this.constraintName = constraintName;
@@ -821,27 +807,27 @@ public abstract class BaseSemanticAnalyzer {
     generateConstraintInfos(child, columnNames.build(), cstrInfos, null, null);
   }
 
-  private static boolean isDefaultValueAllowed(final ExprNodeDesc defaultValExpr) {
+  private static boolean isDefaultValueAllowed(ExprNodeDesc defaultValExpr) {
+    while (FunctionRegistry.isOpCast(defaultValExpr)) {
+      defaultValExpr = defaultValExpr.getChildren().get(0);
+    }
+
     if(defaultValExpr instanceof ExprNodeConstantDesc) {
       return true;
     }
-    else if(FunctionRegistry.isOpCast(defaultValExpr)) {
-      return isDefaultValueAllowed(defaultValExpr.getChildren().get(0));
-    }
-    else if(defaultValExpr instanceof ExprNodeGenericFuncDesc){
-      ExprNodeGenericFuncDesc defFunc = (ExprNodeGenericFuncDesc)defaultValExpr;
-      if(defFunc.getGenericUDF() instanceof GenericUDFOPNull
-          || defFunc.getGenericUDF() instanceof GenericUDFCurrentTimestamp
-          || defFunc.getGenericUDF() instanceof GenericUDFCurrentDate
-          || defFunc.getGenericUDF() instanceof GenericUDFCurrentUser
-          || defFunc.getGenericUDF() instanceof GenericUDFSurrogateKey){
-        return true;
+
+    if(defaultValExpr instanceof ExprNodeGenericFuncDesc){
+      for (ExprNodeDesc argument : defaultValExpr.getChildren()) {
+        if (!isDefaultValueAllowed(argument)) {
+          return false;
+        }
       }
+      return true;
     }
+
     return false;
   }
 
-
   // given an ast node this method recursively goes over checkExpr ast. If it finds a node of type TOK_SUBQUERY_EXPR
   // it throws an error.
   // This method is used to validate check expression since check expression isn't allowed to have subquery
@@ -1833,18 +1819,9 @@ public abstract class BaseSemanticAnalyzer {
     return transactionalInQuery;
   }
 
-  /**
-   * Construct list bucketing context.
-   *
-   * @param skewedColNames
-   * @param skewedValues
-   * @param skewedColValueLocationMaps
-   * @param isStoredAsSubDirectories
-   * @return
-   */
   protected ListBucketingCtx constructListBucketingCtx(List<String> skewedColNames,
       List<List<String>> skewedValues, Map<List<String>, String> skewedColValueLocationMaps,
-      boolean isStoredAsSubDirectories, HiveConf conf) {
+      boolean isStoredAsSubDirectories) {
     ListBucketingCtx lbCtx = new ListBucketingCtx();
     lbCtx.setSkewedColNames(skewedColNames);
     lbCtx.setSkewedColValues(skewedValues);
@@ -2086,7 +2063,7 @@ public abstract class BaseSemanticAnalyzer {
     }
     String normalizedColSpec = originalColSpec;
     if (colType.equals(serdeConstants.DATE_TYPE_NAME)) {
-      normalizedColSpec = normalizeDateCol(colValue, originalColSpec);
+      normalizedColSpec = normalizeDateCol(colValue);
     }
     if (!normalizedColSpec.equals(originalColSpec)) {
       STATIC_LOG.warn("Normalizing partition spec - " + colName + " from "
@@ -2095,8 +2072,7 @@ public abstract class BaseSemanticAnalyzer {
     }
   }
 
-  private static String normalizeDateCol(
-      Object colValue, String originalColSpec) throws SemanticException {
+  private static String normalizeDateCol(Object colValue) throws SemanticException {
     Date value;
     if (colValue instanceof DateWritableV2) {
       value = ((DateWritableV2) colValue).get(); // Time doesn't matter.
@@ -2147,10 +2123,6 @@ public abstract class BaseSemanticAnalyzer {
     }
   }
 
-  private Path tryQualifyPath(Path path) throws IOException {
-    return tryQualifyPath(path,conf);
-  }
-
   public static Path tryQualifyPath(Path path, HiveConf conf) throws IOException {
     try {
       return path.getFileSystem(conf).makeQualified(path);

http://git-wip-us.apache.org/repos/asf/hive/blob/62a51b74/ql/src/java/org/apache/hadoop/hive/ql/parse/DDLSemanticAnalyzer.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/parse/DDLSemanticAnalyzer.java b/ql/src/java/org/apache/hadoop/hive/ql/parse/DDLSemanticAnalyzer.java
index 412fca2..29f6ecf 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/parse/DDLSemanticAnalyzer.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/parse/DDLSemanticAnalyzer.java
@@ -1548,7 +1548,7 @@ public class DDLSemanticAnalyzer extends BaseSemanticAnalyzer {
           inputFormatClass = part.getInputFormatClass();
           isArchived = ArchiveUtils.isArchived(part);
           lbCtx = constructListBucketingCtx(part.getSkewedColNames(), part.getSkewedColValues(),
-              part.getSkewedColValueLocationMaps(), part.isStoredAsSubDirectories(), conf);
+              part.getSkewedColValueLocationMaps(), part.isStoredAsSubDirectories());
           isListBucketed = part.isStoredAsSubDirectories();
           listBucketColNames = part.getSkewedColNames();
         } else {
@@ -1559,7 +1559,7 @@ public class DDLSemanticAnalyzer extends BaseSemanticAnalyzer {
           bucketCols = table.getBucketCols();
           inputFormatClass = table.getInputFormatClass();
           lbCtx = constructListBucketingCtx(table.getSkewedColNames(), table.getSkewedColValues(),
-              table.getSkewedColValueLocationMaps(), table.isStoredAsSubDirectories(), conf);
+              table.getSkewedColValueLocationMaps(), table.isStoredAsSubDirectories());
           isListBucketed = table.isStoredAsSubDirectories();
           listBucketColNames = table.getSkewedColNames();
         }
@@ -2074,7 +2074,7 @@ public class DDLSemanticAnalyzer extends BaseSemanticAnalyzer {
           oldTblPartLoc = partPath;
 
           lbCtx = constructListBucketingCtx(part.getSkewedColNames(), part.getSkewedColValues(),
-              part.getSkewedColValueLocationMaps(), part.isStoredAsSubDirectories(), conf);
+              part.getSkewedColValueLocationMaps(), part.isStoredAsSubDirectories());
         }
       } else {
         inputFormatClass = tblObj.getInputFormatClass();
@@ -2085,7 +2085,7 @@ public class DDLSemanticAnalyzer extends BaseSemanticAnalyzer {
         newTblPartLoc = tblObj.getPath();
 
         lbCtx = constructListBucketingCtx(tblObj.getSkewedColNames(), tblObj.getSkewedColValues(),
-            tblObj.getSkewedColValueLocationMaps(), tblObj.isStoredAsSubDirectories(), conf);
+            tblObj.getSkewedColValueLocationMaps(), tblObj.isStoredAsSubDirectories());
       }
 
       // throw a HiveException for other than rcfile and orcfile.

http://git-wip-us.apache.org/repos/asf/hive/blob/62a51b74/ql/src/java/org/apache/hadoop/hive/ql/parse/SemanticAnalyzer.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/parse/SemanticAnalyzer.java b/ql/src/java/org/apache/hadoop/hive/ql/parse/SemanticAnalyzer.java
index 54b9c67..e81ed50 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/parse/SemanticAnalyzer.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/parse/SemanticAnalyzer.java
@@ -7262,7 +7262,7 @@ public class SemanticAnalyzer extends BaseSemanticAnalyzer {
 
       lbCtx = constructListBucketingCtx(destinationTable.getSkewedColNames(),
           destinationTable.getSkewedColValues(), destinationTable.getSkewedColValueLocationMaps(),
-          destinationTable.isStoredAsSubDirectories(), conf);
+          destinationTable.isStoredAsSubDirectories());
 
       // Create the work for moving the table
       // NOTE: specify Dynamic partitions in dest_tab for WriteEntity
@@ -7379,7 +7379,7 @@ public class SemanticAnalyzer extends BaseSemanticAnalyzer {
 
       lbCtx = constructListBucketingCtx(destinationPartition.getSkewedColNames(),
           destinationPartition.getSkewedColValues(), destinationPartition.getSkewedColValueLocationMaps(),
-          destinationPartition.isStoredAsSubDirectories(), conf);
+          destinationPartition.isStoredAsSubDirectories());
       AcidUtils.Operation acidOp = AcidUtils.Operation.NOT_ACID;
       if (destTableIsFullAcid) {
         acidOp = getAcidType(tableDescriptor.getOutputFileFormatClass(), dest);

http://git-wip-us.apache.org/repos/asf/hive/blob/62a51b74/ql/src/test/queries/clientnegative/default_constraint_invalid_default_value2.q
----------------------------------------------------------------------
diff --git a/ql/src/test/queries/clientnegative/default_constraint_invalid_default_value2.q b/ql/src/test/queries/clientnegative/default_constraint_invalid_default_value2.q
deleted file mode 100644
index ec5b67a..0000000
--- a/ql/src/test/queries/clientnegative/default_constraint_invalid_default_value2.q
+++ /dev/null
@@ -1,2 +0,0 @@
--- only certain UDFs are allowed as default
-create table t (i int, j string default repeat('s', 4));

http://git-wip-us.apache.org/repos/asf/hive/blob/62a51b74/ql/src/test/queries/clientnegative/default_constraint_invalid_default_value_type.q
----------------------------------------------------------------------
diff --git a/ql/src/test/queries/clientnegative/default_constraint_invalid_default_value_type.q b/ql/src/test/queries/clientnegative/default_constraint_invalid_default_value_type.q
deleted file mode 100644
index 1f1a9db..0000000
--- a/ql/src/test/queries/clientnegative/default_constraint_invalid_default_value_type.q
+++ /dev/null
@@ -1,2 +0,0 @@
--- year() isn't valid
-create table t (i int, j string default cast(year("1970-01-01") as string));

http://git-wip-us.apache.org/repos/asf/hive/blob/62a51b74/ql/src/test/queries/clientpositive/insert_into_default_keyword.q
----------------------------------------------------------------------
diff --git a/ql/src/test/queries/clientpositive/insert_into_default_keyword.q b/ql/src/test/queries/clientpositive/insert_into_default_keyword.q
index ebef1a4..fa02ca1 100644
--- a/ql/src/test/queries/clientpositive/insert_into_default_keyword.q
+++ b/ql/src/test/queries/clientpositive/insert_into_default_keyword.q
@@ -46,6 +46,14 @@ INSERT INTO TABLE insert_into1_n0 values(default, DEFAULT);
 SELECT * from insert_into1_n0;
 TRUNCATE table insert_into1_n0;
 
+-- with default complex constraint
+CREATE TABLE insert_into1_n1 (key int, value string DEFAULT cast(round(round(1.245, 2), 1) as string))
+     clustered by (key) into 2 buckets stored as orc TBLPROPERTIES ('transactional'='true');
+EXPLAIN INSERT INTO TABLE insert_into1_n1 values(default, DEFAULT);
+INSERT INTO TABLE insert_into1_n1 values(default, DEFAULT);
+SELECT * from insert_into1_n1;
+TRUNCATE table insert_into1_n1;
+
 -- should be able to use any case for DEFAULT
 EXPLAIN INSERT INTO TABLE insert_into1_n0 values(234, dEfAULt);
 INSERT INTO TABLE insert_into1_n0 values(234, dEfAULt);
@@ -162,4 +170,4 @@ select * from acidTable2;
 
 DROP TABLE acidTable;
 DROP TABLE acidTable2;
-DROP TABLE nonacid_n1;
\ No newline at end of file
+DROP TABLE nonacid_n1;

http://git-wip-us.apache.org/repos/asf/hive/blob/62a51b74/ql/src/test/results/clientnegative/default_constraint_invalid_default_value2.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientnegative/default_constraint_invalid_default_value2.q.out b/ql/src/test/results/clientnegative/default_constraint_invalid_default_value2.q.out
deleted file mode 100644
index 76e5aeb..0000000
--- a/ql/src/test/results/clientnegative/default_constraint_invalid_default_value2.q.out
+++ /dev/null
@@ -1 +0,0 @@
-FAILED: SemanticException [Error 10326]: Invalid Constraint syntax Invalid Default value: repeat('s', 4). DEFAULT only allows constant or function expressions

http://git-wip-us.apache.org/repos/asf/hive/blob/62a51b74/ql/src/test/results/clientnegative/default_constraint_invalid_default_value_type.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientnegative/default_constraint_invalid_default_value_type.q.out b/ql/src/test/results/clientnegative/default_constraint_invalid_default_value_type.q.out
deleted file mode 100644
index 61e0a2f..0000000
--- a/ql/src/test/results/clientnegative/default_constraint_invalid_default_value_type.q.out
+++ /dev/null
@@ -1 +0,0 @@
-FAILED: SemanticException [Error 10326]: Invalid Constraint syntax Invalid Default value: cast(year("1970-01-01") as string). DEFAULT only allows constant or function expressions

http://git-wip-us.apache.org/repos/asf/hive/blob/62a51b74/ql/src/test/results/clientpositive/llap/insert_into_default_keyword.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/llap/insert_into_default_keyword.q.out b/ql/src/test/results/clientpositive/llap/insert_into_default_keyword.q.out
index 4f4d1b1..f7f1882 100644
--- a/ql/src/test/results/clientpositive/llap/insert_into_default_keyword.q.out
+++ b/ql/src/test/results/clientpositive/llap/insert_into_default_keyword.q.out
@@ -865,6 +865,155 @@ PREHOOK: Output: default@insert_into1_n0
 POSTHOOK: query: TRUNCATE table insert_into1_n0
 POSTHOOK: type: TRUNCATETABLE
 POSTHOOK: Output: default@insert_into1_n0
+PREHOOK: query: CREATE TABLE insert_into1_n1 (key int, value string DEFAULT cast(round(round(1.245, 2), 1) as string))
+     clustered by (key) into 2 buckets stored as orc TBLPROPERTIES ('transactional'='true')
+PREHOOK: type: CREATETABLE
+PREHOOK: Output: database:default
+PREHOOK: Output: default@insert_into1_n1
+POSTHOOK: query: CREATE TABLE insert_into1_n1 (key int, value string DEFAULT cast(round(round(1.245, 2), 1) as string))
+     clustered by (key) into 2 buckets stored as orc TBLPROPERTIES ('transactional'='true')
+POSTHOOK: type: CREATETABLE
+POSTHOOK: Output: database:default
+POSTHOOK: Output: default@insert_into1_n1
+PREHOOK: query: EXPLAIN INSERT INTO TABLE insert_into1_n1 values(default, DEFAULT)
+PREHOOK: type: QUERY
+PREHOOK: Input: _dummy_database@_dummy_table
+PREHOOK: Output: default@insert_into1_n1
+POSTHOOK: query: EXPLAIN INSERT INTO TABLE insert_into1_n1 values(default, DEFAULT)
+POSTHOOK: type: QUERY
+POSTHOOK: Input: _dummy_database@_dummy_table
+POSTHOOK: Output: default@insert_into1_n1
+STAGE DEPENDENCIES:
+  Stage-1 is a root stage
+  Stage-2 depends on stages: Stage-1
+  Stage-0 depends on stages: Stage-2
+  Stage-3 depends on stages: Stage-0
+
+STAGE PLANS:
+  Stage: Stage-1
+    Tez
+#### A masked pattern was here ####
+      Edges:
+        Reducer 2 <- Map 1 (CUSTOM_SIMPLE_EDGE)
+        Reducer 3 <- Reducer 2 (CUSTOM_SIMPLE_EDGE)
+#### A masked pattern was here ####
+      Vertices:
+        Map 1 
+            Map Operator Tree:
+                TableScan
+                  alias: _dummy_table
+                  Row Limit Per Split: 1
+                  Statistics: Num rows: 1 Data size: 10 Basic stats: COMPLETE Column stats: COMPLETE
+                  Select Operator
+                    expressions: array(const struct(null,'1.3')) (type: array<struct<col1:void,col2:string>>)
+                    outputColumnNames: _col0
+                    Statistics: Num rows: 1 Data size: 48 Basic stats: COMPLETE Column stats: COMPLETE
+                    UDTF Operator
+                      Statistics: Num rows: 1 Data size: 48 Basic stats: COMPLETE Column stats: COMPLETE
+                      function name: inline
+                      Select Operator
+                        expressions: null (type: void), col2 (type: string)
+                        outputColumnNames: _col0, _col1
+                        Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: COMPLETE
+                        Reduce Output Operator
+                          sort order: 
+                          Map-reduce partition columns: UDFToInteger(_col0) (type: int)
+                          Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: COMPLETE
+                          value expressions: _col0 (type: void), _col1 (type: string)
+            Execution mode: llap
+            LLAP IO: no inputs
+        Reducer 2 
+            Execution mode: llap
+            Reduce Operator Tree:
+              Select Operator
+                expressions: UDFToInteger(VALUE._col0) (type: int), VALUE._col1 (type: string)
+                outputColumnNames: _col0, _col1
+                Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: COMPLETE
+                File Output Operator
+                  compressed: false
+                  Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: COMPLETE
+                  table:
+                      input format: org.apache.hadoop.hive.ql.io.orc.OrcInputFormat
+                      output format: org.apache.hadoop.hive.ql.io.orc.OrcOutputFormat
+                      serde: org.apache.hadoop.hive.ql.io.orc.OrcSerde
+                      name: default.insert_into1_n1
+                  Write Type: INSERT
+                Select Operator
+                  expressions: _col0 (type: int), _col1 (type: string)
+                  outputColumnNames: key, value
+                  Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: COMPLETE
+                  Group By Operator
+                    aggregations: compute_stats(key, 'hll'), compute_stats(value, 'hll')
+                    mode: hash
+                    outputColumnNames: _col0, _col1
+                    Statistics: Num rows: 1 Data size: 864 Basic stats: COMPLETE Column stats: COMPLETE
+                    Reduce Output Operator
+                      sort order: 
+                      Statistics: Num rows: 1 Data size: 864 Basic stats: COMPLETE Column stats: COMPLETE
+                      value expressions: _col0 (type: struct<columntype:string,min:bigint,max:bigint,countnulls:bigint,bitvector:binary>), _col1 (type: struct<columntype:string,maxlength:bigint,sumlength:bigint,count:bigint,countnulls:bigint,bitvector:binary>)
+        Reducer 3 
+            Execution mode: llap
+            Reduce Operator Tree:
+              Group By Operator
+                aggregations: compute_stats(VALUE._col0), compute_stats(VALUE._col1)
+                mode: mergepartial
+                outputColumnNames: _col0, _col1
+                Statistics: Num rows: 1 Data size: 880 Basic stats: COMPLETE Column stats: COMPLETE
+                File Output Operator
+                  compressed: false
+                  Statistics: Num rows: 1 Data size: 880 Basic stats: COMPLETE Column stats: COMPLETE
+                  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
+
+  Stage: Stage-2
+    Dependency Collection
+
+  Stage: Stage-0
+    Move Operator
+      tables:
+          replace: false
+          table:
+              input format: org.apache.hadoop.hive.ql.io.orc.OrcInputFormat
+              output format: org.apache.hadoop.hive.ql.io.orc.OrcOutputFormat
+              serde: org.apache.hadoop.hive.ql.io.orc.OrcSerde
+              name: default.insert_into1_n1
+          Write Type: INSERT
+
+  Stage: Stage-3
+    Stats Work
+      Basic Stats Work:
+      Column Stats Desc:
+          Columns: key, value
+          Column Types: int, string
+          Table: default.insert_into1_n1
+
+PREHOOK: query: INSERT INTO TABLE insert_into1_n1 values(default, DEFAULT)
+PREHOOK: type: QUERY
+PREHOOK: Input: _dummy_database@_dummy_table
+PREHOOK: Output: default@insert_into1_n1
+POSTHOOK: query: INSERT INTO TABLE insert_into1_n1 values(default, DEFAULT)
+POSTHOOK: type: QUERY
+POSTHOOK: Input: _dummy_database@_dummy_table
+POSTHOOK: Output: default@insert_into1_n1
+POSTHOOK: Lineage: insert_into1_n1.key EXPRESSION []
+POSTHOOK: Lineage: insert_into1_n1.value SCRIPT []
+PREHOOK: query: SELECT * from insert_into1_n1
+PREHOOK: type: QUERY
+PREHOOK: Input: default@insert_into1_n1
+#### A masked pattern was here ####
+POSTHOOK: query: SELECT * from insert_into1_n1
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@insert_into1_n1
+#### A masked pattern was here ####
+NULL	1.3
+PREHOOK: query: TRUNCATE table insert_into1_n1
+PREHOOK: type: TRUNCATETABLE
+PREHOOK: Output: default@insert_into1_n1
+POSTHOOK: query: TRUNCATE table insert_into1_n1
+POSTHOOK: type: TRUNCATETABLE
+POSTHOOK: Output: default@insert_into1_n1
 PREHOOK: query: EXPLAIN INSERT INTO TABLE insert_into1_n0 values(234, dEfAULt)
 PREHOOK: type: QUERY
 PREHOOK: Input: _dummy_database@_dummy_table