diff --git a/common/src/java/org/apache/hadoop/hive/conf/HiveConf.java b/common/src/java/org/apache/hadoop/hive/conf/HiveConf.java index 99c26ce..98ab59b 100644 --- a/common/src/java/org/apache/hadoop/hive/conf/HiveConf.java +++ b/common/src/java/org/apache/hadoop/hive/conf/HiveConf.java @@ -3346,6 +3346,7 @@ private static void populateLlapDaemonVarsSet(Set llapDaemonVarsSetLocal SPARK_DYNAMIC_PARTITION_PRUNING_MAX_DATA_SIZE( "hive.spark.dynamic.partition.pruning.max.data.size", 100*1024*1024L, "Maximum total data size in dynamic pruning."), + SPARK_DYNANMIC_PARTITION_PRUNING_COMBINE("hive.spark.dynamic.partition.pruning.combine", false, "When there are similar map trees in the spark work, combine them"), SPARK_USE_GROUPBY_SHUFFLE( "hive.spark.use.groupby.shuffle", true, "Spark groupByKey transformation has better performance but uses unbounded memory." + diff --git a/itests/src/test/resources/testconfiguration.properties b/itests/src/test/resources/testconfiguration.properties index 5ab3076..11e15cb 100644 --- a/itests/src/test/resources/testconfiguration.properties +++ b/itests/src/test/resources/testconfiguration.properties @@ -1370,6 +1370,7 @@ spark.query.files=add_part_multiple.q, \ spark.only.query.files=spark_combine_equivalent_work.q,\ spark_dynamic_partition_pruning.q,\ spark_dynamic_partition_pruning_2.q,\ + spark_dynamic_partition_pruning_combine.q,\ spark_explainuser_1.q,\ spark_vectorized_dynamic_partition_pruning.q,\ spark_use_file_size_for_mapjoin.q,\ diff --git a/ql/src/java/org/apache/hadoop/hive/ql/parse/spark/GenSparkProcContext.java b/ql/src/java/org/apache/hadoop/hive/ql/parse/spark/GenSparkProcContext.java index b0ab495..c59541d 100644 --- a/ql/src/java/org/apache/hadoop/hive/ql/parse/spark/GenSparkProcContext.java +++ b/ql/src/java/org/apache/hadoop/hive/ql/parse/spark/GenSparkProcContext.java @@ -144,6 +144,9 @@ // The set of TableScanOperators for pruning OP trees public final Set> clonedPruningTableScanSet; + public final Map pruningTableScanMap; + + public final Set removeWorkSet; @SuppressWarnings("unchecked") public GenSparkProcContext(HiveConf conf, @@ -184,5 +187,7 @@ public GenSparkProcContext(HiveConf conf, this.fileSinkMap = new LinkedHashMap>(); this.pruningSinkSet = new LinkedHashSet>(); this.clonedPruningTableScanSet = new LinkedHashSet>(); + this.pruningTableScanMap = new LinkedHashMap(); + this.removeWorkSet = new LinkedHashSet(); } } diff --git a/ql/src/java/org/apache/hadoop/hive/ql/parse/spark/GenSparkUtils.java b/ql/src/java/org/apache/hadoop/hive/ql/parse/spark/GenSparkUtils.java index d0a82af..0b22483 100644 --- a/ql/src/java/org/apache/hadoop/hive/ql/parse/spark/GenSparkUtils.java +++ b/ql/src/java/org/apache/hadoop/hive/ql/parse/spark/GenSparkUtils.java @@ -44,6 +44,7 @@ import org.apache.hadoop.hive.ql.exec.SerializationUtilities; import org.apache.hadoop.hive.ql.exec.TableScanOperator; import org.apache.hadoop.hive.ql.exec.UnionOperator; +import org.apache.hadoop.hive.ql.exec.Utilities; import org.apache.hadoop.hive.ql.exec.spark.SparkUtilities; import org.apache.hadoop.hive.ql.optimizer.GenMapRedUtils; import org.apache.hadoop.hive.ql.optimizer.spark.SparkPartitionPruningSinkDesc; @@ -155,8 +156,8 @@ public MapWork createMapWork(GenSparkProcContext context, Operator root, // map work starts with table scan operators Preconditions.checkArgument(root instanceof TableScanOperator, - "AssertionError: expected root to be an instance of TableScanOperator, but was " - + root.getClass().getName()); + "AssertionError: expected root to be an instance of TableScanOperator, but was " + + root.getClass().getName()); String alias = ((TableScanOperator) root).getConf().getAlias(); if (!deferSetup) { @@ -583,4 +584,54 @@ public BaseWork getEnclosingWork(Operator op, GenSparkProcContext procCtx) { } return null; } + + /** + * HIVE-11297: Combine those operator trees which start from the same table scan operator in dynamic partition pruning. + * + * @param op SparkPartitionPruningSinkOperator + */ + public void combineSimilarPruningSinkSet(GenSparkProcContext procCtx, Operator op) { + List> roots = new ArrayList>(); + OperatorUtils.findRoots(op, roots); + if (roots.size() == 1 && roots.get(0) instanceof TableScanOperator) { + TableScanOperator ts = (TableScanOperator) roots.get(0); + if (!procCtx.pruningTableScanMap.containsKey(ts.getOperatorId())) { + BaseWork mapWork = procCtx.rootToWorkMap.get(ts); + procCtx.pruningTableScanMap.put(ts.getOperatorId(), mapWork); + } else { + BaseWork mapWork1 = procCtx.pruningTableScanMap.get(ts.getOperatorId()); + BaseWork mapWork2 = procCtx.rootToWorkMap.get(ts); + combineTwoMapWork(mapWork1, mapWork2); + procCtx.removeWorkSet.add(mapWork2); + } + } + } + + private void combineTwoMapWork(BaseWork mapWork1, BaseWork mapWork2) { + Set> operatorSet1 = mapWork1.getAllOperators(); + Map operatorMap = new HashMap(); + for (Operator op : operatorSet1) { + operatorMap.put(op.getOperatorId(), op); + } + Set> operatorSet2 = mapWork2.getAllOperators(); + for (Operator op : operatorSet2) { + List parentOpList = op.getParentOperators(); + for (Operator parentOp2 : parentOpList) { + if (!operatorMap.containsKey(parentOp2.getOperatorId())) { + LOG.error("Parent operator " + parentOp2.getName() + "was not in the first operator map!"); + } else { + Operator parentOp1 = operatorMap.get(parentOp2.getOperatorId()); + if (!operatorMap.containsKey(op.getOperatorId())) { + parentOp1.getChildOperators().add(op); + ArrayList newChildrenOps = new ArrayList(); + for (Object element : parentOp1.getChildOperators()) { + newChildrenOps.add(element); + } + parentOp1.setChildOperators(newChildrenOps); + } + op.setParentOperators(Utilities.makeList(parentOp1)); + } + } + } + } } diff --git a/ql/src/java/org/apache/hadoop/hive/ql/parse/spark/SparkCompiler.java b/ql/src/java/org/apache/hadoop/hive/ql/parse/spark/SparkCompiler.java index 682b987..e5f2a89 100644 --- a/ql/src/java/org/apache/hadoop/hive/ql/parse/spark/SparkCompiler.java +++ b/ql/src/java/org/apache/hadoop/hive/ql/parse/spark/SparkCompiler.java @@ -386,6 +386,20 @@ protected void generateTaskTree(List> rootTasks, Pa for (Operator prunerSink : procCtx.pruningSinkSet) { utils.processPartitionPruningSink(procCtx, (SparkPartitionPruningSinkOperator) prunerSink); } + boolean combine = conf.getBoolVar(HiveConf.ConfVars.SPARK_DYNANMIC_PARTITION_PRUNING_COMBINE); + if (combine) { + // Combine the similar map tasks which contain partition pruning sinks + for (Operator prunerSink : procCtx.pruningSinkSet) { + utils.combineSimilarPruningSinkSet(procCtx, prunerSink); + } + // Remove extra map tasks after combining similar map tasks + for (Task rootTask : procCtx.rootTasks) { + SparkWork sparkWork = (SparkWork)rootTask.getWork(); + for(BaseWork removeWork : procCtx.removeWorkSet) { + sparkWork.remove(removeWork); + } + } + } PERF_LOGGER.PerfLogEnd(CLASS_NAME, PerfLogger.SPARK_GENERATE_TASK_TREE); } diff --git a/ql/src/test/queries/clientpositive/spark_dynamic_partition_pruning_combine.q b/ql/src/test/queries/clientpositive/spark_dynamic_partition_pruning_combine.q new file mode 100644 index 0000000..e2fdea1 --- /dev/null +++ b/ql/src/test/queries/clientpositive/spark_dynamic_partition_pruning_combine.q @@ -0,0 +1,18 @@ +set hive.optimize.ppd=true; +set hive.ppd.remove.duplicatefilters=true; +set hive.spark.dynamic.partition.pruning=true; +set hive.optimize.metadataonly=false; +set hive.optimize.index.filter=true; +set hive.strict.checks.cartesian.product=false; +set hive.spark.dynamic.partition.pruning=true; +set hive.spark.dynamic.partition.pruning.combine=true; + + +-- SORT_QUERY_RESULTS +create table srcpart_date_hour as select ds as ds, ds as `date`, hr as hr, hr as hour from srcpart group by ds, hr; +-- multiple columns single source +EXPLAIN select count(*) from srcpart join srcpart_date_hour on (srcpart.ds = srcpart_date_hour.ds and srcpart.hr = srcpart_date_hour.hr) where srcpart_date_hour.`date` = '2008-04-08' and srcpart_date_hour.hour = 11; +select count(*) from srcpart join srcpart_date_hour on (srcpart.ds = srcpart_date_hour.ds and srcpart.hr = srcpart_date_hour.hr) where srcpart_date_hour.`date` = '2008-04-08' and srcpart_date_hour.hour = 11; +set hive.spark.dynamic.partition.pruning.combine=false; +EXPLAIN select count(*) from srcpart join srcpart_date_hour on (srcpart.ds = srcpart_date_hour.ds and srcpart.hr = srcpart_date_hour.hr) where srcpart_date_hour.`date` = '2008-04-08' and srcpart_date_hour.hour = 11; +select count(*) from srcpart join srcpart_date_hour on (srcpart.ds = srcpart_date_hour.ds and srcpart.hr = srcpart_date_hour.hr) where srcpart_date_hour.`date` = '2008-04-08' and srcpart_date_hour.hour = 11; diff --git a/ql/src/test/results/clientpositive/spark/spark_dynamic_partition_pruning_combine.q.out b/ql/src/test/results/clientpositive/spark/spark_dynamic_partition_pruning_combine.q.out new file mode 100644 index 0000000..ac42e7c --- /dev/null +++ b/ql/src/test/results/clientpositive/spark/spark_dynamic_partition_pruning_combine.q.out @@ -0,0 +1,341 @@ +PREHOOK: query: create table srcpart_date_hour as select ds as ds, ds as `date`, hr as hr, hr as hour from srcpart group by ds, hr +PREHOOK: type: CREATETABLE_AS_SELECT +PREHOOK: Input: default@srcpart +PREHOOK: Input: default@srcpart@ds=2008-04-08/hr=11 +PREHOOK: Input: default@srcpart@ds=2008-04-08/hr=12 +PREHOOK: Input: default@srcpart@ds=2008-04-09/hr=11 +PREHOOK: Input: default@srcpart@ds=2008-04-09/hr=12 +PREHOOK: Output: database:default +PREHOOK: Output: default@srcpart_date_hour +POSTHOOK: query: create table srcpart_date_hour as select ds as ds, ds as `date`, hr as hr, hr as hour from srcpart group by ds, hr +POSTHOOK: type: CREATETABLE_AS_SELECT +POSTHOOK: Input: default@srcpart +POSTHOOK: Input: default@srcpart@ds=2008-04-08/hr=11 +POSTHOOK: Input: default@srcpart@ds=2008-04-08/hr=12 +POSTHOOK: Input: default@srcpart@ds=2008-04-09/hr=11 +POSTHOOK: Input: default@srcpart@ds=2008-04-09/hr=12 +POSTHOOK: Output: database:default +POSTHOOK: Output: default@srcpart_date_hour +POSTHOOK: Lineage: srcpart_date_hour.date SIMPLE [(srcpart)srcpart.FieldSchema(name:ds, type:string, comment:null), ] +POSTHOOK: Lineage: srcpart_date_hour.ds SIMPLE [(srcpart)srcpart.FieldSchema(name:ds, type:string, comment:null), ] +POSTHOOK: Lineage: srcpart_date_hour.hour SIMPLE [(srcpart)srcpart.FieldSchema(name:hr, type:string, comment:null), ] +POSTHOOK: Lineage: srcpart_date_hour.hr SIMPLE [(srcpart)srcpart.FieldSchema(name:hr, type:string, comment:null), ] +PREHOOK: query: EXPLAIN select count(*) from srcpart join srcpart_date_hour on (srcpart.ds = srcpart_date_hour.ds and srcpart.hr = srcpart_date_hour.hr) where srcpart_date_hour.`date` = '2008-04-08' and srcpart_date_hour.hour = 11 +PREHOOK: type: QUERY +POSTHOOK: query: EXPLAIN select count(*) from srcpart join srcpart_date_hour on (srcpart.ds = srcpart_date_hour.ds and srcpart.hr = srcpart_date_hour.hr) where srcpart_date_hour.`date` = '2008-04-08' and srcpart_date_hour.hour = 11 +POSTHOOK: type: QUERY +STAGE DEPENDENCIES: + Stage-2 is a root stage + Stage-1 depends on stages: Stage-2 + Stage-0 depends on stages: Stage-1 + +STAGE PLANS: + Stage: Stage-2 + Spark +#### A masked pattern was here #### + Vertices: + Map 5 + Map Operator Tree: + TableScan + alias: srcpart_date_hour + filterExpr: ((date = '2008-04-08') and (UDFToDouble(hour) = 11.0) and ds is not null and hr is not null) (type: boolean) + Statistics: Num rows: 4 Data size: 108 Basic stats: COMPLETE Column stats: NONE + Filter Operator + predicate: ((date = '2008-04-08') and (UDFToDouble(hour) = 11.0) and ds is not null and hr is not null) (type: boolean) + Statistics: Num rows: 1 Data size: 27 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: ds (type: string), hr (type: string) + outputColumnNames: _col0, _col2 + Statistics: Num rows: 1 Data size: 27 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: _col0 (type: string) + outputColumnNames: _col0 + Statistics: Num rows: 1 Data size: 27 Basic stats: COMPLETE Column stats: NONE + Group By Operator + keys: _col0 (type: string) + mode: hash + outputColumnNames: _col0 + Statistics: Num rows: 1 Data size: 27 Basic stats: COMPLETE Column stats: NONE + Spark Partition Pruning Sink Operator + partition key expr: ds + Statistics: Num rows: 1 Data size: 27 Basic stats: COMPLETE Column stats: NONE + target column name: ds + target work: Map 1 + Select Operator + expressions: _col2 (type: string) + outputColumnNames: _col0 + Statistics: Num rows: 1 Data size: 27 Basic stats: COMPLETE Column stats: NONE + Group By Operator + keys: _col0 (type: string) + mode: hash + outputColumnNames: _col0 + Statistics: Num rows: 1 Data size: 27 Basic stats: COMPLETE Column stats: NONE + Spark Partition Pruning Sink Operator + partition key expr: hr + Statistics: Num rows: 1 Data size: 27 Basic stats: COMPLETE Column stats: NONE + target column name: hr + target work: Map 1 + + Stage: Stage-1 + Spark + Edges: + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 2), Map 4 (PARTITION-LEVEL SORT, 2) + Reducer 3 <- Reducer 2 (GROUP, 1) +#### A masked pattern was here #### + Vertices: + Map 1 + Map Operator Tree: + TableScan + alias: srcpart + Statistics: Num rows: 2000 Data size: 21248 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: ds (type: string), hr (type: string) + outputColumnNames: _col0, _col1 + Statistics: Num rows: 2000 Data size: 21248 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + key expressions: _col0 (type: string), _col1 (type: string) + sort order: ++ + Map-reduce partition columns: _col0 (type: string), _col1 (type: string) + Statistics: Num rows: 2000 Data size: 21248 Basic stats: COMPLETE Column stats: NONE + Map 4 + Map Operator Tree: + TableScan + alias: srcpart_date_hour + filterExpr: ((date = '2008-04-08') and (UDFToDouble(hour) = 11.0) and ds is not null and hr is not null) (type: boolean) + Statistics: Num rows: 4 Data size: 108 Basic stats: COMPLETE Column stats: NONE + Filter Operator + predicate: ((date = '2008-04-08') and (UDFToDouble(hour) = 11.0) and ds is not null and hr is not null) (type: boolean) + Statistics: Num rows: 1 Data size: 27 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: ds (type: string), hr (type: string) + outputColumnNames: _col0, _col2 + Statistics: Num rows: 1 Data size: 27 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + key expressions: _col0 (type: string), _col2 (type: string) + sort order: ++ + Map-reduce partition columns: _col0 (type: string), _col2 (type: string) + Statistics: Num rows: 1 Data size: 27 Basic stats: COMPLETE Column stats: NONE + Reducer 2 + Reduce Operator Tree: + Join Operator + condition map: + Inner Join 0 to 1 + keys: + 0 _col0 (type: string), _col1 (type: string) + 1 _col0 (type: string), _col2 (type: string) + Statistics: Num rows: 2200 Data size: 23372 Basic stats: COMPLETE Column stats: NONE + Group By Operator + aggregations: count() + mode: hash + outputColumnNames: _col0 + Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + sort order: + Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE + value expressions: _col0 (type: bigint) + Reducer 3 + Reduce Operator Tree: + Group By Operator + aggregations: count(VALUE._col0) + mode: mergepartial + outputColumnNames: _col0 + Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE + File Output Operator + compressed: false + Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE + table: + input format: org.apache.hadoop.mapred.SequenceFileInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + + Stage: Stage-0 + Fetch Operator + limit: -1 + Processor Tree: + ListSink + +PREHOOK: query: select count(*) from srcpart join srcpart_date_hour on (srcpart.ds = srcpart_date_hour.ds and srcpart.hr = srcpart_date_hour.hr) where srcpart_date_hour.`date` = '2008-04-08' and srcpart_date_hour.hour = 11 +PREHOOK: type: QUERY +PREHOOK: Input: default@srcpart +PREHOOK: Input: default@srcpart@ds=2008-04-08/hr=11 +PREHOOK: Input: default@srcpart@ds=2008-04-08/hr=12 +PREHOOK: Input: default@srcpart@ds=2008-04-09/hr=11 +PREHOOK: Input: default@srcpart@ds=2008-04-09/hr=12 +PREHOOK: Input: default@srcpart_date_hour +#### A masked pattern was here #### +POSTHOOK: query: select count(*) from srcpart join srcpart_date_hour on (srcpart.ds = srcpart_date_hour.ds and srcpart.hr = srcpart_date_hour.hr) where srcpart_date_hour.`date` = '2008-04-08' and srcpart_date_hour.hour = 11 +POSTHOOK: type: QUERY +POSTHOOK: Input: default@srcpart +POSTHOOK: Input: default@srcpart@ds=2008-04-08/hr=11 +POSTHOOK: Input: default@srcpart@ds=2008-04-08/hr=12 +POSTHOOK: Input: default@srcpart@ds=2008-04-09/hr=11 +POSTHOOK: Input: default@srcpart@ds=2008-04-09/hr=12 +POSTHOOK: Input: default@srcpart_date_hour +#### A masked pattern was here #### +500 +PREHOOK: query: EXPLAIN select count(*) from srcpart join srcpart_date_hour on (srcpart.ds = srcpart_date_hour.ds and srcpart.hr = srcpart_date_hour.hr) where srcpart_date_hour.`date` = '2008-04-08' and srcpart_date_hour.hour = 11 +PREHOOK: type: QUERY +POSTHOOK: query: EXPLAIN select count(*) from srcpart join srcpart_date_hour on (srcpart.ds = srcpart_date_hour.ds and srcpart.hr = srcpart_date_hour.hr) where srcpart_date_hour.`date` = '2008-04-08' and srcpart_date_hour.hour = 11 +POSTHOOK: type: QUERY +STAGE DEPENDENCIES: + Stage-2 is a root stage + Stage-1 depends on stages: Stage-2 + Stage-0 depends on stages: Stage-1 + +STAGE PLANS: + Stage: Stage-2 + Spark +#### A masked pattern was here #### + Vertices: + Map 5 + Map Operator Tree: + TableScan + alias: srcpart_date_hour + filterExpr: ((date = '2008-04-08') and (UDFToDouble(hour) = 11.0) and ds is not null and hr is not null) (type: boolean) + Statistics: Num rows: 4 Data size: 108 Basic stats: COMPLETE Column stats: NONE + Filter Operator + predicate: ((date = '2008-04-08') and (UDFToDouble(hour) = 11.0) and ds is not null and hr is not null) (type: boolean) + Statistics: Num rows: 1 Data size: 27 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: ds (type: string), hr (type: string) + outputColumnNames: _col0, _col2 + Statistics: Num rows: 1 Data size: 27 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: _col0 (type: string) + outputColumnNames: _col0 + Statistics: Num rows: 1 Data size: 27 Basic stats: COMPLETE Column stats: NONE + Group By Operator + keys: _col0 (type: string) + mode: hash + outputColumnNames: _col0 + Statistics: Num rows: 1 Data size: 27 Basic stats: COMPLETE Column stats: NONE + Spark Partition Pruning Sink Operator + partition key expr: ds + Statistics: Num rows: 1 Data size: 27 Basic stats: COMPLETE Column stats: NONE + target column name: ds + target work: Map 1 + Map 6 + Map Operator Tree: + TableScan + alias: srcpart_date_hour + filterExpr: ((date = '2008-04-08') and (UDFToDouble(hour) = 11.0) and ds is not null and hr is not null) (type: boolean) + Statistics: Num rows: 4 Data size: 108 Basic stats: COMPLETE Column stats: NONE + Filter Operator + predicate: ((date = '2008-04-08') and (UDFToDouble(hour) = 11.0) and ds is not null and hr is not null) (type: boolean) + Statistics: Num rows: 1 Data size: 27 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: ds (type: string), hr (type: string) + outputColumnNames: _col0, _col2 + Statistics: Num rows: 1 Data size: 27 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: _col2 (type: string) + outputColumnNames: _col0 + Statistics: Num rows: 1 Data size: 27 Basic stats: COMPLETE Column stats: NONE + Group By Operator + keys: _col0 (type: string) + mode: hash + outputColumnNames: _col0 + Statistics: Num rows: 1 Data size: 27 Basic stats: COMPLETE Column stats: NONE + Spark Partition Pruning Sink Operator + partition key expr: hr + Statistics: Num rows: 1 Data size: 27 Basic stats: COMPLETE Column stats: NONE + target column name: hr + target work: Map 1 + + Stage: Stage-1 + Spark + Edges: + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 2), Map 4 (PARTITION-LEVEL SORT, 2) + Reducer 3 <- Reducer 2 (GROUP, 1) +#### A masked pattern was here #### + Vertices: + Map 1 + Map Operator Tree: + TableScan + alias: srcpart + Statistics: Num rows: 2000 Data size: 21248 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: ds (type: string), hr (type: string) + outputColumnNames: _col0, _col1 + Statistics: Num rows: 2000 Data size: 21248 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + key expressions: _col0 (type: string), _col1 (type: string) + sort order: ++ + Map-reduce partition columns: _col0 (type: string), _col1 (type: string) + Statistics: Num rows: 2000 Data size: 21248 Basic stats: COMPLETE Column stats: NONE + Map 4 + Map Operator Tree: + TableScan + alias: srcpart_date_hour + filterExpr: ((date = '2008-04-08') and (UDFToDouble(hour) = 11.0) and ds is not null and hr is not null) (type: boolean) + Statistics: Num rows: 4 Data size: 108 Basic stats: COMPLETE Column stats: NONE + Filter Operator + predicate: ((date = '2008-04-08') and (UDFToDouble(hour) = 11.0) and ds is not null and hr is not null) (type: boolean) + Statistics: Num rows: 1 Data size: 27 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: ds (type: string), hr (type: string) + outputColumnNames: _col0, _col2 + Statistics: Num rows: 1 Data size: 27 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + key expressions: _col0 (type: string), _col2 (type: string) + sort order: ++ + Map-reduce partition columns: _col0 (type: string), _col2 (type: string) + Statistics: Num rows: 1 Data size: 27 Basic stats: COMPLETE Column stats: NONE + Reducer 2 + Reduce Operator Tree: + Join Operator + condition map: + Inner Join 0 to 1 + keys: + 0 _col0 (type: string), _col1 (type: string) + 1 _col0 (type: string), _col2 (type: string) + Statistics: Num rows: 2200 Data size: 23372 Basic stats: COMPLETE Column stats: NONE + Group By Operator + aggregations: count() + mode: hash + outputColumnNames: _col0 + Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + sort order: + Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE + value expressions: _col0 (type: bigint) + Reducer 3 + Reduce Operator Tree: + Group By Operator + aggregations: count(VALUE._col0) + mode: mergepartial + outputColumnNames: _col0 + Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE + File Output Operator + compressed: false + Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE + table: + input format: org.apache.hadoop.mapred.SequenceFileInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + + Stage: Stage-0 + Fetch Operator + limit: -1 + Processor Tree: + ListSink + +PREHOOK: query: select count(*) from srcpart join srcpart_date_hour on (srcpart.ds = srcpart_date_hour.ds and srcpart.hr = srcpart_date_hour.hr) where srcpart_date_hour.`date` = '2008-04-08' and srcpart_date_hour.hour = 11 +PREHOOK: type: QUERY +PREHOOK: Input: default@srcpart +PREHOOK: Input: default@srcpart@ds=2008-04-08/hr=11 +PREHOOK: Input: default@srcpart@ds=2008-04-08/hr=12 +PREHOOK: Input: default@srcpart@ds=2008-04-09/hr=11 +PREHOOK: Input: default@srcpart@ds=2008-04-09/hr=12 +PREHOOK: Input: default@srcpart_date_hour +#### A masked pattern was here #### +POSTHOOK: query: select count(*) from srcpart join srcpart_date_hour on (srcpart.ds = srcpart_date_hour.ds and srcpart.hr = srcpart_date_hour.hr) where srcpart_date_hour.`date` = '2008-04-08' and srcpart_date_hour.hour = 11 +POSTHOOK: type: QUERY +POSTHOOK: Input: default@srcpart +POSTHOOK: Input: default@srcpart@ds=2008-04-08/hr=11 +POSTHOOK: Input: default@srcpart@ds=2008-04-08/hr=12 +POSTHOOK: Input: default@srcpart@ds=2008-04-09/hr=11 +POSTHOOK: Input: default@srcpart@ds=2008-04-09/hr=12 +POSTHOOK: Input: default@srcpart_date_hour +#### A masked pattern was here #### +500