You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hive.apache.org by xu...@apache.org on 2015/05/29 06:47:56 UTC

[77/84] [abbrv] hive git commit: HIVE-10840: NumberFormatException while running analyze table partition compute statics query (Ashutosh Chauhan via Gunther Hagleitner)

HIVE-10840: NumberFormatException while running analyze table partition compute statics query (Ashutosh Chauhan via Gunther Hagleitner)


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

Branch: refs/heads/spark
Commit: 4bc4814a6f00adc0cfffa5525e78edde76a7465f
Parents: 49be246
Author: Gunther Hagleitner <gu...@apache.org>
Authored: Thu May 28 11:30:07 2015 -0700
Committer: Gunther Hagleitner <gu...@apache.org>
Committed: Thu May 28 11:30:17 2015 -0700

----------------------------------------------------------------------
 .../test/resources/testconfiguration.properties |   1 +
 .../apache/hadoop/hive/ql/stats/StatsUtils.java |  44 +-
 .../queries/clientpositive/stats_only_null.q    |   3 +
 .../clientpositive/tez/stats_only_null.q.out    | 422 +++++++++++++++++++
 4 files changed, 458 insertions(+), 12 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hive/blob/4bc4814a/itests/src/test/resources/testconfiguration.properties
----------------------------------------------------------------------
diff --git a/itests/src/test/resources/testconfiguration.properties b/itests/src/test/resources/testconfiguration.properties
index ae03283..376c7af 100644
--- a/itests/src/test/resources/testconfiguration.properties
+++ b/itests/src/test/resources/testconfiguration.properties
@@ -156,6 +156,7 @@ minitez.query.files.shared=alter_merge_2_orc.q,\
   stats_counter.q,\
   stats_counter_partitioned.q,\
   stats_noscan_1.q,\
+  stats_only_null.q,\
   subquery_exists.q,\
   subquery_in.q,\
   temp_table.q,\

http://git-wip-us.apache.org/repos/asf/hive/blob/4bc4814a/ql/src/java/org/apache/hadoop/hive/ql/stats/StatsUtils.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/stats/StatsUtils.java b/ql/src/java/org/apache/hadoop/hive/ql/stats/StatsUtils.java
index ad481bc..6615a28 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/stats/StatsUtils.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/stats/StatsUtils.java
@@ -30,6 +30,7 @@ import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hive.common.StatsSetupConst;
 import org.apache.hadoop.hive.common.type.HiveDecimal;
 import org.apache.hadoop.hive.conf.HiveConf;
+import org.apache.hadoop.hive.conf.HiveConf.ConfVars;
 import org.apache.hadoop.hive.metastore.api.AggrStats;
 import org.apache.hadoop.hive.metastore.api.ColumnStatisticsData;
 import org.apache.hadoop.hive.metastore.api.ColumnStatisticsObj;
@@ -400,7 +401,7 @@ public class StatsUtils {
             partCS.setAvgColLen(StatsUtils.getAvgColLenOfVariableLengthTypes(conf,
                 ci.getObjectInspector(), partCS.getColumnType()));
             partCS.setRange(getRangePartitionColumn(partList.getPartitions(), ci.getInternalName(),
-                ci.getType().getTypeName()));
+                ci.getType().getTypeName(), conf.getVar(ConfVars.DEFAULTPARTITIONNAME)));
             colStats.add(partCS);
           }
         }
@@ -416,9 +417,10 @@ public class StatsUtils {
     return distinctVals.size();
   }
 
-  public static Range getRangePartitionColumn(Set<Partition> partitions, String partColName,
-      String colType) {
+  private static Range getRangePartitionColumn(Set<Partition> partitions, String partColName,
+      String colType, String defaultPartName) {
     Range range = null;
+    String partVal;
     if (colType.equalsIgnoreCase(serdeConstants.TINYINT_TYPE_NAME)
         || colType.equalsIgnoreCase(serdeConstants.SMALLINT_TYPE_NAME)
         || colType.equalsIgnoreCase(serdeConstants.INT_TYPE_NAME)
@@ -426,9 +428,15 @@ public class StatsUtils {
       long min = Long.MAX_VALUE;
       long max = Long.MIN_VALUE;
       for (Partition partition : partitions) {
-        long value = Long.parseLong(partition.getSpec().get(partColName));
-        min = Math.min(min, value);
-        max = Math.max(max, value);
+        partVal = partition.getSpec().get(partColName);
+        if (partVal.equals(defaultPartName)) {
+          // partition column value is null.
+          continue;
+        } else {
+          long value = Long.parseLong(partVal);
+          min = Math.min(min, value);
+          max = Math.max(max, value);
+        }
       }
       range = new Range(min, max);
     } else if (colType.equalsIgnoreCase(serdeConstants.FLOAT_TYPE_NAME)
@@ -436,18 +444,30 @@ public class StatsUtils {
       double min = Double.MAX_VALUE;
       double max = Double.MIN_VALUE;
       for (Partition partition : partitions) {
-        double value = Double.parseDouble(partition.getSpec().get(partColName));
-        min = Math.min(min, value);
-        max = Math.max(max, value);
+        partVal = partition.getSpec().get(partColName);
+        if (partVal.equals(defaultPartName)) {
+          // partition column value is null.
+          continue;
+        } else {
+          double value = Double.parseDouble(partVal);
+          min = Math.min(min, value);
+          max = Math.max(max, value);
+        }
       }
       range = new Range(min, max);
     } else if (colType.startsWith(serdeConstants.DECIMAL_TYPE_NAME)) {
       double min = Double.MAX_VALUE;
       double max = Double.MIN_VALUE;
       for (Partition partition : partitions) {
-        double value = new BigDecimal(partition.getSpec().get(partColName)).doubleValue();
-        min = Math.min(min, value);
-        max = Math.max(max, value);
+        partVal = partition.getSpec().get(partColName);
+        if (partVal.equals(defaultPartName)) {
+          // partition column value is null.
+          continue;
+        } else {
+          double value = new BigDecimal(partVal).doubleValue();
+          min = Math.min(min, value);
+          max = Math.max(max, value);
+        }
       }
       range = new Range(min, max);
     } else {

http://git-wip-us.apache.org/repos/asf/hive/blob/4bc4814a/ql/src/test/queries/clientpositive/stats_only_null.q
----------------------------------------------------------------------
diff --git a/ql/src/test/queries/clientpositive/stats_only_null.q b/ql/src/test/queries/clientpositive/stats_only_null.q
index a91022c..5f89499 100644
--- a/ql/src/test/queries/clientpositive/stats_only_null.q
+++ b/ql/src/test/queries/clientpositive/stats_only_null.q
@@ -1,6 +1,7 @@
 set hive.stats.dbclass=fs;
 set hive.compute.query.using.stats=true;
 set hive.stats.autogather=true;
+set hive.stats.fetch.column.stats=true;
 CREATE TABLE temps_null(a double, b int, c STRING, d smallint) STORED AS TEXTFILE; 
 
 CREATE TABLE stats_null(a double, b int, c STRING, d smallint) STORED AS TEXTFILE; 
@@ -50,3 +51,5 @@ drop table stats_null_part;
 drop table temps_null;
 set hive.compute.query.using.stats=false;
 set hive.stats.dbclass=jdbc:derby;
+
+reset hive.stats.fetch.column.stats;

http://git-wip-us.apache.org/repos/asf/hive/blob/4bc4814a/ql/src/test/results/clientpositive/tez/stats_only_null.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/tez/stats_only_null.q.out b/ql/src/test/results/clientpositive/tez/stats_only_null.q.out
new file mode 100644
index 0000000..2725be0
--- /dev/null
+++ b/ql/src/test/results/clientpositive/tez/stats_only_null.q.out
@@ -0,0 +1,422 @@
+PREHOOK: query: CREATE TABLE temps_null(a double, b int, c STRING, d smallint) STORED AS TEXTFILE
+PREHOOK: type: CREATETABLE
+PREHOOK: Output: database:default
+PREHOOK: Output: default@temps_null
+POSTHOOK: query: CREATE TABLE temps_null(a double, b int, c STRING, d smallint) STORED AS TEXTFILE
+POSTHOOK: type: CREATETABLE
+POSTHOOK: Output: database:default
+POSTHOOK: Output: default@temps_null
+PREHOOK: query: CREATE TABLE stats_null(a double, b int, c STRING, d smallint) STORED AS TEXTFILE
+PREHOOK: type: CREATETABLE
+PREHOOK: Output: database:default
+PREHOOK: Output: default@stats_null
+POSTHOOK: query: CREATE TABLE stats_null(a double, b int, c STRING, d smallint) STORED AS TEXTFILE
+POSTHOOK: type: CREATETABLE
+POSTHOOK: Output: database:default
+POSTHOOK: Output: default@stats_null
+PREHOOK: query: CREATE TABLE stats_null_part(a double, b int, c STRING, d smallint) partitioned by (dt string) STORED AS TEXTFILE
+PREHOOK: type: CREATETABLE
+PREHOOK: Output: database:default
+PREHOOK: Output: default@stats_null_part
+POSTHOOK: query: CREATE TABLE stats_null_part(a double, b int, c STRING, d smallint) partitioned by (dt string) STORED AS TEXTFILE
+POSTHOOK: type: CREATETABLE
+POSTHOOK: Output: database:default
+POSTHOOK: Output: default@stats_null_part
+PREHOOK: query: LOAD DATA LOCAL INPATH '../../data/files/null.txt' INTO TABLE temps_null
+PREHOOK: type: LOAD
+#### A masked pattern was here ####
+PREHOOK: Output: default@temps_null
+POSTHOOK: query: LOAD DATA LOCAL INPATH '../../data/files/null.txt' INTO TABLE temps_null
+POSTHOOK: type: LOAD
+#### A masked pattern was here ####
+POSTHOOK: Output: default@temps_null
+PREHOOK: query: insert overwrite table stats_null select * from temps_null
+PREHOOK: type: QUERY
+PREHOOK: Input: default@temps_null
+PREHOOK: Output: default@stats_null
+POSTHOOK: query: insert overwrite table stats_null select * from temps_null
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@temps_null
+POSTHOOK: Output: default@stats_null
+POSTHOOK: Lineage: stats_null.a SIMPLE [(temps_null)temps_null.FieldSchema(name:a, type:double, comment:null), ]
+POSTHOOK: Lineage: stats_null.b SIMPLE [(temps_null)temps_null.FieldSchema(name:b, type:int, comment:null), ]
+POSTHOOK: Lineage: stats_null.c SIMPLE [(temps_null)temps_null.FieldSchema(name:c, type:string, comment:null), ]
+POSTHOOK: Lineage: stats_null.d SIMPLE [(temps_null)temps_null.FieldSchema(name:d, type:smallint, comment:null), ]
+PREHOOK: query: insert into table stats_null_part partition(dt='2010') select * from temps_null where d <=5
+PREHOOK: type: QUERY
+PREHOOK: Input: default@temps_null
+PREHOOK: Output: default@stats_null_part@dt=2010
+POSTHOOK: query: insert into table stats_null_part partition(dt='2010') select * from temps_null where d <=5
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@temps_null
+POSTHOOK: Output: default@stats_null_part@dt=2010
+POSTHOOK: Lineage: stats_null_part PARTITION(dt=2010).a SIMPLE [(temps_null)temps_null.FieldSchema(name:a, type:double, comment:null), ]
+POSTHOOK: Lineage: stats_null_part PARTITION(dt=2010).b SIMPLE [(temps_null)temps_null.FieldSchema(name:b, type:int, comment:null), ]
+POSTHOOK: Lineage: stats_null_part PARTITION(dt=2010).c SIMPLE [(temps_null)temps_null.FieldSchema(name:c, type:string, comment:null), ]
+POSTHOOK: Lineage: stats_null_part PARTITION(dt=2010).d SIMPLE [(temps_null)temps_null.FieldSchema(name:d, type:smallint, comment:null), ]
+PREHOOK: query: insert into table stats_null_part partition(dt='2011') select * from temps_null where d > 5
+PREHOOK: type: QUERY
+PREHOOK: Input: default@temps_null
+PREHOOK: Output: default@stats_null_part@dt=2011
+POSTHOOK: query: insert into table stats_null_part partition(dt='2011') select * from temps_null where d > 5
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@temps_null
+POSTHOOK: Output: default@stats_null_part@dt=2011
+POSTHOOK: Lineage: stats_null_part PARTITION(dt=2011).a SIMPLE [(temps_null)temps_null.FieldSchema(name:a, type:double, comment:null), ]
+POSTHOOK: Lineage: stats_null_part PARTITION(dt=2011).b SIMPLE [(temps_null)temps_null.FieldSchema(name:b, type:int, comment:null), ]
+POSTHOOK: Lineage: stats_null_part PARTITION(dt=2011).c SIMPLE [(temps_null)temps_null.FieldSchema(name:c, type:string, comment:null), ]
+POSTHOOK: Lineage: stats_null_part PARTITION(dt=2011).d SIMPLE [(temps_null)temps_null.FieldSchema(name:d, type:smallint, comment:null), ]
+PREHOOK: query: explain 
+select count(*), count(a), count(b), count(c), count(d) from stats_null
+PREHOOK: type: QUERY
+POSTHOOK: query: explain 
+select count(*), count(a), count(b), count(c), count(d) from stats_null
+POSTHOOK: type: QUERY
+STAGE DEPENDENCIES:
+  Stage-1 is a root stage
+  Stage-0 depends on stages: Stage-1
+
+STAGE PLANS:
+  Stage: Stage-1
+    Tez
+      Edges:
+        Reducer 2 <- Map 1 (SIMPLE_EDGE)
+#### A masked pattern was here ####
+      Vertices:
+        Map 1 
+            Map Operator Tree:
+                TableScan
+                  alias: stats_null
+                  Statistics: Num rows: 10 Data size: 120 Basic stats: COMPLETE Column stats: NONE
+                  Select Operator
+                    expressions: a (type: double), b (type: int), c (type: string), d (type: smallint)
+                    outputColumnNames: _col0, _col1, _col2, _col3
+                    Statistics: Num rows: 10 Data size: 120 Basic stats: COMPLETE Column stats: NONE
+                    Group By Operator
+                      aggregations: count(), count(_col0), count(_col1), count(_col2), count(_col3)
+                      mode: hash
+                      outputColumnNames: _col0, _col1, _col2, _col3, _col4
+                      Statistics: Num rows: 1 Data size: 40 Basic stats: COMPLETE Column stats: NONE
+                      Reduce Output Operator
+                        sort order: 
+                        Statistics: Num rows: 1 Data size: 40 Basic stats: COMPLETE Column stats: NONE
+                        value expressions: _col0 (type: bigint), _col1 (type: bigint), _col2 (type: bigint), _col3 (type: bigint), _col4 (type: bigint)
+        Reducer 2 
+            Reduce Operator Tree:
+              Group By Operator
+                aggregations: count(VALUE._col0), count(VALUE._col1), count(VALUE._col2), count(VALUE._col3), count(VALUE._col4)
+                mode: mergepartial
+                outputColumnNames: _col0, _col1, _col2, _col3, _col4
+                Statistics: Num rows: 1 Data size: 40 Basic stats: COMPLETE Column stats: NONE
+                File Output Operator
+                  compressed: false
+                  Statistics: Num rows: 1 Data size: 40 Basic stats: COMPLETE Column stats: NONE
+                  table:
+                      input format: org.apache.hadoop.mapred.TextInputFormat
+                      output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
+                      serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
+
+  Stage: Stage-0
+    Fetch Operator
+      limit: -1
+      Processor Tree:
+        ListSink
+
+PREHOOK: query: explain 
+select count(*), count(a), count(b), count(c), count(d) from stats_null_part
+PREHOOK: type: QUERY
+POSTHOOK: query: explain 
+select count(*), count(a), count(b), count(c), count(d) from stats_null_part
+POSTHOOK: type: QUERY
+STAGE DEPENDENCIES:
+  Stage-1 is a root stage
+  Stage-0 depends on stages: Stage-1
+
+STAGE PLANS:
+  Stage: Stage-1
+    Tez
+      Edges:
+        Reducer 2 <- Map 1 (SIMPLE_EDGE)
+#### A masked pattern was here ####
+      Vertices:
+        Map 1 
+            Map Operator Tree:
+                TableScan
+                  alias: stats_null_part
+                  Statistics: Num rows: 10 Data size: 120 Basic stats: COMPLETE Column stats: NONE
+                  Select Operator
+                    expressions: a (type: double), b (type: int), c (type: string), d (type: smallint)
+                    outputColumnNames: _col0, _col1, _col2, _col3
+                    Statistics: Num rows: 10 Data size: 120 Basic stats: COMPLETE Column stats: NONE
+                    Group By Operator
+                      aggregations: count(), count(_col0), count(_col1), count(_col2), count(_col3)
+                      mode: hash
+                      outputColumnNames: _col0, _col1, _col2, _col3, _col4
+                      Statistics: Num rows: 1 Data size: 40 Basic stats: COMPLETE Column stats: NONE
+                      Reduce Output Operator
+                        sort order: 
+                        Statistics: Num rows: 1 Data size: 40 Basic stats: COMPLETE Column stats: NONE
+                        value expressions: _col0 (type: bigint), _col1 (type: bigint), _col2 (type: bigint), _col3 (type: bigint), _col4 (type: bigint)
+        Reducer 2 
+            Reduce Operator Tree:
+              Group By Operator
+                aggregations: count(VALUE._col0), count(VALUE._col1), count(VALUE._col2), count(VALUE._col3), count(VALUE._col4)
+                mode: mergepartial
+                outputColumnNames: _col0, _col1, _col2, _col3, _col4
+                Statistics: Num rows: 1 Data size: 40 Basic stats: COMPLETE Column stats: NONE
+                File Output Operator
+                  compressed: false
+                  Statistics: Num rows: 1 Data size: 40 Basic stats: COMPLETE Column stats: NONE
+                  table:
+                      input format: org.apache.hadoop.mapred.TextInputFormat
+                      output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
+                      serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
+
+  Stage: Stage-0
+    Fetch Operator
+      limit: -1
+      Processor Tree:
+        ListSink
+
+PREHOOK: query: analyze table stats_null compute statistics for columns a,b,c,d
+PREHOOK: type: QUERY
+PREHOOK: Input: default@stats_null
+#### A masked pattern was here ####
+POSTHOOK: query: analyze table stats_null compute statistics for columns a,b,c,d
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@stats_null
+#### A masked pattern was here ####
+PREHOOK: query: analyze table stats_null_part partition(dt='2010') compute statistics for columns a,b,c,d
+PREHOOK: type: QUERY
+PREHOOK: Input: default@stats_null_part
+PREHOOK: Input: default@stats_null_part@dt=2010
+#### A masked pattern was here ####
+POSTHOOK: query: analyze table stats_null_part partition(dt='2010') compute statistics for columns a,b,c,d
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@stats_null_part
+POSTHOOK: Input: default@stats_null_part@dt=2010
+#### A masked pattern was here ####
+PREHOOK: query: analyze table stats_null_part partition(dt='2011') compute statistics for columns a,b,c,d
+PREHOOK: type: QUERY
+PREHOOK: Input: default@stats_null_part
+PREHOOK: Input: default@stats_null_part@dt=2011
+#### A masked pattern was here ####
+POSTHOOK: query: analyze table stats_null_part partition(dt='2011') compute statistics for columns a,b,c,d
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@stats_null_part
+POSTHOOK: Input: default@stats_null_part@dt=2011
+#### A masked pattern was here ####
+PREHOOK: query: describe formatted stats_null_part partition (dt='2010')
+PREHOOK: type: DESCTABLE
+PREHOOK: Input: default@stats_null_part
+POSTHOOK: query: describe formatted stats_null_part partition (dt='2010')
+POSTHOOK: type: DESCTABLE
+POSTHOOK: Input: default@stats_null_part
+# col_name            	data_type           	comment             
+	 	 
+a                   	double              	                    
+b                   	int                 	                    
+c                   	string              	                    
+d                   	smallint            	                    
+	 	 
+# Partition Information	 	 
+# col_name            	data_type           	comment             
+	 	 
+dt                  	string              	                    
+	 	 
+# Detailed Partition Information	 	 
+Partition Value:    	[2010]              	 
+Database:           	default             	 
+Table:              	stats_null_part     	 
+#### A masked pattern was here ####
+Protect Mode:       	None                	 
+#### A masked pattern was here ####
+Partition Parameters:	 	 
+	COLUMN_STATS_ACCURATE	true                
+	numFiles            	1                   
+	numRows             	6                   
+	rawDataSize         	71                  
+	totalSize           	77                  
+#### A masked pattern was here ####
+	 	 
+# Storage Information	 	 
+SerDe Library:      	org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe	 
+InputFormat:        	org.apache.hadoop.mapred.TextInputFormat	 
+OutputFormat:       	org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat	 
+Compressed:         	No                  	 
+Num Buckets:        	-1                  	 
+Bucket Columns:     	[]                  	 
+Sort Columns:       	[]                  	 
+Storage Desc Params:	 	 
+	serialization.format	1                   
+PREHOOK: query: describe formatted stats_null_part partition (dt='2011')
+PREHOOK: type: DESCTABLE
+PREHOOK: Input: default@stats_null_part
+POSTHOOK: query: describe formatted stats_null_part partition (dt='2011')
+POSTHOOK: type: DESCTABLE
+POSTHOOK: Input: default@stats_null_part
+# col_name            	data_type           	comment             
+	 	 
+a                   	double              	                    
+b                   	int                 	                    
+c                   	string              	                    
+d                   	smallint            	                    
+	 	 
+# Partition Information	 	 
+# col_name            	data_type           	comment             
+	 	 
+dt                  	string              	                    
+	 	 
+# Detailed Partition Information	 	 
+Partition Value:    	[2011]              	 
+Database:           	default             	 
+Table:              	stats_null_part     	 
+#### A masked pattern was here ####
+Protect Mode:       	None                	 
+#### A masked pattern was here ####
+Partition Parameters:	 	 
+	COLUMN_STATS_ACCURATE	true                
+	numFiles            	1                   
+	numRows             	4                   
+	rawDataSize         	49                  
+	totalSize           	53                  
+#### A masked pattern was here ####
+	 	 
+# Storage Information	 	 
+SerDe Library:      	org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe	 
+InputFormat:        	org.apache.hadoop.mapred.TextInputFormat	 
+OutputFormat:       	org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat	 
+Compressed:         	No                  	 
+Num Buckets:        	-1                  	 
+Bucket Columns:     	[]                  	 
+Sort Columns:       	[]                  	 
+Storage Desc Params:	 	 
+	serialization.format	1                   
+PREHOOK: query: explain 
+select count(*), count(a), count(b), count(c), count(d) from stats_null
+PREHOOK: type: QUERY
+POSTHOOK: query: explain 
+select count(*), count(a), count(b), count(c), count(d) from stats_null
+POSTHOOK: type: QUERY
+STAGE DEPENDENCIES:
+  Stage-0 is a root stage
+
+STAGE PLANS:
+  Stage: Stage-0
+    Fetch Operator
+      limit: 1
+      Processor Tree:
+        ListSink
+
+PREHOOK: query: explain 
+select count(*), count(a), count(b), count(c), count(d) from stats_null_part
+PREHOOK: type: QUERY
+POSTHOOK: query: explain 
+select count(*), count(a), count(b), count(c), count(d) from stats_null_part
+POSTHOOK: type: QUERY
+STAGE DEPENDENCIES:
+  Stage-0 is a root stage
+
+STAGE PLANS:
+  Stage: Stage-0
+    Fetch Operator
+      limit: 1
+      Processor Tree:
+        ListSink
+
+PREHOOK: query: select count(*), count(a), count(b), count(c), count(d) from stats_null
+PREHOOK: type: QUERY
+PREHOOK: Input: default@stats_null
+#### A masked pattern was here ####
+POSTHOOK: query: select count(*), count(a), count(b), count(c), count(d) from stats_null
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@stats_null
+#### A masked pattern was here ####
+10	8	8	10	10
+PREHOOK: query: select count(*), count(a), count(b), count(c), count(d) from stats_null_part
+PREHOOK: type: QUERY
+PREHOOK: Input: default@stats_null_part
+#### A masked pattern was here ####
+POSTHOOK: query: select count(*), count(a), count(b), count(c), count(d) from stats_null_part
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@stats_null_part
+#### A masked pattern was here ####
+10	8	8	10	10
+PREHOOK: query: drop table stats_null_part
+PREHOOK: type: DROPTABLE
+PREHOOK: Input: default@stats_null_part
+PREHOOK: Output: default@stats_null_part
+POSTHOOK: query: drop table stats_null_part
+POSTHOOK: type: DROPTABLE
+POSTHOOK: Input: default@stats_null_part
+POSTHOOK: Output: default@stats_null_part
+PREHOOK: query: CREATE TABLE stats_null_part(a double, b int, c STRING, d smallint) partitioned by (dt int) STORED AS TEXTFILE
+PREHOOK: type: CREATETABLE
+PREHOOK: Output: database:default
+PREHOOK: Output: default@stats_null_part
+POSTHOOK: query: CREATE TABLE stats_null_part(a double, b int, c STRING, d smallint) partitioned by (dt int) STORED AS TEXTFILE
+POSTHOOK: type: CREATETABLE
+POSTHOOK: Output: database:default
+POSTHOOK: Output: default@stats_null_part
+PREHOOK: query: insert into table stats_null_part partition(dt) select a,b,c,d,b from temps_null
+PREHOOK: type: QUERY
+PREHOOK: Input: default@temps_null
+PREHOOK: Output: default@stats_null_part
+POSTHOOK: query: insert into table stats_null_part partition(dt) select a,b,c,d,b from temps_null
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@temps_null
+POSTHOOK: Output: default@stats_null_part@dt=1
+POSTHOOK: Output: default@stats_null_part@dt=__HIVE_DEFAULT_PARTITION__
+POSTHOOK: Lineage: stats_null_part PARTITION(dt=1).a SIMPLE [(temps_null)temps_null.FieldSchema(name:a, type:double, comment:null), ]
+POSTHOOK: Lineage: stats_null_part PARTITION(dt=1).b SIMPLE [(temps_null)temps_null.FieldSchema(name:b, type:int, comment:null), ]
+POSTHOOK: Lineage: stats_null_part PARTITION(dt=1).c SIMPLE [(temps_null)temps_null.FieldSchema(name:c, type:string, comment:null), ]
+POSTHOOK: Lineage: stats_null_part PARTITION(dt=1).d SIMPLE [(temps_null)temps_null.FieldSchema(name:d, type:smallint, comment:null), ]
+POSTHOOK: Lineage: stats_null_part PARTITION(dt=__HIVE_DEFAULT_PARTITION__).a SIMPLE [(temps_null)temps_null.FieldSchema(name:a, type:double, comment:null), ]
+POSTHOOK: Lineage: stats_null_part PARTITION(dt=__HIVE_DEFAULT_PARTITION__).b SIMPLE [(temps_null)temps_null.FieldSchema(name:b, type:int, comment:null), ]
+POSTHOOK: Lineage: stats_null_part PARTITION(dt=__HIVE_DEFAULT_PARTITION__).c SIMPLE [(temps_null)temps_null.FieldSchema(name:c, type:string, comment:null), ]
+POSTHOOK: Lineage: stats_null_part PARTITION(dt=__HIVE_DEFAULT_PARTITION__).d SIMPLE [(temps_null)temps_null.FieldSchema(name:d, type:smallint, comment:null), ]
+PREHOOK: query: analyze table stats_null_part compute statistics for columns
+PREHOOK: type: QUERY
+PREHOOK: Input: default@stats_null_part
+PREHOOK: Input: default@stats_null_part@dt=1
+PREHOOK: Input: default@stats_null_part@dt=__HIVE_DEFAULT_PARTITION__
+#### A masked pattern was here ####
+POSTHOOK: query: analyze table stats_null_part compute statistics for columns
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@stats_null_part
+POSTHOOK: Input: default@stats_null_part@dt=1
+POSTHOOK: Input: default@stats_null_part@dt=__HIVE_DEFAULT_PARTITION__
+#### A masked pattern was here ####
+PREHOOK: query: describe formatted stats_null_part.a partition(dt = 1)
+PREHOOK: type: DESCTABLE
+PREHOOK: Input: default@stats_null_part
+POSTHOOK: query: describe formatted stats_null_part.a partition(dt = 1)
+POSTHOOK: type: DESCTABLE
+POSTHOOK: Input: default@stats_null_part
+# col_name            	data_type           	min                 	max                 	num_nulls           	distinct_count      	avg_col_len         	max_col_len         	num_trues           	num_falses          	comment             
+	 	 	 	 	 	 	 	 	 	 
+a                   	double              	1.0                 	1.0                 	1                   	1                   	                    	                    	                    	                    	from deserializer   
+PREHOOK: query: drop table stats_null
+PREHOOK: type: DROPTABLE
+PREHOOK: Input: default@stats_null
+PREHOOK: Output: default@stats_null
+POSTHOOK: query: drop table stats_null
+POSTHOOK: type: DROPTABLE
+POSTHOOK: Input: default@stats_null
+POSTHOOK: Output: default@stats_null
+PREHOOK: query: drop table stats_null_part
+PREHOOK: type: DROPTABLE
+PREHOOK: Input: default@stats_null_part
+PREHOOK: Output: default@stats_null_part
+POSTHOOK: query: drop table stats_null_part
+POSTHOOK: type: DROPTABLE
+POSTHOOK: Input: default@stats_null_part
+POSTHOOK: Output: default@stats_null_part
+PREHOOK: query: drop table temps_null
+PREHOOK: type: DROPTABLE
+PREHOOK: Input: default@temps_null
+PREHOOK: Output: default@temps_null
+POSTHOOK: query: drop table temps_null
+POSTHOOK: type: DROPTABLE
+POSTHOOK: Input: default@temps_null
+POSTHOOK: Output: default@temps_null