You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hive.apache.org by na...@apache.org on 2012/12/04 10:46:17 UTC
svn commit: r1416854 [1/2] - in /hive/trunk:
common/src/java/org/apache/hadoop/hive/conf/ conf/
hbase-handler/src/test/queries/positive/
hbase-handler/src/test/results/positive/
ql/src/java/org/apache/hadoop/hive/ql/exec/ ql/src/java/org/apache/hadoop/...
Author: namit
Date: Tue Dec 4 09:46:13 2012
New Revision: 1416854
URL: http://svn.apache.org/viewvc?rev=1416854&view=rev
Log:
HIVE-3750 JDBCStatsPublisher fails when ID length exceeds length of ID column
(Kevin Wilfong via namit)
Added:
hive/trunk/hbase-handler/src/test/queries/positive/hbase_stats3.q
hive/trunk/hbase-handler/src/test/results/positive/hbase_stats3.q.out
hive/trunk/ql/src/test/org/apache/hadoop/hive/ql/stats/KeyVerifyingStatsAggregator.java
hive/trunk/ql/src/test/queries/clientpositive/stats19.q
hive/trunk/ql/src/test/results/clientpositive/stats19.q.out
Modified:
hive/trunk/common/src/java/org/apache/hadoop/hive/conf/HiveConf.java
hive/trunk/conf/hive-default.xml.template
hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/exec/FileSinkOperator.java
hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/exec/StatsTask.java
hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/exec/TableScanOperator.java
hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/exec/Utilities.java
hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/optimizer/GenMRFileSink1.java
hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/parse/SemanticAnalyzer.java
hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/plan/FileSinkDesc.java
hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/plan/TableScanDesc.java
Modified: hive/trunk/common/src/java/org/apache/hadoop/hive/conf/HiveConf.java
URL: http://svn.apache.org/viewvc/hive/trunk/common/src/java/org/apache/hadoop/hive/conf/HiveConf.java?rev=1416854&r1=1416853&r2=1416854&view=diff
==============================================================================
--- hive/trunk/common/src/java/org/apache/hadoop/hive/conf/HiveConf.java (original)
+++ hive/trunk/common/src/java/org/apache/hadoop/hive/conf/HiveConf.java Tue Dec 4 09:46:13 2012
@@ -562,6 +562,7 @@ public class HiveConf extends Configurat
// standard error allowed for ndv estimates. A lower value indicates higher accuracy and a
// higher compute cost.
HIVE_STATS_NDV_ERROR("hive.stats.ndv.error", (float)20.0),
+ HIVE_STATS_KEY_PREFIX_MAX_LENGTH("hive.stats.key.prefix.max.length", 200),
// Concurrency
HIVE_SUPPORT_CONCURRENCY("hive.support.concurrency", false),
Modified: hive/trunk/conf/hive-default.xml.template
URL: http://svn.apache.org/viewvc/hive/trunk/conf/hive-default.xml.template?rev=1416854&r1=1416853&r2=1416854&view=diff
==============================================================================
--- hive/trunk/conf/hive-default.xml.template (original)
+++ hive/trunk/conf/hive-default.xml.template Tue Dec 4 09:46:13 2012
@@ -1096,6 +1096,17 @@
</property>
<property>
+ <name>hive.stats.key.prefix.max.length</name>
+ <value>200</value>
+ <description>
+ Determines if when the prefix of the key used for intermediate stats collection
+ exceeds a certain length, a hash of the key is used instead. If the value < 0 then hashing
+ is never used, if the value >= 0 then hashing is used only when the key prefixes length
+ exceeds that value. The key prefix is defined as everything preceding the task ID in the key.
+ </description>
+</property>
+
+<property>
<name>hive.support.concurrency</name>
<value>false</value>
<description>Whether hive supports concurrency or not. A zookeeper instance must be up and running for the default hive lock manager to support read-write locks.</description>
Added: hive/trunk/hbase-handler/src/test/queries/positive/hbase_stats3.q
URL: http://svn.apache.org/viewvc/hive/trunk/hbase-handler/src/test/queries/positive/hbase_stats3.q?rev=1416854&view=auto
==============================================================================
--- hive/trunk/hbase-handler/src/test/queries/positive/hbase_stats3.q (added)
+++ hive/trunk/hbase-handler/src/test/queries/positive/hbase_stats3.q Tue Dec 4 09:46:13 2012
@@ -0,0 +1,50 @@
+set datanucleus.cache.collections=false;
+set hive.stats.autogather=true;
+set hive.stats.atomic=false;
+set hive.stats.collect.uncompressedsize=false;
+
+create table stats_part like srcpart;
+
+set hive.stats.key.prefix.max.length=0;
+
+-- The stats key should be hashed since the max length is too small
+insert overwrite table stats_part partition (ds='2010-04-08', hr = '13') select key, value from src;
+
+desc formatted stats_part partition (ds='2010-04-08', hr = '13');
+
+set hive.stats.key.prefix.max.length=200;
+
+-- The stats key should not be hashed since the max length is large enough
+insert overwrite table stats_part partition (ds='2010-04-08', hr = '13') select key, value from src;
+
+desc formatted stats_part partition (ds='2010-04-08', hr = '13');
+
+set hive.stats.key.prefix.max.length=-1;
+
+-- The stats key should not be hashed since negative values should imply hashing is turned off
+insert overwrite table stats_part partition (ds='2010-04-08', hr = '13') select key, value from src;
+
+desc formatted stats_part partition (ds='2010-04-08', hr = '13');
+
+set hive.stats.key.prefix.max.length=0;
+
+-- Verify the stats are correct for dynamic partitions
+
+-- The stats key should be hashed since the max length is too small
+insert overwrite table stats_part partition (ds='2010-04-08', hr) select key, value, '13' from src;
+
+desc formatted stats_part partition (ds='2010-04-08', hr = '13');
+
+set hive.stats.key.prefix.max.length=200;
+
+-- The stats key should not be hashed since the max length is large enough
+insert overwrite table stats_part partition (ds='2010-04-08', hr) select key, value, '13' from src;
+
+desc formatted stats_part partition (ds='2010-04-08', hr = '13');
+
+set hive.stats.key.prefix.max.length=-1;
+
+-- The stats key should not be hashed since negative values should imply hashing is turned off
+insert overwrite table stats_part partition (ds='2010-04-08', hr) select key, value, '13' from src;
+
+desc formatted stats_part partition (ds='2010-04-08', hr = '13');
Added: hive/trunk/hbase-handler/src/test/results/positive/hbase_stats3.q.out
URL: http://svn.apache.org/viewvc/hive/trunk/hbase-handler/src/test/results/positive/hbase_stats3.q.out?rev=1416854&view=auto
==============================================================================
--- hive/trunk/hbase-handler/src/test/results/positive/hbase_stats3.q.out (added)
+++ hive/trunk/hbase-handler/src/test/results/positive/hbase_stats3.q.out Tue Dec 4 09:46:13 2012
@@ -0,0 +1,387 @@
+PREHOOK: query: create table stats_part like srcpart
+PREHOOK: type: CREATETABLE
+POSTHOOK: query: create table stats_part like srcpart
+POSTHOOK: type: CREATETABLE
+POSTHOOK: Output: default@stats_part
+PREHOOK: query: -- The stats key should be hashed since the max length is too small
+insert overwrite table stats_part partition (ds='2010-04-08', hr = '13') select key, value from src
+PREHOOK: type: QUERY
+PREHOOK: Input: default@src
+PREHOOK: Output: default@stats_part@ds=2010-04-08/hr=13
+POSTHOOK: query: -- The stats key should be hashed since the max length is too small
+insert overwrite table stats_part partition (ds='2010-04-08', hr = '13') select key, value from src
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@src
+POSTHOOK: Output: default@stats_part@ds=2010-04-08/hr=13
+POSTHOOK: Lineage: stats_part PARTITION(ds=2010-04-08,hr=13).key SIMPLE [(src)src.FieldSchema(name:key, type:string, comment:default), ]
+POSTHOOK: Lineage: stats_part PARTITION(ds=2010-04-08,hr=13).value SIMPLE [(src)src.FieldSchema(name:value, type:string, comment:default), ]
+PREHOOK: query: desc formatted stats_part partition (ds='2010-04-08', hr = '13')
+PREHOOK: type: DESCTABLE
+POSTHOOK: query: desc formatted stats_part partition (ds='2010-04-08', hr = '13')
+POSTHOOK: type: DESCTABLE
+POSTHOOK: Lineage: stats_part PARTITION(ds=2010-04-08,hr=13).key SIMPLE [(src)src.FieldSchema(name:key, type:string, comment:default), ]
+POSTHOOK: Lineage: stats_part PARTITION(ds=2010-04-08,hr=13).value SIMPLE [(src)src.FieldSchema(name:value, type:string, comment:default), ]
+# col_name data_type comment
+
+key string default
+value string default
+
+# Partition Information
+# col_name data_type comment
+
+ds string None
+hr string None
+
+# Detailed Partition Information
+Partition Value: [2010-04-08, 13]
+Database: default
+Table: stats_part
+#### A masked pattern was here ####
+Protect Mode: None
+#### A masked pattern was here ####
+Partition Parameters:
+ numFiles 1
+ numRows 500
+ rawDataSize 5312
+ totalSize 5812
+#### 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: -- The stats key should not be hashed since the max length is large enough
+insert overwrite table stats_part partition (ds='2010-04-08', hr = '13') select key, value from src
+PREHOOK: type: QUERY
+PREHOOK: Input: default@src
+PREHOOK: Output: default@stats_part@ds=2010-04-08/hr=13
+POSTHOOK: query: -- The stats key should not be hashed since the max length is large enough
+insert overwrite table stats_part partition (ds='2010-04-08', hr = '13') select key, value from src
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@src
+POSTHOOK: Output: default@stats_part@ds=2010-04-08/hr=13
+POSTHOOK: Lineage: stats_part PARTITION(ds=2010-04-08,hr=13).key SIMPLE [(src)src.FieldSchema(name:key, type:string, comment:default), ]
+POSTHOOK: Lineage: stats_part PARTITION(ds=2010-04-08,hr=13).value SIMPLE [(src)src.FieldSchema(name:value, type:string, comment:default), ]
+POSTHOOK: Lineage: stats_part PARTITION(ds=2010-04-08,hr=13).key SIMPLE [(src)src.FieldSchema(name:key, type:string, comment:default), ]
+POSTHOOK: Lineage: stats_part PARTITION(ds=2010-04-08,hr=13).value SIMPLE [(src)src.FieldSchema(name:value, type:string, comment:default), ]
+PREHOOK: query: desc formatted stats_part partition (ds='2010-04-08', hr = '13')
+PREHOOK: type: DESCTABLE
+POSTHOOK: query: desc formatted stats_part partition (ds='2010-04-08', hr = '13')
+POSTHOOK: type: DESCTABLE
+POSTHOOK: Lineage: stats_part PARTITION(ds=2010-04-08,hr=13).key SIMPLE [(src)src.FieldSchema(name:key, type:string, comment:default), ]
+POSTHOOK: Lineage: stats_part PARTITION(ds=2010-04-08,hr=13).value SIMPLE [(src)src.FieldSchema(name:value, type:string, comment:default), ]
+POSTHOOK: Lineage: stats_part PARTITION(ds=2010-04-08,hr=13).key SIMPLE [(src)src.FieldSchema(name:key, type:string, comment:default), ]
+POSTHOOK: Lineage: stats_part PARTITION(ds=2010-04-08,hr=13).value SIMPLE [(src)src.FieldSchema(name:value, type:string, comment:default), ]
+# col_name data_type comment
+
+key string default
+value string default
+
+# Partition Information
+# col_name data_type comment
+
+ds string None
+hr string None
+
+# Detailed Partition Information
+Partition Value: [2010-04-08, 13]
+Database: default
+Table: stats_part
+#### A masked pattern was here ####
+Protect Mode: None
+#### A masked pattern was here ####
+Partition Parameters:
+ numFiles 1
+ numRows 500
+ rawDataSize 5312
+ totalSize 5812
+#### 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: -- The stats key should not be hashed since negative values should imply hashing is turned off
+insert overwrite table stats_part partition (ds='2010-04-08', hr = '13') select key, value from src
+PREHOOK: type: QUERY
+PREHOOK: Input: default@src
+PREHOOK: Output: default@stats_part@ds=2010-04-08/hr=13
+POSTHOOK: query: -- The stats key should not be hashed since negative values should imply hashing is turned off
+insert overwrite table stats_part partition (ds='2010-04-08', hr = '13') select key, value from src
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@src
+POSTHOOK: Output: default@stats_part@ds=2010-04-08/hr=13
+POSTHOOK: Lineage: stats_part PARTITION(ds=2010-04-08,hr=13).key SIMPLE [(src)src.FieldSchema(name:key, type:string, comment:default), ]
+POSTHOOK: Lineage: stats_part PARTITION(ds=2010-04-08,hr=13).value SIMPLE [(src)src.FieldSchema(name:value, type:string, comment:default), ]
+POSTHOOK: Lineage: stats_part PARTITION(ds=2010-04-08,hr=13).key SIMPLE [(src)src.FieldSchema(name:key, type:string, comment:default), ]
+POSTHOOK: Lineage: stats_part PARTITION(ds=2010-04-08,hr=13).value SIMPLE [(src)src.FieldSchema(name:value, type:string, comment:default), ]
+POSTHOOK: Lineage: stats_part PARTITION(ds=2010-04-08,hr=13).key SIMPLE [(src)src.FieldSchema(name:key, type:string, comment:default), ]
+POSTHOOK: Lineage: stats_part PARTITION(ds=2010-04-08,hr=13).value SIMPLE [(src)src.FieldSchema(name:value, type:string, comment:default), ]
+PREHOOK: query: desc formatted stats_part partition (ds='2010-04-08', hr = '13')
+PREHOOK: type: DESCTABLE
+POSTHOOK: query: desc formatted stats_part partition (ds='2010-04-08', hr = '13')
+POSTHOOK: type: DESCTABLE
+POSTHOOK: Lineage: stats_part PARTITION(ds=2010-04-08,hr=13).key SIMPLE [(src)src.FieldSchema(name:key, type:string, comment:default), ]
+POSTHOOK: Lineage: stats_part PARTITION(ds=2010-04-08,hr=13).value SIMPLE [(src)src.FieldSchema(name:value, type:string, comment:default), ]
+POSTHOOK: Lineage: stats_part PARTITION(ds=2010-04-08,hr=13).key SIMPLE [(src)src.FieldSchema(name:key, type:string, comment:default), ]
+POSTHOOK: Lineage: stats_part PARTITION(ds=2010-04-08,hr=13).value SIMPLE [(src)src.FieldSchema(name:value, type:string, comment:default), ]
+POSTHOOK: Lineage: stats_part PARTITION(ds=2010-04-08,hr=13).key SIMPLE [(src)src.FieldSchema(name:key, type:string, comment:default), ]
+POSTHOOK: Lineage: stats_part PARTITION(ds=2010-04-08,hr=13).value SIMPLE [(src)src.FieldSchema(name:value, type:string, comment:default), ]
+# col_name data_type comment
+
+key string default
+value string default
+
+# Partition Information
+# col_name data_type comment
+
+ds string None
+hr string None
+
+# Detailed Partition Information
+Partition Value: [2010-04-08, 13]
+Database: default
+Table: stats_part
+#### A masked pattern was here ####
+Protect Mode: None
+#### A masked pattern was here ####
+Partition Parameters:
+ numFiles 1
+ numRows 500
+ rawDataSize 5312
+ totalSize 5812
+#### 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: -- Verify the stats are correct for dynamic partitions
+
+-- The stats key should be hashed since the max length is too small
+insert overwrite table stats_part partition (ds='2010-04-08', hr) select key, value, '13' from src
+PREHOOK: type: QUERY
+PREHOOK: Input: default@src
+PREHOOK: Output: default@stats_part@ds=2010-04-08
+POSTHOOK: query: -- Verify the stats are correct for dynamic partitions
+
+-- The stats key should be hashed since the max length is too small
+insert overwrite table stats_part partition (ds='2010-04-08', hr) select key, value, '13' from src
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@src
+POSTHOOK: Output: default@stats_part@ds=2010-04-08/hr=13
+POSTHOOK: Lineage: stats_part PARTITION(ds=2010-04-08,hr=13).key SIMPLE [(src)src.FieldSchema(name:key, type:string, comment:default), ]
+POSTHOOK: Lineage: stats_part PARTITION(ds=2010-04-08,hr=13).value SIMPLE [(src)src.FieldSchema(name:value, type:string, comment:default), ]
+POSTHOOK: Lineage: stats_part PARTITION(ds=2010-04-08,hr=13).key SIMPLE [(src)src.FieldSchema(name:key, type:string, comment:default), ]
+POSTHOOK: Lineage: stats_part PARTITION(ds=2010-04-08,hr=13).value SIMPLE [(src)src.FieldSchema(name:value, type:string, comment:default), ]
+POSTHOOK: Lineage: stats_part PARTITION(ds=2010-04-08,hr=13).key SIMPLE [(src)src.FieldSchema(name:key, type:string, comment:default), ]
+POSTHOOK: Lineage: stats_part PARTITION(ds=2010-04-08,hr=13).value SIMPLE [(src)src.FieldSchema(name:value, type:string, comment:default), ]
+POSTHOOK: Lineage: stats_part PARTITION(ds=2010-04-08,hr=13).key SIMPLE [(src)src.FieldSchema(name:key, type:string, comment:default), ]
+POSTHOOK: Lineage: stats_part PARTITION(ds=2010-04-08,hr=13).value SIMPLE [(src)src.FieldSchema(name:value, type:string, comment:default), ]
+PREHOOK: query: desc formatted stats_part partition (ds='2010-04-08', hr = '13')
+PREHOOK: type: DESCTABLE
+POSTHOOK: query: desc formatted stats_part partition (ds='2010-04-08', hr = '13')
+POSTHOOK: type: DESCTABLE
+POSTHOOK: Lineage: stats_part PARTITION(ds=2010-04-08,hr=13).key SIMPLE [(src)src.FieldSchema(name:key, type:string, comment:default), ]
+POSTHOOK: Lineage: stats_part PARTITION(ds=2010-04-08,hr=13).value SIMPLE [(src)src.FieldSchema(name:value, type:string, comment:default), ]
+POSTHOOK: Lineage: stats_part PARTITION(ds=2010-04-08,hr=13).key SIMPLE [(src)src.FieldSchema(name:key, type:string, comment:default), ]
+POSTHOOK: Lineage: stats_part PARTITION(ds=2010-04-08,hr=13).value SIMPLE [(src)src.FieldSchema(name:value, type:string, comment:default), ]
+POSTHOOK: Lineage: stats_part PARTITION(ds=2010-04-08,hr=13).key SIMPLE [(src)src.FieldSchema(name:key, type:string, comment:default), ]
+POSTHOOK: Lineage: stats_part PARTITION(ds=2010-04-08,hr=13).value SIMPLE [(src)src.FieldSchema(name:value, type:string, comment:default), ]
+POSTHOOK: Lineage: stats_part PARTITION(ds=2010-04-08,hr=13).key SIMPLE [(src)src.FieldSchema(name:key, type:string, comment:default), ]
+POSTHOOK: Lineage: stats_part PARTITION(ds=2010-04-08,hr=13).value SIMPLE [(src)src.FieldSchema(name:value, type:string, comment:default), ]
+# col_name data_type comment
+
+key string default
+value string default
+
+# Partition Information
+# col_name data_type comment
+
+ds string None
+hr string None
+
+# Detailed Partition Information
+Partition Value: [2010-04-08, 13]
+Database: default
+Table: stats_part
+#### A masked pattern was here ####
+Protect Mode: None
+#### A masked pattern was here ####
+Partition Parameters:
+ numFiles 1
+ numRows 500
+ rawDataSize 5312
+ totalSize 5812
+#### 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: -- The stats key should not be hashed since the max length is large enough
+insert overwrite table stats_part partition (ds='2010-04-08', hr) select key, value, '13' from src
+PREHOOK: type: QUERY
+PREHOOK: Input: default@src
+PREHOOK: Output: default@stats_part@ds=2010-04-08
+POSTHOOK: query: -- The stats key should not be hashed since the max length is large enough
+insert overwrite table stats_part partition (ds='2010-04-08', hr) select key, value, '13' from src
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@src
+POSTHOOK: Output: default@stats_part@ds=2010-04-08/hr=13
+POSTHOOK: Lineage: stats_part PARTITION(ds=2010-04-08,hr=13).key SIMPLE [(src)src.FieldSchema(name:key, type:string, comment:default), ]
+POSTHOOK: Lineage: stats_part PARTITION(ds=2010-04-08,hr=13).value SIMPLE [(src)src.FieldSchema(name:value, type:string, comment:default), ]
+POSTHOOK: Lineage: stats_part PARTITION(ds=2010-04-08,hr=13).key SIMPLE [(src)src.FieldSchema(name:key, type:string, comment:default), ]
+POSTHOOK: Lineage: stats_part PARTITION(ds=2010-04-08,hr=13).value SIMPLE [(src)src.FieldSchema(name:value, type:string, comment:default), ]
+POSTHOOK: Lineage: stats_part PARTITION(ds=2010-04-08,hr=13).key SIMPLE [(src)src.FieldSchema(name:key, type:string, comment:default), ]
+POSTHOOK: Lineage: stats_part PARTITION(ds=2010-04-08,hr=13).value SIMPLE [(src)src.FieldSchema(name:value, type:string, comment:default), ]
+POSTHOOK: Lineage: stats_part PARTITION(ds=2010-04-08,hr=13).key SIMPLE [(src)src.FieldSchema(name:key, type:string, comment:default), ]
+POSTHOOK: Lineage: stats_part PARTITION(ds=2010-04-08,hr=13).value SIMPLE [(src)src.FieldSchema(name:value, type:string, comment:default), ]
+POSTHOOK: Lineage: stats_part PARTITION(ds=2010-04-08,hr=13).key SIMPLE [(src)src.FieldSchema(name:key, type:string, comment:default), ]
+POSTHOOK: Lineage: stats_part PARTITION(ds=2010-04-08,hr=13).value SIMPLE [(src)src.FieldSchema(name:value, type:string, comment:default), ]
+PREHOOK: query: desc formatted stats_part partition (ds='2010-04-08', hr = '13')
+PREHOOK: type: DESCTABLE
+POSTHOOK: query: desc formatted stats_part partition (ds='2010-04-08', hr = '13')
+POSTHOOK: type: DESCTABLE
+POSTHOOK: Lineage: stats_part PARTITION(ds=2010-04-08,hr=13).key SIMPLE [(src)src.FieldSchema(name:key, type:string, comment:default), ]
+POSTHOOK: Lineage: stats_part PARTITION(ds=2010-04-08,hr=13).value SIMPLE [(src)src.FieldSchema(name:value, type:string, comment:default), ]
+POSTHOOK: Lineage: stats_part PARTITION(ds=2010-04-08,hr=13).key SIMPLE [(src)src.FieldSchema(name:key, type:string, comment:default), ]
+POSTHOOK: Lineage: stats_part PARTITION(ds=2010-04-08,hr=13).value SIMPLE [(src)src.FieldSchema(name:value, type:string, comment:default), ]
+POSTHOOK: Lineage: stats_part PARTITION(ds=2010-04-08,hr=13).key SIMPLE [(src)src.FieldSchema(name:key, type:string, comment:default), ]
+POSTHOOK: Lineage: stats_part PARTITION(ds=2010-04-08,hr=13).value SIMPLE [(src)src.FieldSchema(name:value, type:string, comment:default), ]
+POSTHOOK: Lineage: stats_part PARTITION(ds=2010-04-08,hr=13).key SIMPLE [(src)src.FieldSchema(name:key, type:string, comment:default), ]
+POSTHOOK: Lineage: stats_part PARTITION(ds=2010-04-08,hr=13).value SIMPLE [(src)src.FieldSchema(name:value, type:string, comment:default), ]
+POSTHOOK: Lineage: stats_part PARTITION(ds=2010-04-08,hr=13).key SIMPLE [(src)src.FieldSchema(name:key, type:string, comment:default), ]
+POSTHOOK: Lineage: stats_part PARTITION(ds=2010-04-08,hr=13).value SIMPLE [(src)src.FieldSchema(name:value, type:string, comment:default), ]
+# col_name data_type comment
+
+key string default
+value string default
+
+# Partition Information
+# col_name data_type comment
+
+ds string None
+hr string None
+
+# Detailed Partition Information
+Partition Value: [2010-04-08, 13]
+Database: default
+Table: stats_part
+#### A masked pattern was here ####
+Protect Mode: None
+#### A masked pattern was here ####
+Partition Parameters:
+ numFiles 1
+ numRows 500
+ rawDataSize 5312
+ totalSize 5812
+#### 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: -- The stats key should not be hashed since negative values should imply hashing is turned off
+insert overwrite table stats_part partition (ds='2010-04-08', hr) select key, value, '13' from src
+PREHOOK: type: QUERY
+PREHOOK: Input: default@src
+PREHOOK: Output: default@stats_part@ds=2010-04-08
+POSTHOOK: query: -- The stats key should not be hashed since negative values should imply hashing is turned off
+insert overwrite table stats_part partition (ds='2010-04-08', hr) select key, value, '13' from src
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@src
+POSTHOOK: Output: default@stats_part@ds=2010-04-08/hr=13
+POSTHOOK: Lineage: stats_part PARTITION(ds=2010-04-08,hr=13).key SIMPLE [(src)src.FieldSchema(name:key, type:string, comment:default), ]
+POSTHOOK: Lineage: stats_part PARTITION(ds=2010-04-08,hr=13).value SIMPLE [(src)src.FieldSchema(name:value, type:string, comment:default), ]
+POSTHOOK: Lineage: stats_part PARTITION(ds=2010-04-08,hr=13).key SIMPLE [(src)src.FieldSchema(name:key, type:string, comment:default), ]
+POSTHOOK: Lineage: stats_part PARTITION(ds=2010-04-08,hr=13).value SIMPLE [(src)src.FieldSchema(name:value, type:string, comment:default), ]
+POSTHOOK: Lineage: stats_part PARTITION(ds=2010-04-08,hr=13).key SIMPLE [(src)src.FieldSchema(name:key, type:string, comment:default), ]
+POSTHOOK: Lineage: stats_part PARTITION(ds=2010-04-08,hr=13).value SIMPLE [(src)src.FieldSchema(name:value, type:string, comment:default), ]
+POSTHOOK: Lineage: stats_part PARTITION(ds=2010-04-08,hr=13).key SIMPLE [(src)src.FieldSchema(name:key, type:string, comment:default), ]
+POSTHOOK: Lineage: stats_part PARTITION(ds=2010-04-08,hr=13).value SIMPLE [(src)src.FieldSchema(name:value, type:string, comment:default), ]
+POSTHOOK: Lineage: stats_part PARTITION(ds=2010-04-08,hr=13).key SIMPLE [(src)src.FieldSchema(name:key, type:string, comment:default), ]
+POSTHOOK: Lineage: stats_part PARTITION(ds=2010-04-08,hr=13).value SIMPLE [(src)src.FieldSchema(name:value, type:string, comment:default), ]
+POSTHOOK: Lineage: stats_part PARTITION(ds=2010-04-08,hr=13).key SIMPLE [(src)src.FieldSchema(name:key, type:string, comment:default), ]
+POSTHOOK: Lineage: stats_part PARTITION(ds=2010-04-08,hr=13).value SIMPLE [(src)src.FieldSchema(name:value, type:string, comment:default), ]
+PREHOOK: query: desc formatted stats_part partition (ds='2010-04-08', hr = '13')
+PREHOOK: type: DESCTABLE
+POSTHOOK: query: desc formatted stats_part partition (ds='2010-04-08', hr = '13')
+POSTHOOK: type: DESCTABLE
+POSTHOOK: Lineage: stats_part PARTITION(ds=2010-04-08,hr=13).key SIMPLE [(src)src.FieldSchema(name:key, type:string, comment:default), ]
+POSTHOOK: Lineage: stats_part PARTITION(ds=2010-04-08,hr=13).value SIMPLE [(src)src.FieldSchema(name:value, type:string, comment:default), ]
+POSTHOOK: Lineage: stats_part PARTITION(ds=2010-04-08,hr=13).key SIMPLE [(src)src.FieldSchema(name:key, type:string, comment:default), ]
+POSTHOOK: Lineage: stats_part PARTITION(ds=2010-04-08,hr=13).value SIMPLE [(src)src.FieldSchema(name:value, type:string, comment:default), ]
+POSTHOOK: Lineage: stats_part PARTITION(ds=2010-04-08,hr=13).key SIMPLE [(src)src.FieldSchema(name:key, type:string, comment:default), ]
+POSTHOOK: Lineage: stats_part PARTITION(ds=2010-04-08,hr=13).value SIMPLE [(src)src.FieldSchema(name:value, type:string, comment:default), ]
+POSTHOOK: Lineage: stats_part PARTITION(ds=2010-04-08,hr=13).key SIMPLE [(src)src.FieldSchema(name:key, type:string, comment:default), ]
+POSTHOOK: Lineage: stats_part PARTITION(ds=2010-04-08,hr=13).value SIMPLE [(src)src.FieldSchema(name:value, type:string, comment:default), ]
+POSTHOOK: Lineage: stats_part PARTITION(ds=2010-04-08,hr=13).key SIMPLE [(src)src.FieldSchema(name:key, type:string, comment:default), ]
+POSTHOOK: Lineage: stats_part PARTITION(ds=2010-04-08,hr=13).value SIMPLE [(src)src.FieldSchema(name:value, type:string, comment:default), ]
+POSTHOOK: Lineage: stats_part PARTITION(ds=2010-04-08,hr=13).key SIMPLE [(src)src.FieldSchema(name:key, type:string, comment:default), ]
+POSTHOOK: Lineage: stats_part PARTITION(ds=2010-04-08,hr=13).value SIMPLE [(src)src.FieldSchema(name:value, type:string, comment:default), ]
+# col_name data_type comment
+
+key string default
+value string default
+
+# Partition Information
+# col_name data_type comment
+
+ds string None
+hr string None
+
+# Detailed Partition Information
+Partition Value: [2010-04-08, 13]
+Database: default
+Table: stats_part
+#### A masked pattern was here ####
+Protect Mode: None
+#### A masked pattern was here ####
+Partition Parameters:
+ numFiles 1
+ numRows 500
+ rawDataSize 5312
+ totalSize 5812
+#### 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
Modified: hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/exec/FileSinkOperator.java
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/exec/FileSinkOperator.java?rev=1416854&r1=1416853&r2=1416854&view=diff
==============================================================================
--- hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/exec/FileSinkOperator.java (original)
+++ hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/exec/FileSinkOperator.java Tue Dec 4 09:46:13 2012
@@ -860,11 +860,16 @@ public class FileSinkOperator extends Te
if (fspKey == "") {
// for non-partitioned/static partitioned table, the key for temp storage is
// common key prefix + static partition spec + taskID
- key = conf.getStatsAggPrefix() + spSpec + taskID;
+ String keyPrefix = Utilities.getHashedStatsPrefix(
+ conf.getStatsAggPrefix() + spSpec, conf.getMaxStatsKeyPrefixLength());
+ key = keyPrefix + taskID;
} else {
// for partitioned table, the key is
// common key prefix + static partition spec + DynamicPartSpec + taskID
- key = conf.getStatsAggPrefix() + spSpec + fspKey + Path.SEPARATOR + taskID;
+ String keyPrefix = Utilities.getHashedStatsPrefix(
+ conf.getStatsAggPrefix() + spSpec + fspKey + Path.SEPARATOR,
+ conf.getMaxStatsKeyPrefixLength());
+ key = keyPrefix + taskID;
}
Map<String, String> statsToPublish = new HashMap<String, String>();
for (String statType : fspValue.stat.getStoredStats()) {
Modified: hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/exec/StatsTask.java
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/exec/StatsTask.java?rev=1416854&r1=1416853&r2=1416854&view=diff
==============================================================================
--- hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/exec/StatsTask.java (original)
+++ hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/exec/StatsTask.java Tue Dec 4 09:46:13 2012
@@ -306,6 +306,8 @@ public class StatsTask extends Task<Stat
List<Partition> partitions = getPartitionsList();
boolean atomic = HiveConf.getBoolVar(conf, HiveConf.ConfVars.HIVE_STATS_ATOMIC);
+ int maxPrefixLength = HiveConf.getIntVar(conf,
+ HiveConf.ConfVars.HIVE_STATS_KEY_PREFIX_MAX_LENGTH);
if (partitions == null) {
// non-partitioned tables:
@@ -325,9 +327,10 @@ public class StatsTask extends Task<Stat
// In case of a non-partitioned table, the key for stats temporary store is "rootDir"
if (statsAggregator != null) {
+ String aggKey = Utilities.getHashedStatsPrefix(work.getAggKey(), maxPrefixLength);
updateStats(collectableStats, tblStats, statsAggregator, parameters,
- work.getAggKey(), atomic);
- statsAggregator.cleanUp(work.getAggKey());
+ aggKey, atomic);
+ statsAggregator.cleanUp(aggKey);
}
// The collectable stats for the aggregator needs to be cleared.
// For eg. if a file is being loaded, the old number of rows are not valid
@@ -368,7 +371,8 @@ public class StatsTask extends Task<Stat
// In that case of a partition, the key for stats temporary store is
// "rootDir/[dynamic_partition_specs/]%"
- String partitionID = work.getAggKey() + Warehouse.makePartPath(partn.getSpec());
+ String partitionID = Utilities.getHashedStatsPrefix(
+ work.getAggKey() + Warehouse.makePartPath(partn.getSpec()), maxPrefixLength);
LOG.info("Stats aggregator : " + partitionID);
Modified: hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/exec/TableScanOperator.java
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/exec/TableScanOperator.java?rev=1416854&r1=1416853&r2=1416854&view=diff
==============================================================================
--- hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/exec/TableScanOperator.java (original)
+++ hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/exec/TableScanOperator.java Tue Dec 4 09:46:13 2012
@@ -27,6 +27,7 @@ import java.util.Map;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.hive.common.FileUtils;
+import org.apache.hadoop.hive.conf.HiveConf;
import org.apache.hadoop.hive.ql.ErrorMsg;
import org.apache.hadoop.hive.ql.metadata.HiveException;
import org.apache.hadoop.hive.ql.metadata.VirtualColumn;
@@ -256,11 +257,15 @@ public class TableScanOperator extends O
if (pspecs.isEmpty()) {
// In case of a non-partitioned table, the key for temp storage is just
// "tableName + taskID"
- key = conf.getStatsAggPrefix() + taskID;
+ String keyPrefix = Utilities.getHashedStatsPrefix(
+ conf.getStatsAggPrefix(), conf.getMaxStatsKeyPrefixLength());
+ key = keyPrefix + taskID;
} else {
// In case of a partition, the key for temp storage is
// "tableName + partitionSpecs + taskID"
- key = conf.getStatsAggPrefix() + pspecs + Path.SEPARATOR + taskID;
+ String keyPrefix = Utilities.getHashedStatsPrefix(
+ conf.getStatsAggPrefix() + pspecs + Path.SEPARATOR, conf.getMaxStatsKeyPrefixLength());
+ key = keyPrefix + taskID;
}
for(String statType : stats.get(pspecs).getStoredStats()) {
statsToPublish.put(statType, Long.toString(stats.get(pspecs).getStat(statType)));
Modified: hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/exec/Utilities.java
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/exec/Utilities.java?rev=1416854&r1=1416853&r2=1416854&view=diff
==============================================================================
--- hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/exec/Utilities.java (original)
+++ hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/exec/Utilities.java Tue Dec 4 09:46:13 2012
@@ -43,6 +43,8 @@ import java.io.UnsupportedEncodingExcept
import java.net.URI;
import java.net.URL;
import java.net.URLClassLoader;
+import java.security.MessageDigest;
+import java.security.NoSuchAlgorithmException;
import java.sql.Connection;
import java.sql.DriverManager;
import java.sql.PreparedStatement;
@@ -1991,6 +1993,29 @@ public final class Utilities {
}
}
+ /**
+ * If statsPrefix's length is greater than maxPrefixLength and maxPrefixLength > 0,
+ * then it returns an MD5 hash of statsPrefix followed by path separator, otherwise
+ * it returns statsPrefix
+ *
+ * @param statsPrefix
+ * @param maxPrefixLength
+ * @return
+ */
+ public static String getHashedStatsPrefix(String statsPrefix, int maxPrefixLength) {
+ String ret = statsPrefix;
+ if (maxPrefixLength >= 0 && statsPrefix.length() > maxPrefixLength) {
+ try {
+ MessageDigest digester = MessageDigest.getInstance("MD5");
+ digester.update(statsPrefix.getBytes());
+ ret = new String(digester.digest()) + Path.SEPARATOR;
+ } catch (NoSuchAlgorithmException e) {
+ throw new RuntimeException(e);
+ }
+ }
+ return ret;
+ }
+
public static void setColumnNameList(JobConf jobConf, Operator op) {
RowSchema rowSchema = op.getSchema();
if (rowSchema == null) {
Modified: hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/optimizer/GenMRFileSink1.java
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/optimizer/GenMRFileSink1.java?rev=1416854&r1=1416853&r2=1416854&view=diff
==============================================================================
--- hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/optimizer/GenMRFileSink1.java (original)
+++ hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/optimizer/GenMRFileSink1.java Tue Dec 4 09:46:13 2012
@@ -257,6 +257,8 @@ public class GenMRFileSink1 implements N
nd.getConf().setGatherStats(true);
mrWork.setGatheringStats(true);
nd.getConf().setStatsReliable(hconf.getBoolVar(HiveConf.ConfVars.HIVE_STATS_RELIABLE));
+ nd.getConf().setMaxStatsKeyPrefixLength(
+ hconf.getIntVar(HiveConf.ConfVars.HIVE_STATS_KEY_PREFIX_MAX_LENGTH));
// mrWork.addDestinationTable(nd.getConf().getTableInfo().getTableName());
// subscribe feeds from the MoveTask so that MoveTask can forward the list
Modified: hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/parse/SemanticAnalyzer.java
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/parse/SemanticAnalyzer.java?rev=1416854&r1=1416853&r2=1416854&view=diff
==============================================================================
--- hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/parse/SemanticAnalyzer.java (original)
+++ hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/parse/SemanticAnalyzer.java Tue Dec 4 09:46:13 2012
@@ -7339,6 +7339,8 @@ public class SemanticAnalyzer extends Ba
} else {
tsDesc.setGatherStats(true);
tsDesc.setStatsReliable(conf.getBoolVar(HiveConf.ConfVars.HIVE_STATS_RELIABLE));
+ tsDesc.setMaxStatsKeyPrefixLength(
+ conf.getIntVar(HiveConf.ConfVars.HIVE_STATS_KEY_PREFIX_MAX_LENGTH));
// append additional virtual columns for storing statistics
Iterator<VirtualColumn> vcs = VirtualColumn.getStatsRegistry(conf).iterator();
Modified: hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/plan/FileSinkDesc.java
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/plan/FileSinkDesc.java?rev=1416854&r1=1416853&r2=1416854&view=diff
==============================================================================
--- hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/plan/FileSinkDesc.java (original)
+++ hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/plan/FileSinkDesc.java Tue Dec 4 09:46:13 2012
@@ -60,6 +60,7 @@ public class FileSinkDesc extends Abstra
transient private List<FileSinkDesc> linkedFileSinkDesc;
private boolean statsReliable;
+ private int maxStatsKeyPrefixLength = -1;
public FileSinkDesc() {
}
@@ -107,6 +108,7 @@ public class FileSinkDesc extends Abstra
ret.setParentDir(parentDir);
ret.setLinkedFileSinkDesc(linkedFileSinkDesc);
ret.setStatsReliable(statsReliable);
+ ret.setMaxStatsKeyPrefixLength(maxStatsKeyPrefixLength);
return (Object) ret;
}
@@ -314,4 +316,12 @@ public class FileSinkDesc extends Abstra
public void setLinkedFileSinkDesc(List<FileSinkDesc> linkedFileSinkDesc) {
this.linkedFileSinkDesc = linkedFileSinkDesc;
}
+
+ public int getMaxStatsKeyPrefixLength() {
+ return maxStatsKeyPrefixLength;
+ }
+
+ public void setMaxStatsKeyPrefixLength(int maxStatsKeyPrefixLength) {
+ this.maxStatsKeyPrefixLength = maxStatsKeyPrefixLength;
+ }
}
Modified: hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/plan/TableScanDesc.java
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/plan/TableScanDesc.java?rev=1416854&r1=1416853&r2=1416854&view=diff
==============================================================================
--- hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/plan/TableScanDesc.java (original)
+++ hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/plan/TableScanDesc.java Tue Dec 4 09:46:13 2012
@@ -49,6 +49,7 @@ public class TableScanDesc extends Abstr
*/
private boolean gatherStats;
private boolean statsReliable;
+ private int maxStatsKeyPrefixLength = -1;
private ExprNodeDesc filterExpr;
@@ -140,4 +141,12 @@ public class TableScanDesc extends Abstr
public void setStatsReliable(boolean statsReliable) {
this.statsReliable = statsReliable;
}
+
+ public int getMaxStatsKeyPrefixLength() {
+ return maxStatsKeyPrefixLength;
+ }
+
+ public void setMaxStatsKeyPrefixLength(int maxStatsKeyPrefixLength) {
+ this.maxStatsKeyPrefixLength = maxStatsKeyPrefixLength;
+ }
}
Added: hive/trunk/ql/src/test/org/apache/hadoop/hive/ql/stats/KeyVerifyingStatsAggregator.java
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/test/org/apache/hadoop/hive/ql/stats/KeyVerifyingStatsAggregator.java?rev=1416854&view=auto
==============================================================================
--- hive/trunk/ql/src/test/org/apache/hadoop/hive/ql/stats/KeyVerifyingStatsAggregator.java (added)
+++ hive/trunk/ql/src/test/org/apache/hadoop/hive/ql/stats/KeyVerifyingStatsAggregator.java Tue Dec 4 09:46:13 2012
@@ -0,0 +1,51 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.hive.ql.stats;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hive.ql.session.SessionState;
+
+/**
+ * An test implementation for StatsAggregator.
+ * aggregateStats prints the length of the keyPrefix to SessionState's out stream
+ * All other methods are no-ops.
+ */
+
+public class KeyVerifyingStatsAggregator implements StatsAggregator {
+
+ public boolean connect(Configuration hconf) {
+ return true;
+ }
+
+ public String aggregateStats(String keyPrefix, String statType) {
+ SessionState ss = SessionState.get();
+ // Have to use the length instead of the actual prefix because the prefix is location dependent
+ // 17 is 16 (16 byte MD5 hash) + 1 for the path separator
+ ss.out.println("Stats prefix is hashed: " + new Boolean(keyPrefix.length() == 17));
+ return null;
+ }
+
+ public boolean closeConnection() {
+ return true;
+ }
+
+ public boolean cleanUp(String keyPrefix) {
+ return true;
+ }
+}
Added: hive/trunk/ql/src/test/queries/clientpositive/stats19.q
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/test/queries/clientpositive/stats19.q?rev=1416854&view=auto
==============================================================================
--- hive/trunk/ql/src/test/queries/clientpositive/stats19.q (added)
+++ hive/trunk/ql/src/test/queries/clientpositive/stats19.q Tue Dec 4 09:46:13 2012
@@ -0,0 +1,105 @@
+set datanucleus.cache.collections=false;
+set hive.stats.autogather=true;
+set hive.stats.reliable=true;
+set hive.stats.dbclass=dummy;
+set hive.stats.default.publisher=org.apache.hadoop.hive.ql.stats.DummyStatsPublisher;
+set hive.stats.default.aggregator=org.apache.hadoop.hive.ql.stats.KeyVerifyingStatsAggregator;
+
+-- Note, its important that the partitions created below have a name greater than 16 characters in
+-- length since KeyVerifyingStatsAggregator depends on checking that a keyPrefix is hashed by the
+-- length of the keyPrefix, having a partition name greather than 16 characters guarantees no false
+-- positives.
+
+create table stats_part like srcpart;
+
+set hive.stats.key.prefix.max.length=0;
+
+-- The stats key should be hashed since the max length is too small
+insert overwrite table stats_part partition (ds='2010-04-08', hr = '13') select key, value from src;
+
+set hive.stats.key.prefix.max.length=200;
+
+-- The stats key should not be hashed since the max length is large enough
+insert overwrite table stats_part partition (ds='2010-04-08', hr = '13') select key, value from src;
+
+set hive.stats.key.prefix.max.length=-1;
+
+-- The stats key should not be hashed since negative values should imply hashing is turned off
+insert overwrite table stats_part partition (ds='2010-04-08', hr = '13') select key, value from src;
+
+set hive.stats.dbclass=jdbc:derby;
+set hive.stats.default.publisher=;
+set hive.stats.default.aggregator=;
+
+set hive.stats.key.prefix.max.length=0;
+
+-- Run the tests again and verify the stats are correct, this should verify that the stats publisher
+-- is hashing as well where appropriate
+
+-- The stats key should be hashed since the max length is too small
+insert overwrite table stats_part partition (ds='2010-04-08', hr = '13') select key, value from src;
+
+desc formatted stats_part partition (ds='2010-04-08', hr = '13');
+
+set hive.stats.key.prefix.max.length=200;
+
+-- The stats key should not be hashed since the max length is large enough
+insert overwrite table stats_part partition (ds='2010-04-08', hr = '13') select key, value from src;
+
+desc formatted stats_part partition (ds='2010-04-08', hr = '13');
+
+set hive.stats.key.prefix.max.length=-1;
+
+-- The stats key should not be hashed since negative values should imply hashing is turned off
+insert overwrite table stats_part partition (ds='2010-04-08', hr = '13') select key, value from src;
+
+desc formatted stats_part partition (ds='2010-04-08', hr = '13');
+
+
+set hive.stats.dbclass=dummy;
+set hive.stats.default.publisher=org.apache.hadoop.hive.ql.stats.DummyStatsPublisher;
+set hive.stats.default.aggregator=org.apache.hadoop.hive.ql.stats.KeyVerifyingStatsAggregator;
+set hive.stats.key.prefix.max.length=0;
+
+-- Do the same for dynamic partitions
+
+-- The stats key should be hashed since the max length is too small
+insert overwrite table stats_part partition (ds='2010-04-08', hr) select key, value, '13' from src;
+
+set hive.stats.key.prefix.max.length=200;
+
+-- The stats key should not be hashed since the max length is large enough
+insert overwrite table stats_part partition (ds='2010-04-08', hr) select key, value, '13' from src;
+
+set hive.stats.key.prefix.max.length=-1;
+
+-- The stats key should not be hashed since negative values should imply hashing is turned off
+insert overwrite table stats_part partition (ds='2010-04-08', hr) select key, value, '13' from src;
+
+set hive.stats.dbclass=jdbc:derby;
+set hive.stats.default.publisher=;
+set hive.stats.default.aggregator=;
+
+set hive.stats.key.prefix.max.length=0;
+
+-- Run the tests again and verify the stats are correct, this should verify that the stats publisher
+-- is hashing as well where appropriate
+
+-- The stats key should be hashed since the max length is too small
+insert overwrite table stats_part partition (ds='2010-04-08', hr) select key, value, '13' from src;
+
+desc formatted stats_part partition (ds='2010-04-08', hr = '13');
+
+set hive.stats.key.prefix.max.length=200;
+
+-- The stats key should not be hashed since the max length is large enough
+insert overwrite table stats_part partition (ds='2010-04-08', hr) select key, value, '13' from src;
+
+desc formatted stats_part partition (ds='2010-04-08', hr = '13');
+
+set hive.stats.key.prefix.max.length=-1;
+
+-- The stats key should not be hashed since negative values should imply hashing is turned off
+insert overwrite table stats_part partition (ds='2010-04-08', hr) select key, value, '13' from src;
+
+desc formatted stats_part partition (ds='2010-04-08', hr = '13');