diff --git hbase-handler/src/test/results/positive/hbasestats.q.out hbase-handler/src/test/results/positive/hbasestats.q.out index 4724ad6c3d..e2061918dc 100644 --- hbase-handler/src/test/results/positive/hbasestats.q.out +++ hbase-handler/src/test/results/positive/hbasestats.q.out @@ -273,12 +273,45 @@ PREHOOK: type: QUERY POSTHOOK: query: explain select count(*) from users POSTHOOK: type: QUERY STAGE DEPENDENCIES: - Stage-0 is a root stage + Stage-1 is a root stage + Stage-0 depends on stages: Stage-1 STAGE PLANS: + Stage: Stage-1 + Map Reduce + Map Operator Tree: + TableScan + alias: users + Statistics: Num rows: 2 Data size: 0 Basic stats: PARTIAL Column stats: COMPLETE + Select Operator + Statistics: Num rows: 2 Data size: 0 Basic stats: PARTIAL Column stats: COMPLETE + Group By Operator + aggregations: count() + mode: hash + outputColumnNames: _col0 + Statistics: Num rows: 1 Data size: 8 Basic stats: PARTIAL Column stats: COMPLETE + Reduce Output Operator + sort order: + Statistics: Num rows: 1 Data size: 8 Basic stats: PARTIAL Column stats: COMPLETE + value expressions: _col0 (type: bigint) + Execution mode: vectorized + Reduce Operator Tree: + Group By Operator + aggregations: count(VALUE._col0) + mode: mergepartial + outputColumnNames: _col0 + Statistics: Num rows: 1 Data size: 8 Basic stats: PARTIAL Column stats: COMPLETE + File Output Operator + compressed: false + Statistics: Num rows: 1 Data size: 8 Basic stats: PARTIAL 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-0 Fetch Operator - limit: 1 + limit: -1 Processor Tree: ListSink diff --git ql/src/java/org/apache/hadoop/hive/ql/optimizer/StatsOptimizer.java ql/src/java/org/apache/hadoop/hive/ql/optimizer/StatsOptimizer.java index a574372dbe..5788d4906e 100644 --- ql/src/java/org/apache/hadoop/hive/ql/optimizer/StatsOptimizer.java +++ ql/src/java/org/apache/hadoop/hive/ql/optimizer/StatsOptimizer.java @@ -17,16 +17,7 @@ */ package org.apache.hadoop.hive.ql.optimizer; -import java.sql.Date; -import java.util.ArrayList; -import java.util.Collection; -import java.util.HashMap; -import java.util.LinkedHashMap; -import java.util.List; -import java.util.Map; -import java.util.Set; -import java.util.Stack; - +import com.google.common.collect.Lists; import org.apache.hadoop.hive.common.StatsSetupConst; import org.apache.hadoop.hive.common.type.HiveDecimal; import org.apache.hadoop.hive.metastore.api.ColumnStatisticsData; @@ -75,7 +66,6 @@ import org.apache.hadoop.hive.ql.udf.generic.GenericUDAFSum; import org.apache.hadoop.hive.serde.serdeConstants; import org.apache.hadoop.hive.serde2.io.DateWritable; -import org.apache.hadoop.hive.serde2.io.TimestampWritable; import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspector; import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspectorFactory; import org.apache.hadoop.hive.serde2.objectinspector.PrimitiveObjectInspector.PrimitiveCategory; @@ -85,7 +75,14 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import com.google.common.collect.Lists; +import java.util.ArrayList; +import java.util.Collection; +import java.util.HashMap; +import java.util.LinkedHashMap; +import java.util.List; +import java.util.Map; +import java.util.Set; +import java.util.Stack; /** There is a set of queries which can be answered entirely from statistics stored in metastore. @@ -179,7 +176,7 @@ public MetaDataProcessor (ParseContext pctx) { abstract Object cast(double doubleValue); } - + enum DateSubType { DAYS {@Override Object cast(long longValue) { return (new DateWritable((int)longValue)).get();} @@ -289,6 +286,10 @@ public Object process(Node nd, Stack stack, NodeProcessorCtx procCtx, Logger.info("Table " + tbl.getTableName() + " is external. Skip StatsOptimizer."); return null; } + if (MetaStoreUtils.isNonNativeTable(tbl.getTTable())) { + Logger.info("Table " + tbl.getTableName() + " is non Native table. Skip StatsOptimizer."); + return null; + } if (AcidUtils.isTransactionalTable(tbl)) { //todo: should this be OK for MM table? Logger.info("Table " + tbl.getTableName() + " is ACID table. Skip StatsOptimizer."); @@ -553,7 +554,7 @@ else if (udaf instanceof GenericUDAFCount) { case Date: { DateColumnStatsData dstats = statData.getDateStats(); if (dstats.isSetHighValue()) { - oneRow.add(DateSubType.DAYS.cast(dstats.getHighValue().getDaysSinceEpoch())); + oneRow.add(DateSubType.DAYS.cast(dstats.getHighValue().getDaysSinceEpoch())); } else { oneRow.add(null); } @@ -690,7 +691,7 @@ else if (udaf instanceof GenericUDAFCount) { case Date: { DateColumnStatsData dstats = statData.getDateStats(); if (dstats.isSetLowValue()) { - oneRow.add(DateSubType.DAYS.cast(dstats.getLowValue().getDaysSinceEpoch())); + oneRow.add(DateSubType.DAYS.cast(dstats.getLowValue().getDaysSinceEpoch())); } else { oneRow.add(null); } diff --git ql/src/test/queries/clientpositive/druidmini_expressions.q ql/src/test/queries/clientpositive/druidmini_expressions.q index 01e2d6faa0..722936ba0a 100644 --- ql/src/test/queries/clientpositive/druidmini_expressions.q +++ ql/src/test/queries/clientpositive/druidmini_expressions.q @@ -49,7 +49,6 @@ EXPLAIN SELECT SUM(cfloat + 1), CAST(SUM(cdouble + ctinyint) AS INTEGER), SUM(ct EXPLAIN SELECT cstring1 || '_'|| cstring2, substring(cstring2, 2, 3) as concat , upper(cstring2), lower(cstring1), SUM(cdouble) as s FROM druid_table_n0 WHERE cstring1 IS NOT NULL AND cstring2 IS NOT NULL AND cstring2 like 'Y%' GROUP BY cstring1 || '_'|| cstring2, substring(cstring2, 2, 3), upper(cstring2), lower(cstring1) ORDER BY concat DESC LIMIT 10; - - +explain extended select count(*) from (select `__time` from druid_table_n0 limit 1) as src ; DROP TABLE druid_table_n0; \ No newline at end of file diff --git ql/src/test/results/clientpositive/druid/druidmini_expressions.q.out ql/src/test/results/clientpositive/druid/druidmini_expressions.q.out index d670eaced9..dbaa3a6ef4 100644 --- ql/src/test/results/clientpositive/druid/druidmini_expressions.q.out +++ ql/src/test/results/clientpositive/druid/druidmini_expressions.q.out @@ -257,6 +257,155 @@ STAGE PLANS: outputColumnNames: _col0, _col1, _col2, _col3, _col4 ListSink +PREHOOK: query: explain extended select count(*) from (select `__time` from druid_table_n0 limit 1) as src +PREHOOK: type: QUERY +POSTHOOK: query: explain extended select count(*) from (select `__time` from druid_table_n0 limit 1) as src +POSTHOOK: type: QUERY +STAGE DEPENDENCIES: + Stage-1 is a root stage + Stage-0 depends on stages: Stage-1 + +STAGE PLANS: + Stage: Stage-1 + Tez +#### A masked pattern was here #### + Edges: + Reducer 2 <- Map 1 (CUSTOM_SIMPLE_EDGE) +#### A masked pattern was here #### + Vertices: + Map 1 + Map Operator Tree: + TableScan + alias: druid_table_n0 + properties: + druid.fieldNames vc + druid.fieldTypes int + druid.query.json {"queryType":"scan","dataSource":"default.druid_table_n0","intervals":["1900-01-01T00:00:00.000Z/3000-01-01T00:00:00.000Z"],"virtualColumns":[{"type":"expression","name":"vc","expression":"0","outputType":"LONG"}],"columns":["vc"],"resultFormat":"compactedList","limit":1} + druid.query.type scan + Statistics: Num rows: 9173 Data size: 0 Basic stats: PARTIAL Column stats: COMPLETE + GatherStats: false + Select Operator + Statistics: Num rows: 9173 Data size: 0 Basic stats: PARTIAL Column stats: COMPLETE + Group By Operator + aggregations: count() + mode: hash + outputColumnNames: _col0 + Statistics: Num rows: 1 Data size: 8 Basic stats: PARTIAL Column stats: COMPLETE + Reduce Output Operator + null sort order: + sort order: + Statistics: Num rows: 1 Data size: 8 Basic stats: PARTIAL Column stats: COMPLETE + tag: -1 + value expressions: _col0 (type: bigint) + auto parallelism: false + Path -> Alias: + hdfs://### HDFS PATH ### [druid_table_n0] + Path -> Partition: + hdfs://### HDFS PATH ### + Partition + base file name: druid_table_n0 + input format: org.apache.hadoop.hive.druid.io.DruidQueryBasedInputFormat + output format: org.apache.hadoop.hive.druid.io.DruidOutputFormat + properties: + COLUMN_STATS_ACCURATE {"BASIC_STATS":"true"} + bucket_count -1 + bucketing_version 2 + column.name.delimiter , + columns __time,cstring1,cstring2,cdouble,cfloat,ctinyint,csmallint,cint,cbigint,cboolean1,cboolean2 + columns.comments + columns.types timestamp with local time zone:string:string:double:float:tinyint:smallint:int:bigint:boolean:boolean + druid.datasource default.druid_table_n0 + druid.fieldNames vc + druid.fieldTypes int + druid.query.granularity MINUTE + druid.query.json {"queryType":"scan","dataSource":"default.druid_table_n0","intervals":["1900-01-01T00:00:00.000Z/3000-01-01T00:00:00.000Z"],"virtualColumns":[{"type":"expression","name":"vc","expression":"0","outputType":"LONG"}],"columns":["vc"],"resultFormat":"compactedList","limit":1} + druid.query.type scan + druid.segment.granularity HOUR +#### A masked pattern was here #### + location hdfs://### HDFS PATH ### + name default.druid_table_n0 + numFiles 0 + numRows 9173 + rawDataSize 0 + serialization.ddl struct druid_table_n0 { timestamp with local time zone __time, string cstring1, string cstring2, double cdouble, float cfloat, byte ctinyint, i16 csmallint, i32 cint, i64 cbigint, bool cboolean1, bool cboolean2} + serialization.format 1 + serialization.lib org.apache.hadoop.hive.druid.serde.DruidSerDe + storage_handler org.apache.hadoop.hive.druid.DruidStorageHandler + totalSize 0 +#### A masked pattern was here #### + serde: org.apache.hadoop.hive.druid.serde.DruidSerDe + + input format: org.apache.hadoop.hive.druid.io.DruidQueryBasedInputFormat + output format: org.apache.hadoop.hive.druid.io.DruidOutputFormat + properties: + COLUMN_STATS_ACCURATE {"BASIC_STATS":"true"} + bucket_count -1 + bucketing_version 2 + column.name.delimiter , + columns __time,cstring1,cstring2,cdouble,cfloat,ctinyint,csmallint,cint,cbigint,cboolean1,cboolean2 + columns.comments + columns.types timestamp with local time zone:string:string:double:float:tinyint:smallint:int:bigint:boolean:boolean + druid.datasource default.druid_table_n0 + druid.fieldNames vc + druid.fieldTypes int + druid.query.granularity MINUTE + druid.query.json {"queryType":"scan","dataSource":"default.druid_table_n0","intervals":["1900-01-01T00:00:00.000Z/3000-01-01T00:00:00.000Z"],"virtualColumns":[{"type":"expression","name":"vc","expression":"0","outputType":"LONG"}],"columns":["vc"],"resultFormat":"compactedList","limit":1} + druid.query.type scan + druid.segment.granularity HOUR +#### A masked pattern was here #### + location hdfs://### HDFS PATH ### + name default.druid_table_n0 + numFiles 0 + numRows 9173 + rawDataSize 0 + serialization.ddl struct druid_table_n0 { timestamp with local time zone __time, string cstring1, string cstring2, double cdouble, float cfloat, byte ctinyint, i16 csmallint, i32 cint, i64 cbigint, bool cboolean1, bool cboolean2} + serialization.format 1 + serialization.lib org.apache.hadoop.hive.druid.serde.DruidSerDe + storage_handler org.apache.hadoop.hive.druid.DruidStorageHandler + totalSize 0 +#### A masked pattern was here #### + serde: org.apache.hadoop.hive.druid.serde.DruidSerDe + name: default.druid_table_n0 + name: default.druid_table_n0 + Truncated Path -> Alias: + /druid_table_n0 [druid_table_n0] + Reducer 2 + Needs Tagging: false + Reduce Operator Tree: + Group By Operator + aggregations: count(VALUE._col0) + mode: mergepartial + outputColumnNames: _col0 + Statistics: Num rows: 1 Data size: 8 Basic stats: PARTIAL Column stats: COMPLETE + File Output Operator + compressed: false + GlobalTableId: 0 + directory: hdfs://### HDFS PATH ### + NumFilesPerFileSink: 1 + Statistics: Num rows: 1 Data size: 8 Basic stats: PARTIAL Column stats: COMPLETE + Stats Publishing Key Prefix: hdfs://### HDFS PATH ### + table: + input format: org.apache.hadoop.mapred.SequenceFileInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat + properties: + columns _col0 + columns.types bigint + escape.delim \ + hive.serialization.extend.additional.nesting.levels true + serialization.escape.crlf true + serialization.format 1 + serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + TotalFiles: 1 + GatherStats: false + MultiFileSpray: false + + Stage: Stage-0 + Fetch Operator + limit: -1 + Processor Tree: + ListSink + PREHOOK: query: DROP TABLE druid_table_n0 PREHOOK: type: DROPTABLE PREHOOK: Input: default@druid_table_n0