diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/spark/SparkDynamicPartitionPruner.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/spark/SparkDynamicPartitionPruner.java index eced0cd96c..2d3d7569f8 100644 --- a/ql/src/java/org/apache/hadoop/hive/ql/exec/spark/SparkDynamicPartitionPruner.java +++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/spark/SparkDynamicPartitionPruner.java @@ -83,15 +83,20 @@ public void initialize(MapWork work, JobConf jobConf) throws SerDeException { for (String id : sourceWorkIds) { List tables = work.getEventSourceTableDescMap().get(id); + // Real column name - on which the operation is being performed List columnNames = work.getEventSourceColumnNameMap().get(id); + // Column type + List columnTypes = work.getEventSourceColumnTypeMap().get(id); List partKeyExprs = work.getEventSourcePartKeyExprMap().get(id); Iterator cit = columnNames.iterator(); + Iterator typit = columnTypes.iterator(); Iterator pit = partKeyExprs.iterator(); for (TableDesc t : tables) { String columnName = cit.next(); + String columnType = typit.next(); ExprNodeDesc partKeyExpr = pit.next(); - SourceInfo si = new SourceInfo(t, partKeyExpr, columnName, jobConf); + SourceInfo si = new SourceInfo(t, partKeyExpr, columnName, columnType, jobConf); if (!sourceInfoMap.containsKey(id)) { sourceInfoMap.put(id, new ArrayList()); } @@ -171,7 +176,7 @@ private void prunePartitionSingleSource(SourceInfo info, MapWork work) ObjectInspector oi = PrimitiveObjectInspectorFactory.getPrimitiveWritableObjectInspector(TypeInfoFactory - .getPrimitiveTypeInfo(info.fieldInspector.getTypeName())); + .getPrimitiveTypeInfo(info.columnType)); ObjectInspectorConverters.Converter converter = ObjectInspectorConverters.getConverter( @@ -241,11 +246,13 @@ private void applyFilterToPartitions( final ObjectInspector fieldInspector; Set values = new HashSet(); final String columnName; + final String columnType; - SourceInfo(TableDesc table, ExprNodeDesc partKey, String columnName, JobConf jobConf) + SourceInfo(TableDesc table, ExprNodeDesc partKey, String columnName, String columnType, JobConf jobConf) throws SerDeException { this.partKey = partKey; this.columnName = columnName; + this.columnType = columnType; deserializer = ReflectionUtils.newInstance(table.getDeserializerClass(), null); deserializer.initialize(jobConf, table.getProperties()); diff --git a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/DynamicPartitionPruningOptimization.java b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/DynamicPartitionPruningOptimization.java index b5a97968d2..cf355d436a 100644 --- a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/DynamicPartitionPruningOptimization.java +++ b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/DynamicPartitionPruningOptimization.java @@ -461,6 +461,7 @@ private void generateEventOperatorPlan(DynamicListContext ctx, ParseContext pars desc.setTable(PlanUtils.getReduceValueTableDesc(PlanUtils .getFieldSchemasFromColumnList(keyExprs, "key"))); desc.setTargetColumnName(column); + desc.setTargetColumnType(columnType); desc.setPartKey(partKey); OperatorFactory.getAndMakeChild(desc, groupByOp); } diff --git a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/spark/SparkPartitionPruningSinkDesc.java b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/spark/SparkPartitionPruningSinkDesc.java index cc7822707b..0ca8c02081 100644 --- a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/spark/SparkPartitionPruningSinkDesc.java +++ b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/spark/SparkPartitionPruningSinkDesc.java @@ -31,6 +31,9 @@ // column in the target table that will be pruned against private String targetColumnName; + // type of target column + private String targetColumnType; + private TableDesc table; private transient TableScanOperator tableScan; @@ -68,7 +71,11 @@ public void setTableScan(TableScanOperator tableScan) { this.tableScan = tableScan; } - @Explain(displayName = "target column name") + @Explain(displayName = "Target column") + public String displayTargetColumn() { + return targetColumnName + " (" + targetColumnType + ")"; + } + public String getTargetColumnName() { return targetColumnName; } @@ -77,6 +84,14 @@ public void setTargetColumnName(String targetColumnName) { this.targetColumnName = targetColumnName; } + public String getTargetColumnType() { + return targetColumnType; + } + + public void setTargetColumnType(String columnType) { + this.targetColumnType = columnType; + } + public ExprNodeDesc getPartKey() { return partKey; } 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 08602e2ae8..ea20f89855 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 @@ -417,6 +417,12 @@ public void processPartitionPruningSink(GenSparkProcContext context, List columns = targetWork.getEventSourceColumnNameMap().get(sourceId); columns.add(desc.getTargetColumnName()); + if (!targetWork.getEventSourceColumnTypeMap().containsKey(sourceId)) { + targetWork.getEventSourceColumnTypeMap().put(sourceId, new LinkedList()); + } + List columnTypes = targetWork.getEventSourceColumnTypeMap().get(sourceId); + columnTypes.add(desc.getTargetColumnType()); + // store partition key expr in map-targetWork if (!targetWork.getEventSourcePartKeyExprMap().containsKey(sourceId)) { targetWork.getEventSourcePartKeyExprMap().put(sourceId, new LinkedList()); diff --git a/ql/src/test/queries/clientpositive/spark_dynamic_partition_pruning.q b/ql/src/test/queries/clientpositive/spark_dynamic_partition_pruning.q index 3b6810bcf7..78e7515b68 100644 --- a/ql/src/test/queries/clientpositive/spark_dynamic_partition_pruning.q +++ b/ql/src/test/queries/clientpositive/spark_dynamic_partition_pruning.q @@ -25,6 +25,22 @@ select count(*) from srcpart join srcpart_date on (srcpart.ds = srcpart_date.ds) set hive.spark.dynamic.partition.pruning=true; select count(*) from srcpart where ds = '2008-04-08'; +-- single column, single key, udf with typechange +EXPLAIN select count(*) from srcpart join srcpart_date on (day(srcpart.ds) = day(srcpart_date.ds)) where srcpart_date.`date` = '2008-04-08'; +select count(*) from srcpart join srcpart_date on (day(srcpart.ds) = day(srcpart_date.ds)) where srcpart_date.`date` = '2008-04-08'; +set hive.spark.dynamic.partition.pruning=false; +EXPLAIN select count(*) from srcpart join srcpart_date on (day(srcpart.ds) = day(srcpart_date.ds)) where srcpart_date.`date` = '2008-04-08'; +select count(*) from srcpart join srcpart_date on (day(srcpart.ds) = day(srcpart_date.ds)) where srcpart_date.`date` = '2008-04-08'; +set hive.spark.dynamic.partition.pruning=true; + +-- multiple udfs and casts +EXPLAIN select count(*) from srcpart join srcpart_date on abs(negative(cast(concat(cast(day(srcpart.ds) as string), "0") as bigint)) + 10) = abs(negative(cast(concat(cast(day(srcpart_date.ds) as string), "0") as bigint)) + 10) where srcpart_date.`date` = '2008-04-08'; +select count(*) from srcpart join srcpart_date on abs(negative(cast(concat(cast(day(srcpart.ds) as string), "0") as bigint)) + 10) = abs(negative(cast(concat(cast(day(srcpart_date.ds) as string), "0") as bigint)) + 10) where srcpart_date.`date` = '2008-04-08'; + +-- implicit type conversion between join columns +EXPLAIN select count(*) from srcpart join srcpart_date on cast(day(srcpart.ds) as smallint) = cast(day(srcpart_date.ds) as decimal) where srcpart_date.`date` = '2008-04-08'; +select count(*) from srcpart join srcpart_date on cast(day(srcpart.ds) as smallint) = cast(day(srcpart_date.ds) as decimal) where srcpart_date.`date` = '2008-04-08'; + -- multiple sources, single key EXPLAIN select count(*) from srcpart join srcpart_date on (srcpart.ds = srcpart_date.ds) join srcpart_hour on (srcpart.hr = srcpart_hour.hr) where srcpart_date.`date` = '2008-04-08' and srcpart_hour.hour = 11; @@ -121,6 +137,10 @@ EXPLAIN select count(*) from srcpart join srcpart_date on (srcpart.ds = srcpart_ select count(*) from srcpart join srcpart_date on (srcpart.ds = srcpart_date.ds) where srcpart_date.`date` = '2008-04-08'; select count(*) from srcpart where ds = '2008-04-08'; +-- single column, single key, udf with typechange +EXPLAIN select count(*) from srcpart join srcpart_date on (day(srcpart.ds) = day(srcpart_date.ds)) where srcpart_date.`date` = '2008-04-08'; +select count(*) from srcpart join srcpart_date on (day(srcpart.ds) = day(srcpart_date.ds)) where srcpart_date.`date` = '2008-04-08'; + -- multiple sources, single key EXPLAIN select count(*) from srcpart join srcpart_date on (srcpart.ds = srcpart_date.ds) join srcpart_hour on (srcpart.hr = srcpart_hour.hr) where srcpart_date.`date` = '2008-04-08' and srcpart_hour.hour = 11; @@ -150,6 +170,10 @@ EXPLAIN select count(*) from srcpart join (select ds as ds, ds as `date` from sr select count(*) from srcpart join (select ds as ds, ds as `date` from srcpart group by ds) s on (srcpart.ds = s.ds) where s.`date` = '2008-04-08'; select count(*) from srcpart where ds = '2008-04-08'; +-- single column, single key, udf with typechange +EXPLAIN select count(*) from srcpart join srcpart_date on (day(srcpart.ds) = day(srcpart_date.ds)) where srcpart_date.`date` = '2008-04-08'; +select count(*) from srcpart join srcpart_date on (day(srcpart.ds) = day(srcpart_date.ds)) where srcpart_date.`date` = '2008-04-08'; + -- left join EXPLAIN select count(*) from srcpart left join srcpart_date on (srcpart.ds = srcpart_date.ds) where srcpart_date.`date` = '2008-04-08'; EXPLAIN select count(*) from srcpart_date left join srcpart on (srcpart.ds = srcpart_date.ds) where srcpart_date.`date` = '2008-04-08'; diff --git a/ql/src/test/results/clientpositive/spark/spark_dynamic_partition_pruning.q.out b/ql/src/test/results/clientpositive/spark/spark_dynamic_partition_pruning.q.out index f1092c57ef..d9da7627b3 100644 --- a/ql/src/test/results/clientpositive/spark/spark_dynamic_partition_pruning.q.out +++ b/ql/src/test/results/clientpositive/spark/spark_dynamic_partition_pruning.q.out @@ -227,9 +227,9 @@ STAGE PLANS: outputColumnNames: _col0 Statistics: Num rows: 1 Data size: 21 Basic stats: COMPLETE Column stats: NONE Spark Partition Pruning Sink Operator + Target column: ds (string) partition key expr: ds Statistics: Num rows: 1 Data size: 21 Basic stats: COMPLETE Column stats: NONE - target column name: ds target work: Map 1 Stage: Stage-1 @@ -446,6 +446,533 @@ POSTHOOK: type: QUERY POSTHOOK: Input: default@srcpart #### A masked pattern was here #### 1000 +PREHOOK: query: EXPLAIN select count(*) from srcpart join srcpart_date on (day(srcpart.ds) = day(srcpart_date.ds)) where srcpart_date.`date` = '2008-04-08' +PREHOOK: type: QUERY +POSTHOOK: query: EXPLAIN select count(*) from srcpart join srcpart_date on (day(srcpart.ds) = day(srcpart_date.ds)) where srcpart_date.`date` = '2008-04-08' +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 + filterExpr: ((date = '2008-04-08') and ds is not null) (type: boolean) + Statistics: Num rows: 2 Data size: 42 Basic stats: COMPLETE Column stats: NONE + Filter Operator + predicate: ((date = '2008-04-08') and ds is not null) (type: boolean) + Statistics: Num rows: 1 Data size: 21 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: ds (type: string) + outputColumnNames: _col0 + Statistics: Num rows: 1 Data size: 21 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: day(_col0) (type: int) + outputColumnNames: _col0 + Statistics: Num rows: 1 Data size: 21 Basic stats: COMPLETE Column stats: NONE + Group By Operator + keys: _col0 (type: int) + mode: hash + outputColumnNames: _col0 + Statistics: Num rows: 1 Data size: 21 Basic stats: COMPLETE Column stats: NONE + Spark Partition Pruning Sink Operator + Target column: ds (string) + partition key expr: day(ds) + Statistics: Num rows: 1 Data size: 21 Basic stats: COMPLETE Column stats: NONE + 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 + filterExpr: ds is not null (type: boolean) + Statistics: Num rows: 2000 Data size: 21248 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: ds (type: string) + outputColumnNames: _col0 + Statistics: Num rows: 2000 Data size: 21248 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + key expressions: day(_col0) (type: int) + sort order: + + Map-reduce partition columns: day(_col0) (type: int) + Statistics: Num rows: 2000 Data size: 21248 Basic stats: COMPLETE Column stats: NONE + Map 4 + Map Operator Tree: + TableScan + alias: srcpart_date + filterExpr: ((date = '2008-04-08') and ds is not null) (type: boolean) + Statistics: Num rows: 2 Data size: 42 Basic stats: COMPLETE Column stats: NONE + Filter Operator + predicate: ((date = '2008-04-08') and ds is not null) (type: boolean) + Statistics: Num rows: 1 Data size: 21 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: ds (type: string) + outputColumnNames: _col0 + Statistics: Num rows: 1 Data size: 21 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + key expressions: day(_col0) (type: int) + sort order: + + Map-reduce partition columns: day(_col0) (type: int) + Statistics: Num rows: 1 Data size: 21 Basic stats: COMPLETE Column stats: NONE + Reducer 2 + Reduce Operator Tree: + Join Operator + condition map: + Inner Join 0 to 1 + keys: + 0 day(_col0) (type: int) + 1 day(_col0) (type: int) + 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 on (day(srcpart.ds) = day(srcpart_date.ds)) where srcpart_date.`date` = '2008-04-08' +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 +#### A masked pattern was here #### +POSTHOOK: query: select count(*) from srcpart join srcpart_date on (day(srcpart.ds) = day(srcpart_date.ds)) where srcpart_date.`date` = '2008-04-08' +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 +#### A masked pattern was here #### +1000 +PREHOOK: query: EXPLAIN select count(*) from srcpart join srcpart_date on (day(srcpart.ds) = day(srcpart_date.ds)) where srcpart_date.`date` = '2008-04-08' +PREHOOK: type: QUERY +POSTHOOK: query: EXPLAIN select count(*) from srcpart join srcpart_date on (day(srcpart.ds) = day(srcpart_date.ds)) where srcpart_date.`date` = '2008-04-08' +POSTHOOK: type: QUERY +STAGE DEPENDENCIES: + Stage-1 is a root stage + Stage-0 depends on stages: Stage-1 + +STAGE PLANS: + 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 + filterExpr: ds is not null (type: boolean) + Statistics: Num rows: 2000 Data size: 21248 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: ds (type: string) + outputColumnNames: _col0 + Statistics: Num rows: 2000 Data size: 21248 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + key expressions: day(_col0) (type: int) + sort order: + + Map-reduce partition columns: day(_col0) (type: int) + Statistics: Num rows: 2000 Data size: 21248 Basic stats: COMPLETE Column stats: NONE + Map 4 + Map Operator Tree: + TableScan + alias: srcpart_date + filterExpr: ((date = '2008-04-08') and ds is not null) (type: boolean) + Statistics: Num rows: 2 Data size: 42 Basic stats: COMPLETE Column stats: NONE + Filter Operator + predicate: ((date = '2008-04-08') and ds is not null) (type: boolean) + Statistics: Num rows: 1 Data size: 21 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: ds (type: string) + outputColumnNames: _col0 + Statistics: Num rows: 1 Data size: 21 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + key expressions: day(_col0) (type: int) + sort order: + + Map-reduce partition columns: day(_col0) (type: int) + Statistics: Num rows: 1 Data size: 21 Basic stats: COMPLETE Column stats: NONE + Reducer 2 + Reduce Operator Tree: + Join Operator + condition map: + Inner Join 0 to 1 + keys: + 0 day(_col0) (type: int) + 1 day(_col0) (type: int) + 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 on (day(srcpart.ds) = day(srcpart_date.ds)) where srcpart_date.`date` = '2008-04-08' +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 +#### A masked pattern was here #### +POSTHOOK: query: select count(*) from srcpart join srcpart_date on (day(srcpart.ds) = day(srcpart_date.ds)) where srcpart_date.`date` = '2008-04-08' +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 +#### A masked pattern was here #### +1000 +PREHOOK: query: EXPLAIN select count(*) from srcpart join srcpart_date on abs(negative(cast(concat(cast(day(srcpart.ds) as string), "0") as bigint)) + 10) = abs(negative(cast(concat(cast(day(srcpart_date.ds) as string), "0") as bigint)) + 10) where srcpart_date.`date` = '2008-04-08' +PREHOOK: type: QUERY +POSTHOOK: query: EXPLAIN select count(*) from srcpart join srcpart_date on abs(negative(cast(concat(cast(day(srcpart.ds) as string), "0") as bigint)) + 10) = abs(negative(cast(concat(cast(day(srcpart_date.ds) as string), "0") as bigint)) + 10) where srcpart_date.`date` = '2008-04-08' +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 + filterExpr: ((date = '2008-04-08') and ds is not null) (type: boolean) + Statistics: Num rows: 2 Data size: 42 Basic stats: COMPLETE Column stats: NONE + Filter Operator + predicate: ((date = '2008-04-08') and ds is not null) (type: boolean) + Statistics: Num rows: 1 Data size: 21 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: ds (type: string) + outputColumnNames: _col0 + Statistics: Num rows: 1 Data size: 21 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: abs(((- UDFToLong(concat(UDFToString(day(_col0)), '0'))) + 10)) (type: bigint) + outputColumnNames: _col0 + Statistics: Num rows: 1 Data size: 21 Basic stats: COMPLETE Column stats: NONE + Group By Operator + keys: _col0 (type: bigint) + mode: hash + outputColumnNames: _col0 + Statistics: Num rows: 1 Data size: 21 Basic stats: COMPLETE Column stats: NONE + Spark Partition Pruning Sink Operator + Target column: ds (string) + partition key expr: abs(((- UDFToLong(concat(UDFToString(day(ds)), '0'))) + 10)) + Statistics: Num rows: 1 Data size: 21 Basic stats: COMPLETE Column stats: NONE + 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 + filterExpr: ds is not null (type: boolean) + Statistics: Num rows: 2000 Data size: 21248 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: ds (type: string) + outputColumnNames: _col0 + Statistics: Num rows: 2000 Data size: 21248 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + key expressions: abs(((- UDFToLong(concat(UDFToString(day(_col0)), '0'))) + 10)) (type: bigint) + sort order: + + Map-reduce partition columns: abs(((- UDFToLong(concat(UDFToString(day(_col0)), '0'))) + 10)) (type: bigint) + Statistics: Num rows: 2000 Data size: 21248 Basic stats: COMPLETE Column stats: NONE + Map 4 + Map Operator Tree: + TableScan + alias: srcpart_date + filterExpr: ((date = '2008-04-08') and ds is not null) (type: boolean) + Statistics: Num rows: 2 Data size: 42 Basic stats: COMPLETE Column stats: NONE + Filter Operator + predicate: ((date = '2008-04-08') and ds is not null) (type: boolean) + Statistics: Num rows: 1 Data size: 21 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: ds (type: string) + outputColumnNames: _col0 + Statistics: Num rows: 1 Data size: 21 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + key expressions: abs(((- UDFToLong(concat(UDFToString(day(_col0)), '0'))) + 10)) (type: bigint) + sort order: + + Map-reduce partition columns: abs(((- UDFToLong(concat(UDFToString(day(_col0)), '0'))) + 10)) (type: bigint) + Statistics: Num rows: 1 Data size: 21 Basic stats: COMPLETE Column stats: NONE + Reducer 2 + Reduce Operator Tree: + Join Operator + condition map: + Inner Join 0 to 1 + keys: + 0 abs(((- UDFToLong(concat(UDFToString(day(_col0)), '0'))) + 10)) (type: bigint) + 1 abs(((- UDFToLong(concat(UDFToString(day(_col0)), '0'))) + 10)) (type: bigint) + 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 on abs(negative(cast(concat(cast(day(srcpart.ds) as string), "0") as bigint)) + 10) = abs(negative(cast(concat(cast(day(srcpart_date.ds) as string), "0") as bigint)) + 10) where srcpart_date.`date` = '2008-04-08' +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 +#### A masked pattern was here #### +POSTHOOK: query: select count(*) from srcpart join srcpart_date on abs(negative(cast(concat(cast(day(srcpart.ds) as string), "0") as bigint)) + 10) = abs(negative(cast(concat(cast(day(srcpart_date.ds) as string), "0") as bigint)) + 10) where srcpart_date.`date` = '2008-04-08' +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 +#### A masked pattern was here #### +1000 +PREHOOK: query: EXPLAIN select count(*) from srcpart join srcpart_date on cast(day(srcpart.ds) as smallint) = cast(day(srcpart_date.ds) as decimal) where srcpart_date.`date` = '2008-04-08' +PREHOOK: type: QUERY +POSTHOOK: query: EXPLAIN select count(*) from srcpart join srcpart_date on cast(day(srcpart.ds) as smallint) = cast(day(srcpart_date.ds) as decimal) where srcpart_date.`date` = '2008-04-08' +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 + filterExpr: ((date = '2008-04-08') and ds is not null) (type: boolean) + Statistics: Num rows: 2 Data size: 42 Basic stats: COMPLETE Column stats: NONE + Filter Operator + predicate: ((date = '2008-04-08') and ds is not null) (type: boolean) + Statistics: Num rows: 1 Data size: 21 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: ds (type: string) + outputColumnNames: _col0 + Statistics: Num rows: 1 Data size: 21 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: CAST( day(_col0) AS decimal(10,0)) (type: decimal(10,0)) + outputColumnNames: _col0 + Statistics: Num rows: 1 Data size: 21 Basic stats: COMPLETE Column stats: NONE + Group By Operator + keys: _col0 (type: decimal(10,0)) + mode: hash + outputColumnNames: _col0 + Statistics: Num rows: 1 Data size: 21 Basic stats: COMPLETE Column stats: NONE + Spark Partition Pruning Sink Operator + Target column: ds (string) + partition key expr: CAST( UDFToShort(day(ds)) AS decimal(10,0)) + Statistics: Num rows: 1 Data size: 21 Basic stats: COMPLETE Column stats: NONE + 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 + filterExpr: ds is not null (type: boolean) + Statistics: Num rows: 2000 Data size: 21248 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: ds (type: string) + outputColumnNames: _col0 + Statistics: Num rows: 2000 Data size: 21248 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + key expressions: CAST( UDFToShort(day(_col0)) AS decimal(10,0)) (type: decimal(10,0)) + sort order: + + Map-reduce partition columns: CAST( UDFToShort(day(_col0)) AS decimal(10,0)) (type: decimal(10,0)) + Statistics: Num rows: 2000 Data size: 21248 Basic stats: COMPLETE Column stats: NONE + Map 4 + Map Operator Tree: + TableScan + alias: srcpart_date + filterExpr: ((date = '2008-04-08') and ds is not null) (type: boolean) + Statistics: Num rows: 2 Data size: 42 Basic stats: COMPLETE Column stats: NONE + Filter Operator + predicate: ((date = '2008-04-08') and ds is not null) (type: boolean) + Statistics: Num rows: 1 Data size: 21 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: ds (type: string) + outputColumnNames: _col0 + Statistics: Num rows: 1 Data size: 21 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + key expressions: CAST( day(_col0) AS decimal(10,0)) (type: decimal(10,0)) + sort order: + + Map-reduce partition columns: CAST( day(_col0) AS decimal(10,0)) (type: decimal(10,0)) + Statistics: Num rows: 1 Data size: 21 Basic stats: COMPLETE Column stats: NONE + Reducer 2 + Reduce Operator Tree: + Join Operator + condition map: + Inner Join 0 to 1 + keys: + 0 CAST( UDFToShort(day(_col0)) AS decimal(10,0)) (type: decimal(10,0)) + 1 CAST( day(_col0) AS decimal(10,0)) (type: decimal(10,0)) + 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 on cast(day(srcpart.ds) as smallint) = cast(day(srcpart_date.ds) as decimal) where srcpart_date.`date` = '2008-04-08' +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 +#### A masked pattern was here #### +POSTHOOK: query: select count(*) from srcpart join srcpart_date on cast(day(srcpart.ds) as smallint) = cast(day(srcpart_date.ds) as decimal) where srcpart_date.`date` = '2008-04-08' +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 +#### A masked pattern was here #### +1000 PREHOOK: query: EXPLAIN select count(*) from srcpart join srcpart_date on (srcpart.ds = srcpart_date.ds) join srcpart_hour on (srcpart.hr = srcpart_hour.hr) where srcpart_date.`date` = '2008-04-08' and srcpart_hour.hour = 11 PREHOOK: type: QUERY @@ -485,9 +1012,9 @@ STAGE PLANS: outputColumnNames: _col0 Statistics: Num rows: 1 Data size: 21 Basic stats: COMPLETE Column stats: NONE Spark Partition Pruning Sink Operator + Target column: ds (string) partition key expr: ds Statistics: Num rows: 1 Data size: 21 Basic stats: COMPLETE Column stats: NONE - target column name: ds target work: Map 1 Map 8 Map Operator Tree: @@ -512,9 +1039,9 @@ STAGE PLANS: outputColumnNames: _col0 Statistics: Num rows: 1 Data size: 5 Basic stats: COMPLETE Column stats: NONE Spark Partition Pruning Sink Operator + Target column: hr (string) partition key expr: hr Statistics: Num rows: 1 Data size: 5 Basic stats: COMPLETE Column stats: NONE - target column name: hr target work: Map 1 Stage: Stage-1 @@ -847,9 +1374,9 @@ STAGE PLANS: outputColumnNames: _col0 Statistics: Num rows: 1 Data size: 27 Basic stats: COMPLETE Column stats: NONE Spark Partition Pruning Sink Operator + Target column: ds (string) 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) @@ -861,9 +1388,9 @@ STAGE PLANS: outputColumnNames: _col0 Statistics: Num rows: 1 Data size: 27 Basic stats: COMPLETE Column stats: NONE Spark Partition Pruning Sink Operator + Target column: hr (string) 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 @@ -1116,9 +1643,9 @@ STAGE PLANS: outputColumnNames: _col0 Statistics: Num rows: 1 Data size: 21 Basic stats: COMPLETE Column stats: NONE Spark Partition Pruning Sink Operator + Target column: ds (string) partition key expr: ds Statistics: Num rows: 1 Data size: 21 Basic stats: COMPLETE Column stats: NONE - target column name: ds target work: Map 1 Stage: Stage-1 @@ -1372,9 +1899,9 @@ STAGE PLANS: outputColumnNames: _col0 Statistics: Num rows: 1 Data size: 7 Basic stats: COMPLETE Column stats: NONE Spark Partition Pruning Sink Operator + Target column: hr (string) partition key expr: UDFToDouble(hr) Statistics: Num rows: 1 Data size: 7 Basic stats: COMPLETE Column stats: NONE - target column name: hr target work: Map 1 Stage: Stage-1 @@ -1512,9 +2039,9 @@ STAGE PLANS: outputColumnNames: _col0 Statistics: Num rows: 1 Data size: 7 Basic stats: COMPLETE Column stats: NONE Spark Partition Pruning Sink Operator + Target column: hr (string) partition key expr: (UDFToDouble(hr) * 2.0) Statistics: Num rows: 1 Data size: 7 Basic stats: COMPLETE Column stats: NONE - target column name: hr target work: Map 1 Stage: Stage-1 @@ -1875,9 +2402,9 @@ STAGE PLANS: outputColumnNames: _col0 Statistics: Num rows: 1 Data size: 7 Basic stats: COMPLETE Column stats: NONE Spark Partition Pruning Sink Operator + Target column: hr (string) partition key expr: UDFToString((UDFToDouble(hr) * 2.0)) Statistics: Num rows: 1 Data size: 7 Basic stats: COMPLETE Column stats: NONE - target column name: hr target work: Map 1 Stage: Stage-1 @@ -2257,9 +2784,9 @@ STAGE PLANS: outputColumnNames: _col0 Statistics: Num rows: 1 Data size: 27 Basic stats: COMPLETE Column stats: NONE Spark Partition Pruning Sink Operator + Target column: ds (string) 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) @@ -2271,9 +2798,9 @@ STAGE PLANS: outputColumnNames: _col0 Statistics: Num rows: 1 Data size: 27 Basic stats: COMPLETE Column stats: NONE Spark Partition Pruning Sink Operator + Target column: hr (string) 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 @@ -2410,9 +2937,9 @@ STAGE PLANS: outputColumnNames: _col0 Statistics: Num rows: 1 Data size: 21 Basic stats: COMPLETE Column stats: NONE Spark Partition Pruning Sink Operator + Target column: ds (string) partition key expr: ds Statistics: Num rows: 1 Data size: 21 Basic stats: COMPLETE Column stats: NONE - target column name: ds target work: Map 1 Stage: Stage-1 @@ -2531,9 +3058,9 @@ STAGE PLANS: outputColumnNames: _col0 Statistics: Num rows: 1 Data size: 21 Basic stats: COMPLETE Column stats: NONE Spark Partition Pruning Sink Operator + Target column: ds (string) partition key expr: ds Statistics: Num rows: 1 Data size: 21 Basic stats: COMPLETE Column stats: NONE - target column name: ds target work: Map 4 Stage: Stage-1 @@ -2651,9 +3178,9 @@ STAGE PLANS: outputColumnNames: _col0 Statistics: Num rows: 1 Data size: 21 Basic stats: COMPLETE Column stats: NONE Spark Partition Pruning Sink Operator + Target column: ds (string) partition key expr: ds Statistics: Num rows: 1 Data size: 21 Basic stats: COMPLETE Column stats: NONE - target column name: ds target work: Map 1 Stage: Stage-1 @@ -2773,9 +3300,9 @@ STAGE PLANS: outputColumnNames: _col0 Statistics: Num rows: 1 Data size: 21 Basic stats: COMPLETE Column stats: NONE Spark Partition Pruning Sink Operator + Target column: ds (string) partition key expr: ds Statistics: Num rows: 1 Data size: 21 Basic stats: COMPLETE Column stats: NONE - target column name: ds target work: Map 1 Map 8 Map Operator Tree: @@ -2800,9 +3327,9 @@ STAGE PLANS: outputColumnNames: _col0 Statistics: Num rows: 1 Data size: 5 Basic stats: COMPLETE Column stats: NONE Spark Partition Pruning Sink Operator + Target column: hr (string) partition key expr: hr Statistics: Num rows: 1 Data size: 5 Basic stats: COMPLETE Column stats: NONE - target column name: hr target work: Map 1 Stage: Stage-1 @@ -3158,9 +3685,9 @@ STAGE PLANS: outputColumnNames: _col0 Statistics: Num rows: 2 Data size: 368 Basic stats: COMPLETE Column stats: NONE Spark Partition Pruning Sink Operator + Target column: ds (string) partition key expr: ds Statistics: Num rows: 2 Data size: 368 Basic stats: COMPLETE Column stats: NONE - target column name: ds target work: Map 1 Reducer 9 Reduce Operator Tree: @@ -3187,9 +3714,9 @@ STAGE PLANS: outputColumnNames: _col0 Statistics: Num rows: 2 Data size: 368 Basic stats: COMPLETE Column stats: NONE Spark Partition Pruning Sink Operator + Target column: ds (string) partition key expr: ds Statistics: Num rows: 2 Data size: 368 Basic stats: COMPLETE Column stats: NONE - target column name: ds target work: Map 1 Stage: Stage-1 @@ -3426,9 +3953,9 @@ STAGE PLANS: outputColumnNames: _col0 Statistics: Num rows: 2 Data size: 368 Basic stats: COMPLETE Column stats: NONE Spark Partition Pruning Sink Operator + Target column: ds (string) partition key expr: ds Statistics: Num rows: 2 Data size: 368 Basic stats: COMPLETE Column stats: NONE - target column name: ds target work: Map 1 Reducer 9 Reduce Operator Tree: @@ -3455,9 +3982,9 @@ STAGE PLANS: outputColumnNames: _col0 Statistics: Num rows: 2 Data size: 368 Basic stats: COMPLETE Column stats: NONE Spark Partition Pruning Sink Operator + Target column: ds (string) partition key expr: ds Statistics: Num rows: 2 Data size: 368 Basic stats: COMPLETE Column stats: NONE - target column name: ds target work: Map 1 Stage: Stage-1 @@ -3697,9 +4224,9 @@ STAGE PLANS: outputColumnNames: _col0 Statistics: Num rows: 2 Data size: 368 Basic stats: COMPLETE Column stats: NONE Spark Partition Pruning Sink Operator + Target column: ds (string) partition key expr: ds Statistics: Num rows: 2 Data size: 368 Basic stats: COMPLETE Column stats: NONE - target column name: ds target work: Map 1 Select Operator expressions: _col0 (type: string) @@ -3711,9 +4238,9 @@ STAGE PLANS: outputColumnNames: _col0 Statistics: Num rows: 2 Data size: 368 Basic stats: COMPLETE Column stats: NONE Spark Partition Pruning Sink Operator + Target column: ds (string) partition key expr: ds Statistics: Num rows: 2 Data size: 368 Basic stats: COMPLETE Column stats: NONE - target column name: ds target work: Map 4 Reducer 13 Reduce Operator Tree: @@ -3740,9 +4267,9 @@ STAGE PLANS: outputColumnNames: _col0 Statistics: Num rows: 2 Data size: 368 Basic stats: COMPLETE Column stats: NONE Spark Partition Pruning Sink Operator + Target column: ds (string) partition key expr: ds Statistics: Num rows: 2 Data size: 368 Basic stats: COMPLETE Column stats: NONE - target column name: ds target work: Map 1 Select Operator expressions: _col0 (type: string) @@ -3754,9 +4281,9 @@ STAGE PLANS: outputColumnNames: _col0 Statistics: Num rows: 2 Data size: 368 Basic stats: COMPLETE Column stats: NONE Spark Partition Pruning Sink Operator + Target column: ds (string) partition key expr: ds Statistics: Num rows: 2 Data size: 368 Basic stats: COMPLETE Column stats: NONE - target column name: ds target work: Map 4 Stage: Stage-1 @@ -3957,9 +4484,9 @@ STAGE PLANS: outputColumnNames: _col0 Statistics: Num rows: 1 Data size: 21 Basic stats: COMPLETE Column stats: NONE Spark Partition Pruning Sink Operator + Target column: ds (string) partition key expr: ds Statistics: Num rows: 1 Data size: 21 Basic stats: COMPLETE Column stats: NONE - target column name: ds target work: Map 1 Local Work: Map Reduce Local Work @@ -4049,6 +4576,130 @@ POSTHOOK: type: QUERY POSTHOOK: Input: default@srcpart #### A masked pattern was here #### 1000 +PREHOOK: query: EXPLAIN select count(*) from srcpart join srcpart_date on (day(srcpart.ds) = day(srcpart_date.ds)) where srcpart_date.`date` = '2008-04-08' +PREHOOK: type: QUERY +POSTHOOK: query: EXPLAIN select count(*) from srcpart join srcpart_date on (day(srcpart.ds) = day(srcpart_date.ds)) where srcpart_date.`date` = '2008-04-08' +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 3 + Map Operator Tree: + TableScan + alias: srcpart_date + filterExpr: ((date = '2008-04-08') and ds is not null) (type: boolean) + Statistics: Num rows: 2 Data size: 42 Basic stats: COMPLETE Column stats: NONE + Filter Operator + predicate: ((date = '2008-04-08') and ds is not null) (type: boolean) + Statistics: Num rows: 1 Data size: 21 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: ds (type: string) + outputColumnNames: _col0 + Statistics: Num rows: 1 Data size: 21 Basic stats: COMPLETE Column stats: NONE + Spark HashTable Sink Operator + keys: + 0 day(_col0) (type: int) + 1 day(_col0) (type: int) + Select Operator + expressions: day(_col0) (type: int) + outputColumnNames: _col0 + Statistics: Num rows: 1 Data size: 21 Basic stats: COMPLETE Column stats: NONE + Group By Operator + keys: _col0 (type: int) + mode: hash + outputColumnNames: _col0 + Statistics: Num rows: 1 Data size: 21 Basic stats: COMPLETE Column stats: NONE + Spark Partition Pruning Sink Operator + Target column: ds (string) + partition key expr: day(ds) + Statistics: Num rows: 1 Data size: 21 Basic stats: COMPLETE Column stats: NONE + target work: Map 1 + Local Work: + Map Reduce Local Work + + Stage: Stage-1 + Spark + Edges: + Reducer 2 <- Map 1 (GROUP, 1) +#### A masked pattern was here #### + Vertices: + Map 1 + Map Operator Tree: + TableScan + alias: srcpart + filterExpr: ds is not null (type: boolean) + Statistics: Num rows: 2000 Data size: 21248 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: ds (type: string) + outputColumnNames: _col0 + Statistics: Num rows: 2000 Data size: 21248 Basic stats: COMPLETE Column stats: NONE + Map Join Operator + condition map: + Inner Join 0 to 1 + keys: + 0 day(_col0) (type: int) + 1 day(_col0) (type: int) + input vertices: + 1 Map 3 + 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) + Local Work: + Map Reduce Local Work + Reducer 2 + 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 on (day(srcpart.ds) = day(srcpart_date.ds)) where srcpart_date.`date` = '2008-04-08' +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 +#### A masked pattern was here #### +POSTHOOK: query: select count(*) from srcpart join srcpart_date on (day(srcpart.ds) = day(srcpart_date.ds)) where srcpart_date.`date` = '2008-04-08' +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 +#### A masked pattern was here #### +1000 PREHOOK: query: EXPLAIN select count(*) from srcpart join srcpart_date on (srcpart.ds = srcpart_date.ds) join srcpart_hour on (srcpart.hr = srcpart_hour.hr) where srcpart_date.`date` = '2008-04-08' and srcpart_hour.hour = 11 PREHOOK: type: QUERY @@ -4092,9 +4743,9 @@ STAGE PLANS: outputColumnNames: _col0 Statistics: Num rows: 1 Data size: 21 Basic stats: COMPLETE Column stats: NONE Spark Partition Pruning Sink Operator + Target column: ds (string) partition key expr: ds Statistics: Num rows: 1 Data size: 21 Basic stats: COMPLETE Column stats: NONE - target column name: ds target work: Map 1 Local Work: Map Reduce Local Work @@ -4125,9 +4776,9 @@ STAGE PLANS: outputColumnNames: _col0 Statistics: Num rows: 1 Data size: 5 Basic stats: COMPLETE Column stats: NONE Spark Partition Pruning Sink Operator + Target column: hr (string) partition key expr: hr Statistics: Num rows: 1 Data size: 5 Basic stats: COMPLETE Column stats: NONE - target column name: hr target work: Map 1 Local Work: Map Reduce Local Work @@ -4271,9 +4922,9 @@ STAGE PLANS: outputColumnNames: _col0 Statistics: Num rows: 1 Data size: 27 Basic stats: COMPLETE Column stats: NONE Spark Partition Pruning Sink Operator + Target column: ds (string) 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) @@ -4285,9 +4936,9 @@ STAGE PLANS: outputColumnNames: _col0 Statistics: Num rows: 1 Data size: 27 Basic stats: COMPLETE Column stats: NONE Spark Partition Pruning Sink Operator + Target column: hr (string) 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 Local Work: Map Reduce Local Work @@ -4417,9 +5068,9 @@ STAGE PLANS: outputColumnNames: _col0 Statistics: Num rows: 1 Data size: 21 Basic stats: COMPLETE Column stats: NONE Spark Partition Pruning Sink Operator + Target column: ds (string) partition key expr: ds Statistics: Num rows: 1 Data size: 21 Basic stats: COMPLETE Column stats: NONE - target column name: ds target work: Map 1 Local Work: Map Reduce Local Work @@ -4522,9 +5173,9 @@ STAGE PLANS: outputColumnNames: _col0 Statistics: Num rows: 1 Data size: 7 Basic stats: COMPLETE Column stats: NONE Spark Partition Pruning Sink Operator + Target column: hr (string) partition key expr: UDFToDouble(hr) Statistics: Num rows: 1 Data size: 7 Basic stats: COMPLETE Column stats: NONE - target column name: hr target work: Map 1 Local Work: Map Reduce Local Work @@ -4646,9 +5297,9 @@ STAGE PLANS: outputColumnNames: _col0 Statistics: Num rows: 1 Data size: 7 Basic stats: COMPLETE Column stats: NONE Spark Partition Pruning Sink Operator + Target column: hr (string) partition key expr: (UDFToDouble(hr) * 2.0) Statistics: Num rows: 1 Data size: 7 Basic stats: COMPLETE Column stats: NONE - target column name: hr target work: Map 1 Local Work: Map Reduce Local Work @@ -4867,6 +5518,130 @@ POSTHOOK: type: QUERY POSTHOOK: Input: default@srcpart #### A masked pattern was here #### 1000 +PREHOOK: query: EXPLAIN select count(*) from srcpart join srcpart_date on (day(srcpart.ds) = day(srcpart_date.ds)) where srcpart_date.`date` = '2008-04-08' +PREHOOK: type: QUERY +POSTHOOK: query: EXPLAIN select count(*) from srcpart join srcpart_date on (day(srcpart.ds) = day(srcpart_date.ds)) where srcpart_date.`date` = '2008-04-08' +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 3 + Map Operator Tree: + TableScan + alias: srcpart_date + filterExpr: ((date = '2008-04-08') and ds is not null) (type: boolean) + Statistics: Num rows: 2 Data size: 42 Basic stats: COMPLETE Column stats: NONE + Filter Operator + predicate: ((date = '2008-04-08') and ds is not null) (type: boolean) + Statistics: Num rows: 1 Data size: 21 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: ds (type: string) + outputColumnNames: _col0 + Statistics: Num rows: 1 Data size: 21 Basic stats: COMPLETE Column stats: NONE + Spark HashTable Sink Operator + keys: + 0 day(_col0) (type: int) + 1 day(_col0) (type: int) + Select Operator + expressions: day(_col0) (type: int) + outputColumnNames: _col0 + Statistics: Num rows: 1 Data size: 21 Basic stats: COMPLETE Column stats: NONE + Group By Operator + keys: _col0 (type: int) + mode: hash + outputColumnNames: _col0 + Statistics: Num rows: 1 Data size: 21 Basic stats: COMPLETE Column stats: NONE + Spark Partition Pruning Sink Operator + Target column: ds (string) + partition key expr: day(ds) + Statistics: Num rows: 1 Data size: 21 Basic stats: COMPLETE Column stats: NONE + target work: Map 1 + Local Work: + Map Reduce Local Work + + Stage: Stage-1 + Spark + Edges: + Reducer 2 <- Map 1 (GROUP, 1) +#### A masked pattern was here #### + Vertices: + Map 1 + Map Operator Tree: + TableScan + alias: srcpart + filterExpr: ds is not null (type: boolean) + Statistics: Num rows: 2000 Data size: 21248 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: ds (type: string) + outputColumnNames: _col0 + Statistics: Num rows: 2000 Data size: 21248 Basic stats: COMPLETE Column stats: NONE + Map Join Operator + condition map: + Inner Join 0 to 1 + keys: + 0 day(_col0) (type: int) + 1 day(_col0) (type: int) + input vertices: + 1 Map 3 + 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) + Local Work: + Map Reduce Local Work + Reducer 2 + 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 on (day(srcpart.ds) = day(srcpart_date.ds)) where srcpart_date.`date` = '2008-04-08' +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 +#### A masked pattern was here #### +POSTHOOK: query: select count(*) from srcpart join srcpart_date on (day(srcpart.ds) = day(srcpart_date.ds)) where srcpart_date.`date` = '2008-04-08' +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 +#### A masked pattern was here #### +1000 PREHOOK: query: EXPLAIN select count(*) from srcpart left join srcpart_date on (srcpart.ds = srcpart_date.ds) where srcpart_date.`date` = '2008-04-08' PREHOOK: type: QUERY POSTHOOK: query: EXPLAIN select count(*) from srcpart left join srcpart_date on (srcpart.ds = srcpart_date.ds) where srcpart_date.`date` = '2008-04-08' @@ -4908,9 +5683,9 @@ STAGE PLANS: outputColumnNames: _col0 Statistics: Num rows: 1 Data size: 21 Basic stats: COMPLETE Column stats: NONE Spark Partition Pruning Sink Operator + Target column: ds (string) partition key expr: ds Statistics: Num rows: 1 Data size: 21 Basic stats: COMPLETE Column stats: NONE - target column name: ds target work: Map 1 Local Work: Map Reduce Local Work @@ -5195,9 +5970,9 @@ STAGE PLANS: outputColumnNames: _col0 Statistics: Num rows: 1 Data size: 21 Basic stats: COMPLETE Column stats: NONE Spark Partition Pruning Sink Operator + Target column: ds (string) partition key expr: ds Statistics: Num rows: 1 Data size: 21 Basic stats: COMPLETE Column stats: NONE - target column name: ds target work: Map 1 Local Work: Map Reduce Local Work @@ -5228,9 +6003,9 @@ STAGE PLANS: outputColumnNames: _col0 Statistics: Num rows: 1 Data size: 5 Basic stats: COMPLETE Column stats: NONE Spark Partition Pruning Sink Operator + Target column: hr (string) partition key expr: hr Statistics: Num rows: 1 Data size: 5 Basic stats: COMPLETE Column stats: NONE - target column name: hr target work: Map 1 Local Work: Map Reduce Local Work @@ -5529,9 +6304,9 @@ STAGE PLANS: outputColumnNames: _col0 Statistics: Num rows: 2 Data size: 368 Basic stats: COMPLETE Column stats: NONE Spark Partition Pruning Sink Operator + Target column: ds (string) partition key expr: ds Statistics: Num rows: 2 Data size: 368 Basic stats: COMPLETE Column stats: NONE - target column name: ds target work: Map 1 Reducer 9 Reduce Operator Tree: @@ -5558,9 +6333,9 @@ STAGE PLANS: outputColumnNames: _col0 Statistics: Num rows: 2 Data size: 368 Basic stats: COMPLETE Column stats: NONE Spark Partition Pruning Sink Operator + Target column: ds (string) partition key expr: ds Statistics: Num rows: 2 Data size: 368 Basic stats: COMPLETE Column stats: NONE - target column name: ds target work: Map 1 Stage: Stage-1 diff --git a/ql/src/test/results/clientpositive/spark/spark_dynamic_partition_pruning_2.q.out b/ql/src/test/results/clientpositive/spark/spark_dynamic_partition_pruning_2.q.out index d902694a19..8a794e749b 100644 --- a/ql/src/test/results/clientpositive/spark/spark_dynamic_partition_pruning_2.q.out +++ b/ql/src/test/results/clientpositive/spark/spark_dynamic_partition_pruning_2.q.out @@ -184,9 +184,9 @@ STAGE PLANS: outputColumnNames: _col0 Statistics: Num rows: 2 Data size: 10 Basic stats: COMPLETE Column stats: NONE Spark Partition Pruning Sink Operator + Target column: dim_shops_id (int) partition key expr: dim_shops_id Statistics: Num rows: 2 Data size: 10 Basic stats: COMPLETE Column stats: NONE - target column name: dim_shops_id target work: Map 1 Local Work: Map Reduce Local Work @@ -726,9 +726,9 @@ STAGE PLANS: outputColumnNames: _col0 Statistics: Num rows: 2 Data size: 10 Basic stats: COMPLETE Column stats: NONE Spark Partition Pruning Sink Operator + Target column: dim_shops_id (int) partition key expr: dim_shops_id Statistics: Num rows: 2 Data size: 10 Basic stats: COMPLETE Column stats: NONE - target column name: dim_shops_id target work: Map 1 Local Work: Map Reduce Local Work @@ -887,9 +887,9 @@ STAGE PLANS: outputColumnNames: _col0 Statistics: Num rows: 1 Data size: 5 Basic stats: COMPLETE Column stats: NONE Spark Partition Pruning Sink Operator + Target column: dim_shops_id (int) partition key expr: dim_shops_id Statistics: Num rows: 1 Data size: 5 Basic stats: COMPLETE Column stats: NONE - target column name: dim_shops_id target work: Map 1 Local Work: Map Reduce Local Work @@ -925,9 +925,9 @@ STAGE PLANS: outputColumnNames: _col0 Statistics: Num rows: 1 Data size: 5 Basic stats: COMPLETE Column stats: NONE Spark Partition Pruning Sink Operator + Target column: dim_shops_id (int) partition key expr: dim_shops_id Statistics: Num rows: 1 Data size: 5 Basic stats: COMPLETE Column stats: NONE - target column name: dim_shops_id target work: Map 3 Local Work: Map Reduce Local Work diff --git a/ql/src/test/results/clientpositive/spark/spark_dynamic_partition_pruning_3.q.out b/ql/src/test/results/clientpositive/spark/spark_dynamic_partition_pruning_3.q.out index 9e583e9686..82e746a862 100644 --- a/ql/src/test/results/clientpositive/spark/spark_dynamic_partition_pruning_3.q.out +++ b/ql/src/test/results/clientpositive/spark/spark_dynamic_partition_pruning_3.q.out @@ -94,9 +94,9 @@ STAGE PLANS: outputColumnNames: _col0 Statistics: Num rows: 10 Data size: 11 Basic stats: COMPLETE Column stats: NONE Spark Partition Pruning Sink Operator + Target column: part_col (int) partition key expr: part_col Statistics: Num rows: 10 Data size: 11 Basic stats: COMPLETE Column stats: NONE - target column name: part_col target work: Map 1 Local Work: Map Reduce Local Work @@ -181,9 +181,9 @@ STAGE PLANS: outputColumnNames: _col0 Statistics: Num rows: 1 Data size: 1 Basic stats: COMPLETE Column stats: NONE Spark Partition Pruning Sink Operator + Target column: part_col (int) partition key expr: part_col Statistics: Num rows: 1 Data size: 1 Basic stats: COMPLETE Column stats: NONE - target column name: part_col target work: Map 3 Local Work: Map Reduce Local Work @@ -230,9 +230,9 @@ STAGE PLANS: outputColumnNames: _col0 Statistics: Num rows: 11 Data size: 12 Basic stats: COMPLETE Column stats: NONE Spark Partition Pruning Sink Operator + Target column: part_col (int) partition key expr: part_col Statistics: Num rows: 11 Data size: 12 Basic stats: COMPLETE Column stats: NONE - target column name: part_col target work: Map 1 Local Work: Map Reduce Local Work