diff --git itests/src/test/resources/testconfiguration.properties itests/src/test/resources/testconfiguration.properties index 536e418..b84d4b4 100644 --- itests/src/test/resources/testconfiguration.properties +++ itests/src/test/resources/testconfiguration.properties @@ -186,7 +186,8 @@ minitez.query.files=bucket_map_join_tez1.q,\ tez_joins_explain.q,\ tez_schema_evolution.q,\ tez_union.q,\ - tez_union_decimal.q + tez_union_decimal.q,\ + vectorized_dynamic_partition_pruning.q beeline.positive.exclude=add_part_exist.q,\ alter1.q,\ diff --git ql/src/java/org/apache/hadoop/hive/ql/exec/AppMasterEventOperator.java ql/src/java/org/apache/hadoop/hive/ql/exec/AppMasterEventOperator.java index 7315be5..b562392 100644 --- ql/src/java/org/apache/hadoop/hive/ql/exec/AppMasterEventOperator.java +++ ql/src/java/org/apache/hadoop/hive/ql/exec/AppMasterEventOperator.java @@ -44,10 +44,10 @@ @SuppressWarnings({ "deprecation", "serial" }) public class AppMasterEventOperator extends Operator { - private transient Serializer serializer; - private transient DataOutputBuffer buffer; - private transient boolean hasReachedMaxSize = false; - private transient long MAX_SIZE; + protected transient Serializer serializer; + protected transient DataOutputBuffer buffer; + protected transient boolean hasReachedMaxSize = false; + protected transient long MAX_SIZE; @Override public void initializeOp(Configuration hconf) throws HiveException { @@ -57,7 +57,7 @@ public void initializeOp(Configuration hconf) throws HiveException { initDataBuffer(false); } - private void initDataBuffer(boolean skipPruning) throws HiveException { + protected void initDataBuffer(boolean skipPruning) throws HiveException { buffer = new DataOutputBuffer(); try { // where does this go to? diff --git ql/src/java/org/apache/hadoop/hive/ql/exec/OperatorFactory.java ql/src/java/org/apache/hadoop/hive/ql/exec/OperatorFactory.java index ed8692d..eb29458 100644 --- ql/src/java/org/apache/hadoop/hive/ql/exec/OperatorFactory.java +++ ql/src/java/org/apache/hadoop/hive/ql/exec/OperatorFactory.java @@ -18,6 +18,7 @@ package org.apache.hadoop.hive.ql.exec; +import org.apache.hadoop.hive.ql.exec.vector.VectorAppMasterEventOperator; import org.apache.hadoop.hive.ql.exec.vector.VectorExtractOperator; import org.apache.hadoop.hive.ql.exec.vector.VectorFileSinkOperator; import org.apache.hadoop.hive.ql.exec.vector.VectorFilterOperator; @@ -118,6 +119,10 @@ static { vectorOpvec = new ArrayList(); + vectorOpvec.add(new OpTuple(AppMasterEventDesc.class, + VectorAppMasterEventOperator.class)); + vectorOpvec.add(new OpTuple(DynamicPruningEventDesc.class, + VectorAppMasterEventOperator.class)); vectorOpvec.add(new OpTuple(SelectDesc.class, VectorSelectOperator.class)); vectorOpvec.add(new OpTuple(GroupByDesc.class, VectorGroupByOperator.class)); vectorOpvec.add(new OpTuple(MapJoinDesc.class, VectorMapJoinOperator.class)); diff --git ql/src/java/org/apache/hadoop/hive/ql/exec/vector/VectorAppMasterEventOperator.java ql/src/java/org/apache/hadoop/hive/ql/exec/vector/VectorAppMasterEventOperator.java new file mode 100644 index 0000000..d05cc23 --- /dev/null +++ ql/src/java/org/apache/hadoop/hive/ql/exec/vector/VectorAppMasterEventOperator.java @@ -0,0 +1,128 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hadoop.hive.ql.exec.vector; + +import java.io.IOException; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.hive.ql.exec.AppMasterEventOperator; +import org.apache.hadoop.hive.ql.exec.vector.expressions.VectorExpressionWriter; +import org.apache.hadoop.hive.ql.exec.vector.expressions.VectorExpressionWriterFactory; +import org.apache.hadoop.hive.ql.metadata.HiveException; +import org.apache.hadoop.hive.ql.plan.AppMasterEventDesc; +import org.apache.hadoop.hive.ql.plan.OperatorDesc; +import org.apache.hadoop.hive.common.StatsSetupConst; +import org.apache.hadoop.hive.serde2.SerDeException; +import org.apache.hadoop.hive.serde2.SerDeStats; +import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspector; +import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspectorUtils; +import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspectorUtils.ObjectInspectorCopyOption; +import org.apache.hadoop.hive.serde2.objectinspector.StructObjectInspector; +import org.apache.hadoop.io.ObjectWritable; +import org.apache.hadoop.io.Text; +import org.apache.hadoop.io.Writable; + +/** + * App Master Event operator implementation. + **/ +public class VectorAppMasterEventOperator extends AppMasterEventOperator { + + private static final long serialVersionUID = 1L; + + protected transient Object[] singleRow; + + protected transient VectorExpressionWriter[] valueWriters; + + public VectorAppMasterEventOperator(VectorizationContext context, + OperatorDesc conf) { + super(); + this.conf = (AppMasterEventDesc) conf; + } + + public VectorAppMasterEventOperator() { + } + + @Override + public void initializeOp(Configuration hconf) throws HiveException { + super.initializeOp(hconf); + valueWriters = VectorExpressionWriterFactory.getExpressionWriters( + (StructObjectInspector) inputObjInspectors[0]); + singleRow = new Object[valueWriters.length]; + } + + @Override + public void processOp(Object data, int tag) throws HiveException { + + VectorizedRowBatch vrg = (VectorizedRowBatch) data; + + Writable [] records = null; + Writable recordValue = null; + boolean vectorizedSerde = false; + + try { + if (serializer instanceof VectorizedSerde) { + recordValue = ((VectorizedSerde) serializer).serializeVector(vrg, + inputObjInspectors[0]); + records = (Writable[]) ((ObjectWritable) recordValue).get(); + vectorizedSerde = true; + } + } catch (SerDeException e1) { + throw new HiveException(e1); + } + + for (int i = 0; i < vrg.size; i++) { + Writable row = null; + if (vectorizedSerde) { + row = records[i]; + } else { + if (vrg.valueWriters == null) { + vrg.setValueWriters(this.valueWriters); + } + try { + row = serializer.serialize(getRowObject(vrg, i), inputObjInspectors[0]); + } catch (SerDeException ex) { + throw new HiveException(ex); + } + } + try { + row.write(buffer); + if (buffer.getLength() > MAX_SIZE) { + LOG.info("Disabling AM events. Buffer size too large: " + buffer.getLength()); + hasReachedMaxSize = true; + buffer = null; + } + } catch (Exception e) { + throw new HiveException(e); + } + } + } + + private Object[] getRowObject(VectorizedRowBatch vrg, int rowIndex) + throws HiveException { + int batchIndex = rowIndex; + if (vrg.selectedInUse) { + batchIndex = vrg.selected[rowIndex]; + } + for (int i = 0; i < vrg.projectionSize; i++) { + ColumnVector vectorColumn = vrg.cols[vrg.projectedColumns[i]]; + singleRow[i] = vrg.valueWriters[i].writeValue(vectorColumn, batchIndex); + } + return singleRow; + } +} diff --git ql/src/java/org/apache/hadoop/hive/ql/optimizer/physical/Vectorizer.java ql/src/java/org/apache/hadoop/hive/ql/optimizer/physical/Vectorizer.java index e682dba..51baed2 100644 --- ql/src/java/org/apache/hadoop/hive/ql/optimizer/physical/Vectorizer.java +++ ql/src/java/org/apache/hadoop/hive/ql/optimizer/physical/Vectorizer.java @@ -816,6 +816,7 @@ boolean validateMapWorkOperator(Operator op, boolean isT break; case FILESINK: case LIMIT: + case EVENT: ret = true; break; default: @@ -855,6 +856,7 @@ boolean validateReduceWorkOperator(Operator op) { ret = validateFileSinkOperator((FileSinkOperator) op); break; case LIMIT: + case EVENT: ret = true; break; default: @@ -1196,6 +1198,7 @@ private void fixupParentChildOperators(Operator op, Oper case REDUCESINK: case LIMIT: case EXTRACT: + case EVENT: vectorOp = OperatorFactory.getVectorOperator(op.getConf(), vContext); break; default: diff --git ql/src/test/queries/clientpositive/vectorized_dynamic_partition_pruning.q ql/src/test/queries/clientpositive/vectorized_dynamic_partition_pruning.q new file mode 100644 index 0000000..1197f7d --- /dev/null +++ ql/src/test/queries/clientpositive/vectorized_dynamic_partition_pruning.q @@ -0,0 +1,192 @@ +set hive.optimize.ppd=true; +set hive.ppd.remove.duplicatefilters=true; +set hive.tez.dynamic.partition.pruning=true; +set hive.optimize.metadataonly=false; +set hive.optimize.index.filter=true; +set hive.vectorized.execution.enabled=true; + + +select distinct ds from srcpart; +select distinct hr from srcpart; + +EXPLAIN create table srcpart_date as select ds as ds, ds as date from srcpart group by ds; +create table srcpart_date stored as orc as select ds as ds, ds as date from srcpart group by ds; +create table srcpart_hour stored as orc as select hr as hr, hr as hour from srcpart group by hr; +create table srcpart_date_hour stored as orc as select ds as ds, ds as date, hr as hr, hr as hour from srcpart group by ds, hr; +create table srcpart_double_hour stored as orc as select (hr*2) as hr, hr as hour from srcpart group by hr; + +-- single column, single key +EXPLAIN select count(*) from srcpart join srcpart_date on (srcpart.ds = srcpart_date.ds) where srcpart_date.date = '2008-04-08'; +select count(*) from srcpart join srcpart_date on (srcpart.ds = srcpart_date.ds) where srcpart_date.date = '2008-04-08'; +set hive.tez.dynamic.partition.pruning=false; +EXPLAIN select count(*) from srcpart join srcpart_date on (srcpart.ds = srcpart_date.ds) where srcpart_date.date = '2008-04-08'; +select count(*) from srcpart join srcpart_date on (srcpart.ds = srcpart_date.ds) where srcpart_date.date = '2008-04-08'; +set hive.tez.dynamic.partition.pruning=true; +select count(*) from srcpart where ds = '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; +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; +set hive.tez.dynamic.partition.pruning=false; +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; +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; +set hive.tez.dynamic.partition.pruning=true; +select count(*) from srcpart where hr = 11 and ds = '2008-04-08'; + +-- 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.tez.dynamic.partition.pruning=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; +set hive.tez.dynamic.partition.pruning=true; +select count(*) from srcpart where ds = '2008-04-08' and hr = 11; + +-- empty set +EXPLAIN select count(*) from srcpart join srcpart_date on (srcpart.ds = srcpart_date.ds) where srcpart_date.date = 'I DONT EXIST'; +select count(*) from srcpart join srcpart_date on (srcpart.ds = srcpart_date.ds) where srcpart_date.date = 'I DONT EXIST'; +set hive.tez.dynamic.partition.pruning=false; +EXPLAIN select count(*) from srcpart join srcpart_date on (srcpart.ds = srcpart_date.ds) where srcpart_date.date = 'I DONT EXIST'; +select count(*) from srcpart join srcpart_date on (srcpart.ds = srcpart_date.ds) where srcpart_date.date = 'I DONT EXIST'; +set hive.tez.dynamic.partition.pruning=true; +select count(*) from srcpart where ds = 'I DONT EXIST'; + +-- expressions +EXPLAIN select count(*) from srcpart join srcpart_double_hour on (srcpart.hr = cast(srcpart_double_hour.hr/2 as int)) where srcpart_double_hour.hour = 11; +select count(*) from srcpart join srcpart_double_hour on (srcpart.hr = cast(srcpart_double_hour.hr/2 as int)) where srcpart_double_hour.hour = 11; +EXPLAIN select count(*) from srcpart join srcpart_double_hour on (srcpart.hr*2 = srcpart_double_hour.hr) where srcpart_double_hour.hour = 11; +select count(*) from srcpart join srcpart_double_hour on (srcpart.hr*2 = srcpart_double_hour.hr) where srcpart_double_hour.hour = 11; +set hive.tez.dynamic.partition.pruning=false; +EXPLAIN select count(*) from srcpart join srcpart_double_hour on (srcpart.hr = cast(srcpart_double_hour.hr/2 as int)) where srcpart_double_hour.hour = 11; +select count(*) from srcpart join srcpart_double_hour on (srcpart.hr = cast(srcpart_double_hour.hr/2 as int)) where srcpart_double_hour.hour = 11; +EXPLAIN select count(*) from srcpart join srcpart_double_hour on (srcpart.hr*2 = srcpart_double_hour.hr) where srcpart_double_hour.hour = 11; +select count(*) from srcpart join srcpart_double_hour on (srcpart.hr*2 = srcpart_double_hour.hr) where srcpart_double_hour.hour = 11; +set hive.tez.dynamic.partition.pruning=true; +select count(*) from srcpart where hr = 11; +EXPLAIN select count(*) from srcpart join srcpart_double_hour on (cast(srcpart.hr*2 as string) = cast(srcpart_double_hour.hr as string)) where srcpart_double_hour.hour = 11; +select count(*) from srcpart join srcpart_double_hour on (cast(srcpart.hr*2 as string) = cast(srcpart_double_hour.hr as string)) where srcpart_double_hour.hour = 11; +set hive.tez.dynamic.partition.pruning=true; +select count(*) from srcpart where cast(hr as string) = 11; + + +-- parent is reduce tasks +EXPLAIN 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 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'; + +-- non-equi join +EXPLAIN select count(*) from srcpart, srcpart_date_hour where (srcpart_date_hour.date = '2008-04-08' and srcpart_date_hour.hour = 11) and (srcpart.ds = srcpart_date_hour.ds or srcpart.hr = srcpart_date_hour.hr); +select count(*) from srcpart, srcpart_date_hour where (srcpart_date_hour.date = '2008-04-08' and srcpart_date_hour.hour = 11) and (srcpart.ds = srcpart_date_hour.ds or srcpart.hr = srcpart_date_hour.hr); + +-- old style join syntax +EXPLAIN select count(*) from srcpart, srcpart_date_hour where srcpart_date_hour.date = '2008-04-08' and srcpart_date_hour.hour = 11 and srcpart.ds = srcpart_date_hour.ds and srcpart.hr = srcpart_date_hour.hr; +select count(*) from srcpart, srcpart_date_hour where srcpart_date_hour.date = '2008-04-08' and srcpart_date_hour.hour = 11 and srcpart.ds = srcpart_date_hour.ds and srcpart.hr = srcpart_date_hour.hr; + +-- 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'; + +-- full outer +EXPLAIN select count(*) from srcpart full outer join srcpart_date on (srcpart.ds = srcpart_date.ds) where srcpart_date.date = '2008-04-08'; + +-- with static pruning +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 and srcpart.hr = 11; +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 and srcpart.hr = 11; +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.hr = 13; +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.hr = 13; + +-- union + subquery +EXPLAIN select count(*) from srcpart where srcpart.ds in (select max(srcpart.ds) from srcpart union all select min(srcpart.ds) from srcpart); +select count(*) from srcpart where srcpart.ds in (select max(srcpart.ds) from srcpart union all select min(srcpart.ds) from srcpart); +EXPLAIN select distinct(ds) from srcpart where srcpart.ds in (select max(srcpart.ds) from srcpart union all select min(srcpart.ds) from srcpart); +select distinct(ds) from srcpart where srcpart.ds in (select max(srcpart.ds) from srcpart union all select min(srcpart.ds) from srcpart); +EXPLAIN select ds from (select distinct(ds) as ds from srcpart union all select distinct(ds) as ds from srcpart) s where s.ds in (select max(srcpart.ds) from srcpart union all select min(srcpart.ds) from srcpart); +select ds from (select distinct(ds) as ds from srcpart union all select distinct(ds) as ds from srcpart) s where s.ds in (select max(srcpart.ds) from srcpart union all select min(srcpart.ds) from srcpart); + +set hive.auto.convert.join=true; +set hive.auto.convert.join.noconditionaltask = true; +set hive.auto.convert.join.noconditionaltask.size = 10000000; + +-- single column, single key +EXPLAIN select count(*) from srcpart join srcpart_date on (srcpart.ds = srcpart_date.ds) where srcpart_date.date = '2008-04-08'; +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'; + +-- 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; +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; +select count(*) from srcpart where hr = 11 and ds = '2008-04-08'; + +-- 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; +select count(*) from srcpart where ds = '2008-04-08' and hr = 11; + +-- empty set +EXPLAIN select count(*) from srcpart join srcpart_date on (srcpart.ds = srcpart_date.ds) where srcpart_date.date = 'I DONT EXIST'; +-- Disabled until TEZ-1486 is fixed +-- select count(*) from srcpart join srcpart_date on (srcpart.ds = srcpart_date.ds) where srcpart_date.date = 'I DONT EXIST'; + +-- expressions +EXPLAIN select count(*) from srcpart join srcpart_double_hour on (srcpart.hr = cast(srcpart_double_hour.hr/2 as int)) where srcpart_double_hour.hour = 11; +select count(*) from srcpart join srcpart_double_hour on (srcpart.hr = cast(srcpart_double_hour.hr/2 as int)) where srcpart_double_hour.hour = 11; +EXPLAIN select count(*) from srcpart join srcpart_double_hour on (srcpart.hr*2 = srcpart_double_hour.hr) where srcpart_double_hour.hour = 11; +select count(*) from srcpart join srcpart_double_hour on (srcpart.hr*2 = srcpart_double_hour.hr) where srcpart_double_hour.hour = 11; +select count(*) from srcpart where hr = 11; + +-- parent is reduce tasks +EXPLAIN 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 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'; + +-- 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'; + +-- full outer +EXPLAIN select count(*) from srcpart full outer join srcpart_date on (srcpart.ds = srcpart_date.ds) where srcpart_date.date = '2008-04-08'; + +-- with static pruning +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 and srcpart.hr = 11; +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 and srcpart.hr = 11; +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.hr = 13; +-- Disabled until TEZ-1486 is fixed +-- 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.hr = 13; + +-- union + subquery +EXPLAIN select distinct(ds) from srcpart where srcpart.ds in (select max(srcpart.ds) from srcpart union all select min(srcpart.ds) from srcpart); +select distinct(ds) from srcpart where srcpart.ds in (select max(srcpart.ds) from srcpart union all select min(srcpart.ds) from srcpart); + + +-- different file format +create table srcpart_orc (key int, value string) partitioned by (ds string, hr int) stored as orc; + + +set hive.exec.dynamic.partition.mode=nonstrict; +set hive.vectorized.execution.enabled=false; +set hive.exec.max.dynamic.partitions=1000; + +insert into table srcpart_orc partition (ds, hr) select key, value, ds, hr from srcpart; +EXPLAIN select count(*) from srcpart_orc join srcpart_date_hour on (srcpart_orc.ds = srcpart_date_hour.ds and srcpart_orc.hr = srcpart_date_hour.hr) where srcpart_date_hour.hour = 11 and (srcpart_date_hour.date = '2008-04-08' or srcpart_date_hour.date = '2008-04-09'); +select count(*) from srcpart_orc join srcpart_date_hour on (srcpart_orc.ds = srcpart_date_hour.ds and srcpart_orc.hr = srcpart_date_hour.hr) where srcpart_date_hour.hour = 11 and (srcpart_date_hour.date = '2008-04-08' or srcpart_date_hour.date = '2008-04-09'); +select count(*) from srcpart where (ds = '2008-04-08' or ds = '2008-04-09') and hr = 11; + +drop table srcpart_orc; +drop table srcpart_date; +drop table srcpart_hour; +drop table srcpart_date_hour; +drop table srcpart_double_hour; diff --git ql/src/test/results/clientpositive/tez/vectorized_dynamic_partition_pruning.q.out ql/src/test/results/clientpositive/tez/vectorized_dynamic_partition_pruning.q.out new file mode 100644 index 0000000..22053ff --- /dev/null +++ ql/src/test/results/clientpositive/tez/vectorized_dynamic_partition_pruning.q.out @@ -0,0 +1,5421 @@ +PREHOOK: query: select distinct ds from srcpart +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 +#### A masked pattern was here #### +POSTHOOK: query: select distinct ds from srcpart +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 +#### A masked pattern was here #### +2008-04-08 +2008-04-09 +PREHOOK: query: select distinct hr from srcpart +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 +#### A masked pattern was here #### +POSTHOOK: query: select distinct hr from srcpart +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 +#### A masked pattern was here #### +11 +12 +PREHOOK: query: EXPLAIN create table srcpart_date as select ds as ds, ds as date from srcpart group by ds +PREHOOK: type: CREATETABLE_AS_SELECT +POSTHOOK: query: EXPLAIN create table srcpart_date as select ds as ds, ds as date from srcpart group by ds +POSTHOOK: type: CREATETABLE_AS_SELECT +STAGE DEPENDENCIES: + Stage-1 is a root stage + Stage-2 depends on stages: Stage-1 + Stage-4 depends on stages: Stage-2, Stage-0 + Stage-3 depends on stages: Stage-4 + Stage-0 depends on stages: Stage-1 + +STAGE PLANS: + Stage: Stage-1 + Tez + Edges: + Reducer 2 <- Map 1 (SIMPLE_EDGE) +#### A masked pattern was here #### + Vertices: + Map 1 + Map Operator Tree: + TableScan + alias: srcpart + Statistics: Num rows: 2000 Data size: 21248 Basic stats: COMPLETE Column stats: COMPLETE + Select Operator + expressions: ds (type: string) + outputColumnNames: ds + Statistics: Num rows: 2000 Data size: 21248 Basic stats: COMPLETE Column stats: COMPLETE + Group By Operator + keys: ds (type: string) + mode: hash + outputColumnNames: _col0 + Statistics: Num rows: 2000 Data size: 368000 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + key expressions: _col0 (type: string) + sort order: + + Map-reduce partition columns: _col0 (type: string) + Statistics: Num rows: 2000 Data size: 368000 Basic stats: COMPLETE Column stats: COMPLETE + Reducer 2 + Reduce Operator Tree: + Group By Operator + keys: KEY._col0 (type: string) + mode: mergepartial + outputColumnNames: _col0 + Statistics: Num rows: 1000 Data size: 184000 Basic stats: COMPLETE Column stats: COMPLETE + Select Operator + expressions: _col0 (type: string), _col0 (type: string) + outputColumnNames: _col0, _col1 + Statistics: Num rows: 1000 Data size: 368000 Basic stats: COMPLETE Column stats: COMPLETE + File Output Operator + compressed: false + Statistics: Num rows: 1000 Data size: 368000 Basic stats: COMPLETE Column stats: COMPLETE + table: + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + name: default.srcpart_date + Execution mode: vectorized + + Stage: Stage-2 + Dependency Collection + + Stage: Stage-4 + Create Table Operator: + Create Table + columns: ds string, date string + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.IgnoreKeyTextOutputFormat + serde name: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + name: default.srcpart_date + + Stage: Stage-3 + Stats-Aggr Operator + + Stage: Stage-0 + Move Operator + files: + hdfs directory: true +#### A masked pattern was here #### + +PREHOOK: query: create table srcpart_date stored as orc as select ds as ds, ds as date from srcpart group by ds +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 +POSTHOOK: query: create table srcpart_date stored as orc as select ds as ds, ds as date from srcpart group by ds +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 +PREHOOK: query: create table srcpart_hour stored as orc as select hr as hr, hr as hour from srcpart group by 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_hour +POSTHOOK: query: create table srcpart_hour stored as orc as select hr as hr, hr as hour from srcpart group by 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_hour +PREHOOK: query: create table srcpart_date_hour stored as orc 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 stored as orc 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 +PREHOOK: query: create table srcpart_double_hour stored as orc as select (hr*2) as hr, hr as hour from srcpart group by 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_double_hour +POSTHOOK: query: create table srcpart_double_hour stored as orc as select (hr*2) as hr, hr as hour from srcpart group by 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_double_hour +PREHOOK: query: -- single column, single key +EXPLAIN select count(*) from srcpart join srcpart_date on (srcpart.ds = srcpart_date.ds) where srcpart_date.date = '2008-04-08' +PREHOOK: type: QUERY +POSTHOOK: query: -- single column, single key +EXPLAIN select count(*) from srcpart join srcpart_date on (srcpart.ds = 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 + Tez + Edges: + Reducer 2 <- Map 1 (SIMPLE_EDGE), Map 4 (SIMPLE_EDGE) + Reducer 3 <- Reducer 2 (SIMPLE_EDGE) +#### 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: COMPLETE + Reduce Output Operator + key expressions: ds (type: string) + sort order: + + Map-reduce partition columns: ds (type: string) + Statistics: Num rows: 2000 Data size: 21248 Basic stats: COMPLETE Column stats: COMPLETE + Map 4 + Map Operator Tree: + TableScan + alias: srcpart_date + filterExpr: (ds is not null and (date = '2008-04-08')) (type: boolean) + Statistics: Num rows: 2 Data size: 376 Basic stats: COMPLETE Column stats: NONE + Filter Operator + predicate: (ds is not null and (date = '2008-04-08')) (type: boolean) + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Reduce Output Operator + key expressions: ds (type: string) + sort order: + + Map-reduce partition columns: ds (type: string) + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Select Operator + expressions: ds (type: string) + outputColumnNames: _col0 + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Group By Operator + keys: _col0 (type: string) + mode: hash + outputColumnNames: _col0 + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Dynamic Partitioning Event Operator + Target Input: srcpart + Partition key expr: ds + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Target column: ds + Target Vertex: Map 1 + Execution mode: vectorized + Reducer 2 + Reduce Operator Tree: + Join Operator + condition map: + Inner Join 0 to 1 + condition expressions: + 0 + 1 + Statistics: Num rows: 2200 Data size: 23372 Basic stats: COMPLETE Column stats: NONE + Select Operator + 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 + Select Operator + expressions: _col0 (type: bigint) + 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.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + Execution mode: vectorized + + Stage: Stage-0 + Fetch Operator + limit: -1 + Processor Tree: + ListSink + +PREHOOK: query: select count(*) from srcpart join srcpart_date on (srcpart.ds = 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 (srcpart.ds = 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) where srcpart_date.date = '2008-04-08' +PREHOOK: type: QUERY +POSTHOOK: query: EXPLAIN select count(*) from srcpart join srcpart_date on (srcpart.ds = 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 + Tez + Edges: + Reducer 2 <- Map 1 (SIMPLE_EDGE), Map 4 (SIMPLE_EDGE) + Reducer 3 <- Reducer 2 (SIMPLE_EDGE) +#### 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: COMPLETE + Reduce Output Operator + key expressions: ds (type: string) + sort order: + + Map-reduce partition columns: ds (type: string) + Statistics: Num rows: 2000 Data size: 21248 Basic stats: COMPLETE Column stats: COMPLETE + Map 4 + Map Operator Tree: + TableScan + alias: srcpart_date + filterExpr: (ds is not null and (date = '2008-04-08')) (type: boolean) + Statistics: Num rows: 2 Data size: 376 Basic stats: COMPLETE Column stats: NONE + Filter Operator + predicate: (ds is not null and (date = '2008-04-08')) (type: boolean) + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Reduce Output Operator + key expressions: ds (type: string) + sort order: + + Map-reduce partition columns: ds (type: string) + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Execution mode: vectorized + Reducer 2 + Reduce Operator Tree: + Join Operator + condition map: + Inner Join 0 to 1 + condition expressions: + 0 + 1 + Statistics: Num rows: 2200 Data size: 23372 Basic stats: COMPLETE Column stats: NONE + Select Operator + 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 + Select Operator + expressions: _col0 (type: bigint) + 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.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + Execution mode: vectorized + + Stage: Stage-0 + Fetch Operator + limit: -1 + Processor Tree: + ListSink + +PREHOOK: query: select count(*) from srcpart join srcpart_date on (srcpart.ds = 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 (srcpart.ds = 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: select count(*) from srcpart where ds = '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 +#### A masked pattern was here #### +POSTHOOK: query: select count(*) from srcpart where ds = '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 +#### A masked pattern was here #### +1000 +PREHOOK: query: -- 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 +PREHOOK: type: QUERY +POSTHOOK: query: -- 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 +POSTHOOK: type: QUERY +STAGE DEPENDENCIES: + Stage-1 is a root stage + Stage-0 depends on stages: Stage-1 + +STAGE PLANS: + Stage: Stage-1 + Tez + Edges: + Reducer 2 <- Map 1 (SIMPLE_EDGE), Map 6 (SIMPLE_EDGE) + Reducer 3 <- Map 5 (SIMPLE_EDGE), Reducer 2 (SIMPLE_EDGE) + Reducer 4 <- Reducer 3 (SIMPLE_EDGE) +#### 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: COMPLETE + Reduce Output Operator + key expressions: ds (type: string) + sort order: + + Map-reduce partition columns: ds (type: string) + Statistics: Num rows: 2000 Data size: 21248 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: hr (type: string) + Map 5 + Map Operator Tree: + TableScan + alias: srcpart_hour + filterExpr: (hr is not null and (hour = 11)) (type: boolean) + Statistics: Num rows: 2 Data size: 344 Basic stats: COMPLETE Column stats: NONE + Filter Operator + predicate: (hr is not null and (hour = 11)) (type: boolean) + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Reduce Output Operator + key expressions: hr (type: string) + sort order: + + Map-reduce partition columns: hr (type: string) + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Select Operator + expressions: hr (type: string) + outputColumnNames: _col0 + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Group By Operator + keys: _col0 (type: string) + mode: hash + outputColumnNames: _col0 + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Dynamic Partitioning Event Operator + Target Input: srcpart + Partition key expr: hr + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Target column: hr + Target Vertex: Map 1 + Map 6 + Map Operator Tree: + TableScan + alias: srcpart_date + filterExpr: (ds is not null and (date = '2008-04-08')) (type: boolean) + Statistics: Num rows: 2 Data size: 376 Basic stats: COMPLETE Column stats: NONE + Filter Operator + predicate: (ds is not null and (date = '2008-04-08')) (type: boolean) + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Reduce Output Operator + key expressions: ds (type: string) + sort order: + + Map-reduce partition columns: ds (type: string) + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Select Operator + expressions: ds (type: string) + outputColumnNames: _col0 + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Group By Operator + keys: _col0 (type: string) + mode: hash + outputColumnNames: _col0 + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Dynamic Partitioning Event Operator + Target Input: srcpart + Partition key expr: ds + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Target column: ds + Target Vertex: Map 1 + Execution mode: vectorized + Reducer 2 + Reduce Operator Tree: + Join Operator + condition map: + Inner Join 0 to 1 + condition expressions: + 0 {VALUE._col2} + 1 + outputColumnNames: _col3 + Statistics: Num rows: 2200 Data size: 23372 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + key expressions: _col3 (type: string) + sort order: + + Map-reduce partition columns: _col3 (type: string) + Statistics: Num rows: 2200 Data size: 23372 Basic stats: COMPLETE Column stats: NONE + Reducer 3 + Reduce Operator Tree: + Join Operator + condition map: + Inner Join 0 to 1 + condition expressions: + 0 + 1 + Statistics: Num rows: 2420 Data size: 25709 Basic stats: COMPLETE Column stats: NONE + Select Operator + Statistics: Num rows: 2420 Data size: 25709 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 4 + 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 + Select Operator + expressions: _col0 (type: bigint) + 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.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + Execution mode: vectorized + + Stage: Stage-0 + Fetch Operator + limit: -1 + Processor Tree: + ListSink + +PREHOOK: query: 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 +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 +PREHOOK: Input: default@srcpart_hour +#### A masked pattern was here #### +POSTHOOK: query: 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 +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 +POSTHOOK: Input: default@srcpart_hour +#### A masked pattern was here #### +500 +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 +POSTHOOK: 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 +POSTHOOK: type: QUERY +STAGE DEPENDENCIES: + Stage-1 is a root stage + Stage-0 depends on stages: Stage-1 + +STAGE PLANS: + Stage: Stage-1 + Tez + Edges: + Reducer 2 <- Map 1 (SIMPLE_EDGE), Map 6 (SIMPLE_EDGE) + Reducer 3 <- Map 5 (SIMPLE_EDGE), Reducer 2 (SIMPLE_EDGE) + Reducer 4 <- Reducer 3 (SIMPLE_EDGE) +#### A masked pattern was here #### + Vertices: + Map 1 + Map Operator Tree: + TableScan + alias: srcpart + filterExpr: (ds is not null and hr is not null) (type: boolean) + Statistics: Num rows: 2000 Data size: 21248 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + key expressions: ds (type: string) + sort order: + + Map-reduce partition columns: ds (type: string) + Statistics: Num rows: 2000 Data size: 21248 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: hr (type: string) + Map 5 + Map Operator Tree: + TableScan + alias: srcpart_hour + filterExpr: (hr is not null and (hour = 11)) (type: boolean) + Statistics: Num rows: 2 Data size: 344 Basic stats: COMPLETE Column stats: NONE + Filter Operator + predicate: (hr is not null and (hour = 11)) (type: boolean) + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Reduce Output Operator + key expressions: hr (type: string) + sort order: + + Map-reduce partition columns: hr (type: string) + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Map 6 + Map Operator Tree: + TableScan + alias: srcpart_date + filterExpr: (ds is not null and (date = '2008-04-08')) (type: boolean) + Statistics: Num rows: 2 Data size: 376 Basic stats: COMPLETE Column stats: NONE + Filter Operator + predicate: (ds is not null and (date = '2008-04-08')) (type: boolean) + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Reduce Output Operator + key expressions: ds (type: string) + sort order: + + Map-reduce partition columns: ds (type: string) + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Execution mode: vectorized + Reducer 2 + Reduce Operator Tree: + Join Operator + condition map: + Inner Join 0 to 1 + condition expressions: + 0 {VALUE._col2} + 1 + outputColumnNames: _col3 + Statistics: Num rows: 2200 Data size: 23372 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + key expressions: _col3 (type: string) + sort order: + + Map-reduce partition columns: _col3 (type: string) + Statistics: Num rows: 2200 Data size: 23372 Basic stats: COMPLETE Column stats: NONE + Reducer 3 + Reduce Operator Tree: + Join Operator + condition map: + Inner Join 0 to 1 + condition expressions: + 0 + 1 + Statistics: Num rows: 2420 Data size: 25709 Basic stats: COMPLETE Column stats: NONE + Select Operator + Statistics: Num rows: 2420 Data size: 25709 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 4 + 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 + Select Operator + expressions: _col0 (type: bigint) + 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.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + Execution mode: vectorized + + Stage: Stage-0 + Fetch Operator + limit: -1 + Processor Tree: + ListSink + +PREHOOK: query: 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 +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 +PREHOOK: Input: default@srcpart_hour +#### A masked pattern was here #### +POSTHOOK: query: 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 +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 +POSTHOOK: Input: default@srcpart_hour +#### A masked pattern was here #### +500 +PREHOOK: query: select count(*) from srcpart where hr = 11 and ds = '2008-04-08' +PREHOOK: type: QUERY +PREHOOK: Input: default@srcpart +PREHOOK: Input: default@srcpart@ds=2008-04-08/hr=11 +#### A masked pattern was here #### +POSTHOOK: query: select count(*) from srcpart where hr = 11 and ds = '2008-04-08' +POSTHOOK: type: QUERY +POSTHOOK: Input: default@srcpart +POSTHOOK: Input: default@srcpart@ds=2008-04-08/hr=11 +#### A masked pattern was here #### +500 +PREHOOK: query: -- 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 +PREHOOK: type: QUERY +POSTHOOK: query: -- 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 +POSTHOOK: type: QUERY +STAGE DEPENDENCIES: + Stage-1 is a root stage + Stage-0 depends on stages: Stage-1 + +STAGE PLANS: + Stage: Stage-1 + Tez + Edges: + Reducer 2 <- Map 1 (SIMPLE_EDGE), Map 4 (SIMPLE_EDGE) + Reducer 3 <- Reducer 2 (SIMPLE_EDGE) +#### 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: COMPLETE + Reduce Output Operator + key expressions: ds (type: string), hr (type: string) + sort order: ++ + Map-reduce partition columns: ds (type: string), hr (type: string) + Statistics: Num rows: 2000 Data size: 21248 Basic stats: COMPLETE Column stats: COMPLETE + Map 4 + Map Operator Tree: + TableScan + alias: srcpart_date_hour + filterExpr: (((ds is not null and hr is not null) and (date = '2008-04-08')) and (hour = 11)) (type: boolean) + Statistics: Num rows: 4 Data size: 1440 Basic stats: COMPLETE Column stats: NONE + Filter Operator + predicate: (((ds is not null and hr is not null) and (date = '2008-04-08')) and (hour = 11)) (type: boolean) + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Reduce Output Operator + key expressions: ds (type: string), hr (type: string) + sort order: ++ + Map-reduce partition columns: ds (type: string), hr (type: string) + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Select Operator + expressions: ds (type: string) + outputColumnNames: _col0 + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Group By Operator + keys: _col0 (type: string) + mode: hash + outputColumnNames: _col0 + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Dynamic Partitioning Event Operator + Target Input: srcpart + Partition key expr: ds + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Target column: ds + Target Vertex: Map 1 + Select Operator + expressions: hr (type: string) + outputColumnNames: _col0 + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Group By Operator + keys: _col0 (type: string) + mode: hash + outputColumnNames: _col0 + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Dynamic Partitioning Event Operator + Target Input: srcpart + Partition key expr: hr + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Target column: hr + Target Vertex: Map 1 + Reducer 2 + Reduce Operator Tree: + Join Operator + condition map: + Inner Join 0 to 1 + condition expressions: + 0 + 1 + Statistics: Num rows: 2200 Data size: 23372 Basic stats: COMPLETE Column stats: NONE + Select Operator + 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 + Select Operator + expressions: _col0 (type: bigint) + 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.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + Execution mode: vectorized + + 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-1 is a root stage + Stage-0 depends on stages: Stage-1 + +STAGE PLANS: + Stage: Stage-1 + Tez + Edges: + Reducer 2 <- Map 1 (SIMPLE_EDGE), Map 4 (SIMPLE_EDGE) + Reducer 3 <- Reducer 2 (SIMPLE_EDGE) +#### A masked pattern was here #### + Vertices: + Map 1 + Map Operator Tree: + TableScan + alias: srcpart + filterExpr: (ds is not null and hr is not null) (type: boolean) + Statistics: Num rows: 2000 Data size: 21248 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + key expressions: ds (type: string), hr (type: string) + sort order: ++ + Map-reduce partition columns: ds (type: string), hr (type: string) + Statistics: Num rows: 2000 Data size: 21248 Basic stats: COMPLETE Column stats: COMPLETE + Map 4 + Map Operator Tree: + TableScan + alias: srcpart_date_hour + filterExpr: (((ds is not null and hr is not null) and (date = '2008-04-08')) and (hour = 11)) (type: boolean) + Statistics: Num rows: 4 Data size: 1440 Basic stats: COMPLETE Column stats: NONE + Filter Operator + predicate: (((ds is not null and hr is not null) and (date = '2008-04-08')) and (hour = 11)) (type: boolean) + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Reduce Output Operator + key expressions: ds (type: string), hr (type: string) + sort order: ++ + Map-reduce partition columns: ds (type: string), hr (type: string) + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Reducer 2 + Reduce Operator Tree: + Join Operator + condition map: + Inner Join 0 to 1 + condition expressions: + 0 + 1 + Statistics: Num rows: 2200 Data size: 23372 Basic stats: COMPLETE Column stats: NONE + Select Operator + 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 + Select Operator + expressions: _col0 (type: bigint) + 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.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + Execution mode: vectorized + + 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: select count(*) from srcpart where ds = '2008-04-08' and hr = 11 +PREHOOK: type: QUERY +PREHOOK: Input: default@srcpart +PREHOOK: Input: default@srcpart@ds=2008-04-08/hr=11 +#### A masked pattern was here #### +POSTHOOK: query: select count(*) from srcpart where ds = '2008-04-08' and hr = 11 +POSTHOOK: type: QUERY +POSTHOOK: Input: default@srcpart +POSTHOOK: Input: default@srcpart@ds=2008-04-08/hr=11 +#### A masked pattern was here #### +500 +PREHOOK: query: -- empty set +EXPLAIN select count(*) from srcpart join srcpart_date on (srcpart.ds = srcpart_date.ds) where srcpart_date.date = 'I DONT EXIST' +PREHOOK: type: QUERY +POSTHOOK: query: -- empty set +EXPLAIN select count(*) from srcpart join srcpart_date on (srcpart.ds = srcpart_date.ds) where srcpart_date.date = 'I DONT EXIST' +POSTHOOK: type: QUERY +STAGE DEPENDENCIES: + Stage-1 is a root stage + Stage-0 depends on stages: Stage-1 + +STAGE PLANS: + Stage: Stage-1 + Tez + Edges: + Reducer 2 <- Map 1 (SIMPLE_EDGE), Map 4 (SIMPLE_EDGE) + Reducer 3 <- Reducer 2 (SIMPLE_EDGE) +#### 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: COMPLETE + Reduce Output Operator + key expressions: ds (type: string) + sort order: + + Map-reduce partition columns: ds (type: string) + Statistics: Num rows: 2000 Data size: 21248 Basic stats: COMPLETE Column stats: COMPLETE + Map 4 + Map Operator Tree: + TableScan + alias: srcpart_date + filterExpr: (ds is not null and (date = 'I DONT EXIST')) (type: boolean) + Statistics: Num rows: 2 Data size: 376 Basic stats: COMPLETE Column stats: NONE + Filter Operator + predicate: (ds is not null and (date = 'I DONT EXIST')) (type: boolean) + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Reduce Output Operator + key expressions: ds (type: string) + sort order: + + Map-reduce partition columns: ds (type: string) + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Select Operator + expressions: ds (type: string) + outputColumnNames: _col0 + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Group By Operator + keys: _col0 (type: string) + mode: hash + outputColumnNames: _col0 + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Dynamic Partitioning Event Operator + Target Input: srcpart + Partition key expr: ds + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Target column: ds + Target Vertex: Map 1 + Execution mode: vectorized + Reducer 2 + Reduce Operator Tree: + Join Operator + condition map: + Inner Join 0 to 1 + condition expressions: + 0 + 1 + Statistics: Num rows: 2200 Data size: 23372 Basic stats: COMPLETE Column stats: NONE + Select Operator + 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 + Select Operator + expressions: _col0 (type: bigint) + 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.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + Execution mode: vectorized + + Stage: Stage-0 + Fetch Operator + limit: -1 + Processor Tree: + ListSink + +PREHOOK: query: select count(*) from srcpart join srcpart_date on (srcpart.ds = srcpart_date.ds) where srcpart_date.date = 'I DONT EXIST' +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 (srcpart.ds = srcpart_date.ds) where srcpart_date.date = 'I DONT EXIST' +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 #### +0 +PREHOOK: query: EXPLAIN select count(*) from srcpart join srcpart_date on (srcpart.ds = srcpart_date.ds) where srcpart_date.date = 'I DONT EXIST' +PREHOOK: type: QUERY +POSTHOOK: query: EXPLAIN select count(*) from srcpart join srcpart_date on (srcpart.ds = srcpart_date.ds) where srcpart_date.date = 'I DONT EXIST' +POSTHOOK: type: QUERY +STAGE DEPENDENCIES: + Stage-1 is a root stage + Stage-0 depends on stages: Stage-1 + +STAGE PLANS: + Stage: Stage-1 + Tez + Edges: + Reducer 2 <- Map 1 (SIMPLE_EDGE), Map 4 (SIMPLE_EDGE) + Reducer 3 <- Reducer 2 (SIMPLE_EDGE) +#### 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: COMPLETE + Reduce Output Operator + key expressions: ds (type: string) + sort order: + + Map-reduce partition columns: ds (type: string) + Statistics: Num rows: 2000 Data size: 21248 Basic stats: COMPLETE Column stats: COMPLETE + Map 4 + Map Operator Tree: + TableScan + alias: srcpart_date + filterExpr: (ds is not null and (date = 'I DONT EXIST')) (type: boolean) + Statistics: Num rows: 2 Data size: 376 Basic stats: COMPLETE Column stats: NONE + Filter Operator + predicate: (ds is not null and (date = 'I DONT EXIST')) (type: boolean) + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Reduce Output Operator + key expressions: ds (type: string) + sort order: + + Map-reduce partition columns: ds (type: string) + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Execution mode: vectorized + Reducer 2 + Reduce Operator Tree: + Join Operator + condition map: + Inner Join 0 to 1 + condition expressions: + 0 + 1 + Statistics: Num rows: 2200 Data size: 23372 Basic stats: COMPLETE Column stats: NONE + Select Operator + 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 + Select Operator + expressions: _col0 (type: bigint) + 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.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + Execution mode: vectorized + + Stage: Stage-0 + Fetch Operator + limit: -1 + Processor Tree: + ListSink + +PREHOOK: query: select count(*) from srcpart join srcpart_date on (srcpart.ds = srcpart_date.ds) where srcpart_date.date = 'I DONT EXIST' +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 (srcpart.ds = srcpart_date.ds) where srcpart_date.date = 'I DONT EXIST' +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 #### +0 +PREHOOK: query: select count(*) from srcpart where ds = 'I DONT EXIST' +PREHOOK: type: QUERY +PREHOOK: Input: default@srcpart +#### A masked pattern was here #### +POSTHOOK: query: select count(*) from srcpart where ds = 'I DONT EXIST' +POSTHOOK: type: QUERY +POSTHOOK: Input: default@srcpart +#### A masked pattern was here #### +0 +PREHOOK: query: -- expressions +EXPLAIN select count(*) from srcpart join srcpart_double_hour on (srcpart.hr = cast(srcpart_double_hour.hr/2 as int)) where srcpart_double_hour.hour = 11 +PREHOOK: type: QUERY +POSTHOOK: query: -- expressions +EXPLAIN select count(*) from srcpart join srcpart_double_hour on (srcpart.hr = cast(srcpart_double_hour.hr/2 as int)) where srcpart_double_hour.hour = 11 +POSTHOOK: type: QUERY +STAGE DEPENDENCIES: + Stage-1 is a root stage + Stage-0 depends on stages: Stage-1 + +STAGE PLANS: + Stage: Stage-1 + Tez + Edges: + Reducer 2 <- Map 1 (SIMPLE_EDGE), Map 4 (SIMPLE_EDGE) + Reducer 3 <- Reducer 2 (SIMPLE_EDGE) +#### A masked pattern was here #### + Vertices: + Map 1 + Map Operator Tree: + TableScan + alias: srcpart + filterExpr: UDFToDouble(hr) is not null (type: boolean) + Statistics: Num rows: 2000 Data size: 21248 Basic stats: COMPLETE Column stats: COMPLETE + Filter Operator + predicate: UDFToDouble(hr) is not null (type: boolean) + Statistics: Num rows: 1000 Data size: 0 Basic stats: PARTIAL Column stats: COMPLETE + Reduce Output Operator + key expressions: UDFToDouble(hr) (type: double) + sort order: + + Map-reduce partition columns: UDFToDouble(hr) (type: double) + Statistics: Num rows: 1000 Data size: 0 Basic stats: PARTIAL Column stats: COMPLETE + Map 4 + Map Operator Tree: + TableScan + alias: srcpart_double_hour + filterExpr: (UDFToDouble(UDFToInteger((hr / 2))) is not null and (hour = 11)) (type: boolean) + Statistics: Num rows: 2 Data size: 188 Basic stats: COMPLETE Column stats: NONE + Filter Operator + predicate: (UDFToDouble(UDFToInteger((hr / 2))) is not null and (hour = 11)) (type: boolean) + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Reduce Output Operator + key expressions: UDFToDouble(UDFToInteger((hr / 2))) (type: double) + sort order: + + Map-reduce partition columns: UDFToDouble(UDFToInteger((hr / 2))) (type: double) + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Select Operator + expressions: UDFToDouble(UDFToInteger((hr / 2))) (type: double) + outputColumnNames: _col0 + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Group By Operator + keys: _col0 (type: double) + mode: hash + outputColumnNames: _col0 + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Dynamic Partitioning Event Operator + Target Input: srcpart + Partition key expr: UDFToDouble(hr) + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Target column: hr + Target Vertex: Map 1 + Reducer 2 + Reduce Operator Tree: + Join Operator + condition map: + Inner Join 0 to 1 + condition expressions: + 0 + 1 + Statistics: Num rows: 1100 Data size: 0 Basic stats: PARTIAL Column stats: NONE + Select Operator + Statistics: Num rows: 1100 Data size: 0 Basic stats: PARTIAL 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 + Select Operator + expressions: _col0 (type: bigint) + 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.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + Execution mode: vectorized + + Stage: Stage-0 + Fetch Operator + limit: -1 + Processor Tree: + ListSink + +PREHOOK: query: select count(*) from srcpart join srcpart_double_hour on (srcpart.hr = cast(srcpart_double_hour.hr/2 as int)) where srcpart_double_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_double_hour +#### A masked pattern was here #### +POSTHOOK: query: select count(*) from srcpart join srcpart_double_hour on (srcpart.hr = cast(srcpart_double_hour.hr/2 as int)) where srcpart_double_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_double_hour +#### A masked pattern was here #### +1000 +PREHOOK: query: EXPLAIN select count(*) from srcpart join srcpart_double_hour on (srcpart.hr*2 = srcpart_double_hour.hr) where srcpart_double_hour.hour = 11 +PREHOOK: type: QUERY +POSTHOOK: query: EXPLAIN select count(*) from srcpart join srcpart_double_hour on (srcpart.hr*2 = srcpart_double_hour.hr) where srcpart_double_hour.hour = 11 +POSTHOOK: type: QUERY +STAGE DEPENDENCIES: + Stage-1 is a root stage + Stage-0 depends on stages: Stage-1 + +STAGE PLANS: + Stage: Stage-1 + Tez + Edges: + Reducer 2 <- Map 1 (SIMPLE_EDGE), Map 4 (SIMPLE_EDGE) + Reducer 3 <- Reducer 2 (SIMPLE_EDGE) +#### A masked pattern was here #### + Vertices: + Map 1 + Map Operator Tree: + TableScan + alias: srcpart + filterExpr: (hr * 2) is not null (type: boolean) + Statistics: Num rows: 2000 Data size: 21248 Basic stats: COMPLETE Column stats: COMPLETE + Filter Operator + predicate: (hr * 2) is not null (type: boolean) + Statistics: Num rows: 1000 Data size: 0 Basic stats: PARTIAL Column stats: COMPLETE + Reduce Output Operator + key expressions: (hr * 2) (type: double) + sort order: + + Map-reduce partition columns: (hr * 2) (type: double) + Statistics: Num rows: 1000 Data size: 0 Basic stats: PARTIAL Column stats: COMPLETE + Map 4 + Map Operator Tree: + TableScan + alias: srcpart_double_hour + filterExpr: (hr is not null and (hour = 11)) (type: boolean) + Statistics: Num rows: 2 Data size: 188 Basic stats: COMPLETE Column stats: NONE + Filter Operator + predicate: (hr is not null and (hour = 11)) (type: boolean) + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Reduce Output Operator + key expressions: hr (type: double) + sort order: + + Map-reduce partition columns: hr (type: double) + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Select Operator + expressions: hr (type: double) + outputColumnNames: _col0 + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Group By Operator + keys: _col0 (type: double) + mode: hash + outputColumnNames: _col0 + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Dynamic Partitioning Event Operator + Target Input: srcpart + Partition key expr: (hr * 2) + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Target column: hr + Target Vertex: Map 1 + Reducer 2 + Reduce Operator Tree: + Join Operator + condition map: + Inner Join 0 to 1 + condition expressions: + 0 + 1 + Statistics: Num rows: 1100 Data size: 0 Basic stats: PARTIAL Column stats: NONE + Select Operator + Statistics: Num rows: 1100 Data size: 0 Basic stats: PARTIAL 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 + Select Operator + expressions: _col0 (type: bigint) + 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.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + Execution mode: vectorized + + Stage: Stage-0 + Fetch Operator + limit: -1 + Processor Tree: + ListSink + +PREHOOK: query: select count(*) from srcpart join srcpart_double_hour on (srcpart.hr*2 = srcpart_double_hour.hr) where srcpart_double_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_double_hour +#### A masked pattern was here #### +POSTHOOK: query: select count(*) from srcpart join srcpart_double_hour on (srcpart.hr*2 = srcpart_double_hour.hr) where srcpart_double_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_double_hour +#### A masked pattern was here #### +1000 +PREHOOK: query: EXPLAIN select count(*) from srcpart join srcpart_double_hour on (srcpart.hr = cast(srcpart_double_hour.hr/2 as int)) where srcpart_double_hour.hour = 11 +PREHOOK: type: QUERY +POSTHOOK: query: EXPLAIN select count(*) from srcpart join srcpart_double_hour on (srcpart.hr = cast(srcpart_double_hour.hr/2 as int)) where srcpart_double_hour.hour = 11 +POSTHOOK: type: QUERY +STAGE DEPENDENCIES: + Stage-1 is a root stage + Stage-0 depends on stages: Stage-1 + +STAGE PLANS: + Stage: Stage-1 + Tez + Edges: + Reducer 2 <- Map 1 (SIMPLE_EDGE), Map 4 (SIMPLE_EDGE) + Reducer 3 <- Reducer 2 (SIMPLE_EDGE) +#### A masked pattern was here #### + Vertices: + Map 1 + Map Operator Tree: + TableScan + alias: srcpart + filterExpr: UDFToDouble(hr) is not null (type: boolean) + Statistics: Num rows: 2000 Data size: 21248 Basic stats: COMPLETE Column stats: COMPLETE + Filter Operator + predicate: UDFToDouble(hr) is not null (type: boolean) + Statistics: Num rows: 1000 Data size: 0 Basic stats: PARTIAL Column stats: COMPLETE + Reduce Output Operator + key expressions: UDFToDouble(hr) (type: double) + sort order: + + Map-reduce partition columns: UDFToDouble(hr) (type: double) + Statistics: Num rows: 1000 Data size: 0 Basic stats: PARTIAL Column stats: COMPLETE + Map 4 + Map Operator Tree: + TableScan + alias: srcpart_double_hour + filterExpr: (UDFToDouble(UDFToInteger((hr / 2))) is not null and (hour = 11)) (type: boolean) + Statistics: Num rows: 2 Data size: 188 Basic stats: COMPLETE Column stats: NONE + Filter Operator + predicate: (UDFToDouble(UDFToInteger((hr / 2))) is not null and (hour = 11)) (type: boolean) + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Reduce Output Operator + key expressions: UDFToDouble(UDFToInteger((hr / 2))) (type: double) + sort order: + + Map-reduce partition columns: UDFToDouble(UDFToInteger((hr / 2))) (type: double) + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Reducer 2 + Reduce Operator Tree: + Join Operator + condition map: + Inner Join 0 to 1 + condition expressions: + 0 + 1 + Statistics: Num rows: 1100 Data size: 0 Basic stats: PARTIAL Column stats: NONE + Select Operator + Statistics: Num rows: 1100 Data size: 0 Basic stats: PARTIAL 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 + Select Operator + expressions: _col0 (type: bigint) + 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.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + Execution mode: vectorized + + Stage: Stage-0 + Fetch Operator + limit: -1 + Processor Tree: + ListSink + +PREHOOK: query: select count(*) from srcpart join srcpart_double_hour on (srcpart.hr = cast(srcpart_double_hour.hr/2 as int)) where srcpart_double_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_double_hour +#### A masked pattern was here #### +POSTHOOK: query: select count(*) from srcpart join srcpart_double_hour on (srcpart.hr = cast(srcpart_double_hour.hr/2 as int)) where srcpart_double_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_double_hour +#### A masked pattern was here #### +1000 +PREHOOK: query: EXPLAIN select count(*) from srcpart join srcpart_double_hour on (srcpart.hr*2 = srcpart_double_hour.hr) where srcpart_double_hour.hour = 11 +PREHOOK: type: QUERY +POSTHOOK: query: EXPLAIN select count(*) from srcpart join srcpart_double_hour on (srcpart.hr*2 = srcpart_double_hour.hr) where srcpart_double_hour.hour = 11 +POSTHOOK: type: QUERY +STAGE DEPENDENCIES: + Stage-1 is a root stage + Stage-0 depends on stages: Stage-1 + +STAGE PLANS: + Stage: Stage-1 + Tez + Edges: + Reducer 2 <- Map 1 (SIMPLE_EDGE), Map 4 (SIMPLE_EDGE) + Reducer 3 <- Reducer 2 (SIMPLE_EDGE) +#### A masked pattern was here #### + Vertices: + Map 1 + Map Operator Tree: + TableScan + alias: srcpart + filterExpr: (hr * 2) is not null (type: boolean) + Statistics: Num rows: 2000 Data size: 21248 Basic stats: COMPLETE Column stats: COMPLETE + Filter Operator + predicate: (hr * 2) is not null (type: boolean) + Statistics: Num rows: 1000 Data size: 0 Basic stats: PARTIAL Column stats: COMPLETE + Reduce Output Operator + key expressions: (hr * 2) (type: double) + sort order: + + Map-reduce partition columns: (hr * 2) (type: double) + Statistics: Num rows: 1000 Data size: 0 Basic stats: PARTIAL Column stats: COMPLETE + Map 4 + Map Operator Tree: + TableScan + alias: srcpart_double_hour + filterExpr: (hr is not null and (hour = 11)) (type: boolean) + Statistics: Num rows: 2 Data size: 188 Basic stats: COMPLETE Column stats: NONE + Filter Operator + predicate: (hr is not null and (hour = 11)) (type: boolean) + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Reduce Output Operator + key expressions: hr (type: double) + sort order: + + Map-reduce partition columns: hr (type: double) + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Reducer 2 + Reduce Operator Tree: + Join Operator + condition map: + Inner Join 0 to 1 + condition expressions: + 0 + 1 + Statistics: Num rows: 1100 Data size: 0 Basic stats: PARTIAL Column stats: NONE + Select Operator + Statistics: Num rows: 1100 Data size: 0 Basic stats: PARTIAL 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 + Select Operator + expressions: _col0 (type: bigint) + 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.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + Execution mode: vectorized + + Stage: Stage-0 + Fetch Operator + limit: -1 + Processor Tree: + ListSink + +PREHOOK: query: select count(*) from srcpart join srcpart_double_hour on (srcpart.hr*2 = srcpart_double_hour.hr) where srcpart_double_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_double_hour +#### A masked pattern was here #### +POSTHOOK: query: select count(*) from srcpart join srcpart_double_hour on (srcpart.hr*2 = srcpart_double_hour.hr) where srcpart_double_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_double_hour +#### A masked pattern was here #### +1000 +PREHOOK: query: select count(*) from srcpart where hr = 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-09/hr=11 +#### A masked pattern was here #### +POSTHOOK: query: select count(*) from srcpart where hr = 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-09/hr=11 +#### A masked pattern was here #### +1000 +PREHOOK: query: EXPLAIN select count(*) from srcpart join srcpart_double_hour on (cast(srcpart.hr*2 as string) = cast(srcpart_double_hour.hr as string)) where srcpart_double_hour.hour = 11 +PREHOOK: type: QUERY +POSTHOOK: query: EXPLAIN select count(*) from srcpart join srcpart_double_hour on (cast(srcpart.hr*2 as string) = cast(srcpart_double_hour.hr as string)) where srcpart_double_hour.hour = 11 +POSTHOOK: type: QUERY +STAGE DEPENDENCIES: + Stage-1 is a root stage + Stage-0 depends on stages: Stage-1 + +STAGE PLANS: + Stage: Stage-1 + Tez + Edges: + Reducer 2 <- Map 1 (SIMPLE_EDGE), Map 4 (SIMPLE_EDGE) + Reducer 3 <- Reducer 2 (SIMPLE_EDGE) +#### A masked pattern was here #### + Vertices: + Map 1 + Map Operator Tree: + TableScan + alias: srcpart + filterExpr: UDFToString((hr * 2)) is not null (type: boolean) + Statistics: Num rows: 2000 Data size: 21248 Basic stats: COMPLETE Column stats: COMPLETE + Filter Operator + predicate: UDFToString((hr * 2)) is not null (type: boolean) + Statistics: Num rows: 1000 Data size: 0 Basic stats: PARTIAL Column stats: COMPLETE + Reduce Output Operator + key expressions: UDFToString((hr * 2)) (type: string) + sort order: + + Map-reduce partition columns: UDFToString((hr * 2)) (type: string) + Statistics: Num rows: 1000 Data size: 0 Basic stats: PARTIAL Column stats: COMPLETE + Map 4 + Map Operator Tree: + TableScan + alias: srcpart_double_hour + filterExpr: (UDFToString(hr) is not null and (hour = 11)) (type: boolean) + Statistics: Num rows: 2 Data size: 188 Basic stats: COMPLETE Column stats: NONE + Filter Operator + predicate: (UDFToString(hr) is not null and (hour = 11)) (type: boolean) + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Reduce Output Operator + key expressions: UDFToString(hr) (type: string) + sort order: + + Map-reduce partition columns: UDFToString(hr) (type: string) + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Select Operator + expressions: UDFToString(hr) (type: string) + outputColumnNames: _col0 + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Group By Operator + keys: _col0 (type: string) + mode: hash + outputColumnNames: _col0 + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Dynamic Partitioning Event Operator + Target Input: srcpart + Partition key expr: UDFToString((hr * 2)) + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Target column: hr + Target Vertex: Map 1 + Reducer 2 + Reduce Operator Tree: + Join Operator + condition map: + Inner Join 0 to 1 + condition expressions: + 0 + 1 + Statistics: Num rows: 1100 Data size: 0 Basic stats: PARTIAL Column stats: NONE + Select Operator + Statistics: Num rows: 1100 Data size: 0 Basic stats: PARTIAL 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 + Select Operator + expressions: _col0 (type: bigint) + 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.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + Execution mode: vectorized + + Stage: Stage-0 + Fetch Operator + limit: -1 + Processor Tree: + ListSink + +PREHOOK: query: select count(*) from srcpart join srcpart_double_hour on (cast(srcpart.hr*2 as string) = cast(srcpart_double_hour.hr as string)) where srcpart_double_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_double_hour +#### A masked pattern was here #### +POSTHOOK: query: select count(*) from srcpart join srcpart_double_hour on (cast(srcpart.hr*2 as string) = cast(srcpart_double_hour.hr as string)) where srcpart_double_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_double_hour +#### A masked pattern was here #### +1000 +PREHOOK: query: select count(*) from srcpart where cast(hr as string) = 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-09/hr=11 +#### A masked pattern was here #### +POSTHOOK: query: select count(*) from srcpart where cast(hr as string) = 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-09/hr=11 +#### A masked pattern was here #### +1000 +PREHOOK: query: -- parent is reduce tasks +EXPLAIN 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' +PREHOOK: type: QUERY +POSTHOOK: query: -- parent is reduce tasks +EXPLAIN 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' +POSTHOOK: type: QUERY +STAGE DEPENDENCIES: + Stage-1 is a root stage + Stage-0 depends on stages: Stage-1 + +STAGE PLANS: + Stage: Stage-1 + Tez + Edges: + Reducer 2 <- Map 1 (SIMPLE_EDGE), Reducer 5 (SIMPLE_EDGE) + Reducer 3 <- Reducer 2 (SIMPLE_EDGE) + Reducer 5 <- Map 4 (SIMPLE_EDGE) +#### 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: COMPLETE + Reduce Output Operator + key expressions: ds (type: string) + sort order: + + Map-reduce partition columns: ds (type: string) + Statistics: Num rows: 2000 Data size: 21248 Basic stats: COMPLETE Column stats: COMPLETE + Select Operator + expressions: ds (type: string) + outputColumnNames: _col0 + Statistics: Num rows: 2000 Data size: 0 Basic stats: PARTIAL Column stats: COMPLETE + Group By Operator + keys: _col0 (type: string) + mode: hash + outputColumnNames: _col0 + Statistics: Num rows: 2000 Data size: 0 Basic stats: PARTIAL Column stats: COMPLETE + Dynamic Partitioning Event Operator + Target Input: srcpart + Partition key expr: ds + Statistics: Num rows: 2000 Data size: 0 Basic stats: PARTIAL Column stats: COMPLETE + Target column: ds + Target Vertex: Map 4 + Map 4 + Map Operator Tree: + TableScan + alias: srcpart + filterExpr: (ds is not null and (ds = '2008-04-08')) (type: boolean) + Statistics: Num rows: 1000 Data size: 10624 Basic stats: COMPLETE Column stats: COMPLETE + Select Operator + expressions: ds (type: string) + outputColumnNames: ds + Statistics: Num rows: 1000 Data size: 10624 Basic stats: COMPLETE Column stats: COMPLETE + Group By Operator + keys: ds (type: string) + mode: hash + outputColumnNames: _col0 + Statistics: Num rows: 1000 Data size: 184000 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + key expressions: _col0 (type: string) + sort order: + + Map-reduce partition columns: _col0 (type: string) + Statistics: Num rows: 1000 Data size: 184000 Basic stats: COMPLETE Column stats: COMPLETE + Reducer 2 + Reduce Operator Tree: + Join Operator + condition map: + Inner Join 0 to 1 + condition expressions: + 0 + 1 + Statistics: Num rows: 500 Data size: 0 Basic stats: PARTIAL Column stats: COMPLETE + Select Operator + Statistics: Num rows: 500 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: COMPLETE Column stats: COMPLETE + Reduce Output Operator + sort order: + Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE + 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: COMPLETE + Select Operator + expressions: _col0 (type: bigint) + outputColumnNames: _col0 + Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE + File Output Operator + compressed: false + Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE + table: + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + Execution mode: vectorized + Reducer 5 + Reduce Operator Tree: + Group By Operator + keys: KEY._col0 (type: string) + mode: mergepartial + outputColumnNames: _col0 + Statistics: Num rows: 500 Data size: 92000 Basic stats: COMPLETE Column stats: COMPLETE + Select Operator + expressions: _col0 (type: string) + outputColumnNames: _col0 + Statistics: Num rows: 500 Data size: 92000 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + key expressions: _col0 (type: string) + sort order: + + Map-reduce partition columns: _col0 (type: string) + Statistics: Num rows: 500 Data size: 92000 Basic stats: COMPLETE Column stats: COMPLETE + Execution mode: vectorized + + Stage: Stage-0 + Fetch Operator + limit: -1 + Processor Tree: + ListSink + +PREHOOK: query: 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' +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 +#### A masked pattern was here #### +POSTHOOK: query: 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' +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 +#### A masked pattern was here #### +1000 +PREHOOK: query: select count(*) from srcpart where ds = '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 +#### A masked pattern was here #### +POSTHOOK: query: select count(*) from srcpart where ds = '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 +#### A masked pattern was here #### +1000 +Warning: Shuffle Join JOIN[4][tables = [srcpart, srcpart_date_hour]] in Stage 'Reducer 2' is a cross product +PREHOOK: query: -- non-equi join +EXPLAIN select count(*) from srcpart, srcpart_date_hour where (srcpart_date_hour.date = '2008-04-08' and srcpart_date_hour.hour = 11) and (srcpart.ds = srcpart_date_hour.ds or srcpart.hr = srcpart_date_hour.hr) +PREHOOK: type: QUERY +POSTHOOK: query: -- non-equi join +EXPLAIN select count(*) from srcpart, srcpart_date_hour where (srcpart_date_hour.date = '2008-04-08' and srcpart_date_hour.hour = 11) and (srcpart.ds = srcpart_date_hour.ds or srcpart.hr = srcpart_date_hour.hr) +POSTHOOK: type: QUERY +STAGE DEPENDENCIES: + Stage-1 is a root stage + Stage-0 depends on stages: Stage-1 + +STAGE PLANS: + Stage: Stage-1 + Tez + Edges: + Reducer 2 <- Map 1 (SIMPLE_EDGE), Map 4 (SIMPLE_EDGE) + Reducer 3 <- Reducer 2 (SIMPLE_EDGE) +#### 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: COMPLETE + Reduce Output Operator + sort order: + Statistics: Num rows: 2000 Data size: 21248 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: ds (type: string), hr (type: string) + Map 4 + Map Operator Tree: + TableScan + alias: srcpart_date_hour + filterExpr: ((date = '2008-04-08') and (hour = 11)) (type: boolean) + Statistics: Num rows: 4 Data size: 1440 Basic stats: COMPLETE Column stats: NONE + Filter Operator + predicate: ((date = '2008-04-08') and (hour = 11)) (type: boolean) + Statistics: Num rows: 1 Data size: 360 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + sort order: + Statistics: Num rows: 1 Data size: 360 Basic stats: COMPLETE Column stats: NONE + value expressions: ds (type: string), hr (type: string) + Reducer 2 + Reduce Operator Tree: + Join Operator + condition map: + Inner Join 0 to 1 + condition expressions: + 0 {VALUE._col2} {VALUE._col3} + 1 {VALUE._col0} {VALUE._col2} + outputColumnNames: _col2, _col3, _col7, _col9 + Statistics: Num rows: 2200 Data size: 23372 Basic stats: COMPLETE Column stats: NONE + Filter Operator + predicate: ((_col2 = _col7) or (_col3 = _col9)) (type: boolean) + Statistics: Num rows: 2200 Data size: 23372 Basic stats: COMPLETE Column stats: NONE + Select Operator + 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 + Select Operator + expressions: _col0 (type: bigint) + 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.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + Execution mode: vectorized + + Stage: Stage-0 + Fetch Operator + limit: -1 + Processor Tree: + ListSink + +Warning: Shuffle Join JOIN[4][tables = [srcpart, srcpart_date_hour]] in Stage 'Reducer 2' is a cross product +PREHOOK: query: select count(*) from srcpart, srcpart_date_hour where (srcpart_date_hour.date = '2008-04-08' and srcpart_date_hour.hour = 11) and (srcpart.ds = srcpart_date_hour.ds or srcpart.hr = srcpart_date_hour.hr) +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, srcpart_date_hour where (srcpart_date_hour.date = '2008-04-08' and srcpart_date_hour.hour = 11) and (srcpart.ds = srcpart_date_hour.ds or srcpart.hr = srcpart_date_hour.hr) +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 #### +1500 +PREHOOK: query: -- old style join syntax +EXPLAIN select count(*) from srcpart, srcpart_date_hour where srcpart_date_hour.date = '2008-04-08' and srcpart_date_hour.hour = 11 and srcpart.ds = srcpart_date_hour.ds and srcpart.hr = srcpart_date_hour.hr +PREHOOK: type: QUERY +POSTHOOK: query: -- old style join syntax +EXPLAIN select count(*) from srcpart, srcpart_date_hour where srcpart_date_hour.date = '2008-04-08' and srcpart_date_hour.hour = 11 and srcpart.ds = srcpart_date_hour.ds and srcpart.hr = srcpart_date_hour.hr +POSTHOOK: type: QUERY +STAGE DEPENDENCIES: + Stage-1 is a root stage + Stage-0 depends on stages: Stage-1 + +STAGE PLANS: + Stage: Stage-1 + Tez + Edges: + Reducer 2 <- Map 1 (SIMPLE_EDGE), Map 4 (SIMPLE_EDGE) + Reducer 3 <- Reducer 2 (SIMPLE_EDGE) +#### 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: COMPLETE + Reduce Output Operator + key expressions: ds (type: string), hr (type: string) + sort order: ++ + Map-reduce partition columns: ds (type: string), hr (type: string) + Statistics: Num rows: 2000 Data size: 21248 Basic stats: COMPLETE Column stats: COMPLETE + Map 4 + Map Operator Tree: + TableScan + alias: srcpart_date_hour + filterExpr: (((ds is not null and hr is not null) and (date = '2008-04-08')) and (hour = 11)) (type: boolean) + Statistics: Num rows: 4 Data size: 1440 Basic stats: COMPLETE Column stats: NONE + Filter Operator + predicate: (((ds is not null and hr is not null) and (date = '2008-04-08')) and (hour = 11)) (type: boolean) + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Reduce Output Operator + key expressions: ds (type: string), hr (type: string) + sort order: ++ + Map-reduce partition columns: ds (type: string), hr (type: string) + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Select Operator + expressions: ds (type: string) + outputColumnNames: _col0 + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Group By Operator + keys: _col0 (type: string) + mode: hash + outputColumnNames: _col0 + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Dynamic Partitioning Event Operator + Target Input: srcpart + Partition key expr: ds + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Target column: ds + Target Vertex: Map 1 + Select Operator + expressions: hr (type: string) + outputColumnNames: _col0 + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Group By Operator + keys: _col0 (type: string) + mode: hash + outputColumnNames: _col0 + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Dynamic Partitioning Event Operator + Target Input: srcpart + Partition key expr: hr + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Target column: hr + Target Vertex: Map 1 + Reducer 2 + Reduce Operator Tree: + Join Operator + condition map: + Inner Join 0 to 1 + condition expressions: + 0 {KEY.reducesinkkey0} {KEY.reducesinkkey1} + 1 {KEY.reducesinkkey0} {KEY.reducesinkkey1} + outputColumnNames: _col2, _col3, _col7, _col9 + Statistics: Num rows: 2200 Data size: 23372 Basic stats: COMPLETE Column stats: NONE + Filter Operator + predicate: ((_col2 = _col7) and (_col3 = _col9)) (type: boolean) + Statistics: Num rows: 550 Data size: 5843 Basic stats: COMPLETE Column stats: NONE + Select Operator + Statistics: Num rows: 550 Data size: 5843 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 + Select Operator + expressions: _col0 (type: bigint) + 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.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + Execution mode: vectorized + + Stage: Stage-0 + Fetch Operator + limit: -1 + Processor Tree: + ListSink + +PREHOOK: query: select count(*) from srcpart, srcpart_date_hour where srcpart_date_hour.date = '2008-04-08' and srcpart_date_hour.hour = 11 and srcpart.ds = srcpart_date_hour.ds and srcpart.hr = srcpart_date_hour.hr +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, srcpart_date_hour where srcpart_date_hour.date = '2008-04-08' and srcpart_date_hour.hour = 11 and srcpart.ds = srcpart_date_hour.ds and srcpart.hr = srcpart_date_hour.hr +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: -- left join +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: -- left join +EXPLAIN select count(*) from srcpart left join srcpart_date on (srcpart.ds = 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 + Tez + Edges: + Reducer 2 <- Map 1 (SIMPLE_EDGE), Map 4 (SIMPLE_EDGE) + Reducer 3 <- Reducer 2 (SIMPLE_EDGE) +#### 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: COMPLETE + Reduce Output Operator + key expressions: ds (type: string) + sort order: + + Map-reduce partition columns: ds (type: string) + Statistics: Num rows: 2000 Data size: 21248 Basic stats: COMPLETE Column stats: COMPLETE + Map 4 + Map Operator Tree: + TableScan + alias: srcpart_date + Statistics: Num rows: 2 Data size: 376 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + key expressions: ds (type: string) + sort order: + + Map-reduce partition columns: ds (type: string) + Statistics: Num rows: 2 Data size: 376 Basic stats: COMPLETE Column stats: NONE + value expressions: date (type: string) + Execution mode: vectorized + Reducer 2 + Reduce Operator Tree: + Join Operator + condition map: + Left Outer Join0 to 1 + condition expressions: + 0 + 1 {VALUE._col0} + outputColumnNames: _col8 + Statistics: Num rows: 2200 Data size: 23372 Basic stats: COMPLETE Column stats: NONE + Filter Operator + predicate: (_col8 = '2008-04-08') (type: boolean) + Statistics: Num rows: 1100 Data size: 11686 Basic stats: COMPLETE Column stats: NONE + Select Operator + Statistics: Num rows: 1100 Data size: 11686 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 + Select Operator + expressions: _col0 (type: bigint) + 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.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + Execution mode: vectorized + + Stage: Stage-0 + Fetch Operator + limit: -1 + Processor Tree: + ListSink + +PREHOOK: query: EXPLAIN select count(*) from srcpart_date left join srcpart on (srcpart.ds = srcpart_date.ds) where srcpart_date.date = '2008-04-08' +PREHOOK: type: QUERY +POSTHOOK: query: EXPLAIN select count(*) from srcpart_date left join srcpart on (srcpart.ds = 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 + Tez + Edges: + Reducer 2 <- Map 1 (SIMPLE_EDGE), Map 4 (SIMPLE_EDGE) + Reducer 3 <- Reducer 2 (SIMPLE_EDGE) +#### 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: COMPLETE + Reduce Output Operator + key expressions: ds (type: string) + sort order: + + Map-reduce partition columns: ds (type: string) + Statistics: Num rows: 2000 Data size: 21248 Basic stats: COMPLETE Column stats: COMPLETE + Map 4 + Map Operator Tree: + TableScan + alias: srcpart_date + filterExpr: (date = '2008-04-08') (type: boolean) + Statistics: Num rows: 2 Data size: 376 Basic stats: COMPLETE Column stats: NONE + Filter Operator + predicate: (date = '2008-04-08') (type: boolean) + Statistics: Num rows: 1 Data size: 188 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + key expressions: ds (type: string) + sort order: + + Map-reduce partition columns: ds (type: string) + Statistics: Num rows: 1 Data size: 188 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: ds (type: string) + outputColumnNames: _col0 + Statistics: Num rows: 1 Data size: 188 Basic stats: COMPLETE Column stats: NONE + Group By Operator + keys: _col0 (type: string) + mode: hash + outputColumnNames: _col0 + Statistics: Num rows: 1 Data size: 188 Basic stats: COMPLETE Column stats: NONE + Dynamic Partitioning Event Operator + Target Input: srcpart + Partition key expr: ds + Statistics: Num rows: 1 Data size: 188 Basic stats: COMPLETE Column stats: NONE + Target column: ds + Target Vertex: Map 1 + Execution mode: vectorized + Reducer 2 + Reduce Operator Tree: + Join Operator + condition map: + Left Outer Join0 to 1 + condition expressions: + 0 + 1 + Statistics: Num rows: 2200 Data size: 23372 Basic stats: COMPLETE Column stats: NONE + Select Operator + 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 + Select Operator + expressions: _col0 (type: bigint) + 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.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + Execution mode: vectorized + + Stage: Stage-0 + Fetch Operator + limit: -1 + Processor Tree: + ListSink + +PREHOOK: query: -- full outer +EXPLAIN select count(*) from srcpart full outer join srcpart_date on (srcpart.ds = srcpart_date.ds) where srcpart_date.date = '2008-04-08' +PREHOOK: type: QUERY +POSTHOOK: query: -- full outer +EXPLAIN select count(*) from srcpart full outer join srcpart_date on (srcpart.ds = 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 + Tez + Edges: + Reducer 2 <- Map 1 (SIMPLE_EDGE), Map 4 (SIMPLE_EDGE) + Reducer 3 <- Reducer 2 (SIMPLE_EDGE) +#### 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: COMPLETE + Reduce Output Operator + key expressions: ds (type: string) + sort order: + + Map-reduce partition columns: ds (type: string) + Statistics: Num rows: 2000 Data size: 21248 Basic stats: COMPLETE Column stats: COMPLETE + Map 4 + Map Operator Tree: + TableScan + alias: srcpart_date + Statistics: Num rows: 2 Data size: 376 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + key expressions: ds (type: string) + sort order: + + Map-reduce partition columns: ds (type: string) + Statistics: Num rows: 2 Data size: 376 Basic stats: COMPLETE Column stats: NONE + value expressions: date (type: string) + Execution mode: vectorized + Reducer 2 + Reduce Operator Tree: + Join Operator + condition map: + Outer Join 0 to 1 + condition expressions: + 0 + 1 {VALUE._col0} + outputColumnNames: _col8 + Statistics: Num rows: 2200 Data size: 23372 Basic stats: COMPLETE Column stats: NONE + Filter Operator + predicate: (_col8 = '2008-04-08') (type: boolean) + Statistics: Num rows: 1100 Data size: 11686 Basic stats: COMPLETE Column stats: NONE + Select Operator + Statistics: Num rows: 1100 Data size: 11686 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 + Select Operator + expressions: _col0 (type: bigint) + 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.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + Execution mode: vectorized + + Stage: Stage-0 + Fetch Operator + limit: -1 + Processor Tree: + ListSink + +PREHOOK: query: -- with static pruning +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 and srcpart.hr = 11 +PREHOOK: type: QUERY +POSTHOOK: query: -- with static pruning +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 and srcpart.hr = 11 +POSTHOOK: type: QUERY +STAGE DEPENDENCIES: + Stage-1 is a root stage + Stage-0 depends on stages: Stage-1 + +STAGE PLANS: + Stage: Stage-1 + Tez + Edges: + Reducer 2 <- Map 1 (SIMPLE_EDGE), Map 6 (SIMPLE_EDGE) + Reducer 3 <- Map 5 (SIMPLE_EDGE), Reducer 2 (SIMPLE_EDGE) + Reducer 4 <- Reducer 3 (SIMPLE_EDGE) +#### A masked pattern was here #### + Vertices: + Map 1 + Map Operator Tree: + TableScan + alias: srcpart + filterExpr: (hr = 11) (type: boolean) + Statistics: Num rows: 1000 Data size: 10624 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + key expressions: ds (type: string) + sort order: + + Map-reduce partition columns: ds (type: string) + Statistics: Num rows: 1000 Data size: 10624 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: hr (type: string) + Map 5 + Map Operator Tree: + TableScan + alias: srcpart_hour + filterExpr: ((hr is not null and (hour = 11)) and (hr = 11)) (type: boolean) + Statistics: Num rows: 2 Data size: 344 Basic stats: COMPLETE Column stats: NONE + Filter Operator + predicate: ((hr is not null and (hour = 11)) and (hr = 11)) (type: boolean) + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Reduce Output Operator + key expressions: '11' (type: string) + sort order: + + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Select Operator + expressions: '11' (type: string) + outputColumnNames: _col0 + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Group By Operator + keys: _col0 (type: string) + mode: hash + outputColumnNames: _col0 + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Dynamic Partitioning Event Operator + Target Input: srcpart + Partition key expr: hr + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Target column: hr + Target Vertex: Map 1 + Map 6 + Map Operator Tree: + TableScan + alias: srcpart_date + filterExpr: (ds is not null and (date = '2008-04-08')) (type: boolean) + Statistics: Num rows: 2 Data size: 376 Basic stats: COMPLETE Column stats: NONE + Filter Operator + predicate: (ds is not null and (date = '2008-04-08')) (type: boolean) + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Reduce Output Operator + key expressions: ds (type: string) + sort order: + + Map-reduce partition columns: ds (type: string) + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Select Operator + expressions: ds (type: string) + outputColumnNames: _col0 + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Group By Operator + keys: _col0 (type: string) + mode: hash + outputColumnNames: _col0 + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Dynamic Partitioning Event Operator + Target Input: srcpart + Partition key expr: ds + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Target column: ds + Target Vertex: Map 1 + Execution mode: vectorized + Reducer 2 + Reduce Operator Tree: + Join Operator + condition map: + Inner Join 0 to 1 + condition expressions: + 0 {VALUE._col2} + 1 + outputColumnNames: _col3 + Statistics: Num rows: 1100 Data size: 11686 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + key expressions: _col3 (type: string) + sort order: + + Map-reduce partition columns: _col3 (type: string) + Statistics: Num rows: 1100 Data size: 11686 Basic stats: COMPLETE Column stats: NONE + Reducer 3 + Reduce Operator Tree: + Join Operator + condition map: + Inner Join 0 to 1 + condition expressions: + 0 + 1 + Statistics: Num rows: 1210 Data size: 12854 Basic stats: COMPLETE Column stats: NONE + Select Operator + Statistics: Num rows: 1210 Data size: 12854 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 4 + 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 + Select Operator + expressions: _col0 (type: bigint) + 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.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + Execution mode: vectorized + + Stage: Stage-0 + Fetch Operator + limit: -1 + Processor Tree: + ListSink + +PREHOOK: query: 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 and srcpart.hr = 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-09/hr=11 +PREHOOK: Input: default@srcpart_date +PREHOOK: Input: default@srcpart_hour +#### A masked pattern was here #### +POSTHOOK: query: 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 and srcpart.hr = 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-09/hr=11 +POSTHOOK: Input: default@srcpart_date +POSTHOOK: Input: default@srcpart_hour +#### A masked pattern was here #### +500 +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.hr = 13 +PREHOOK: type: QUERY +POSTHOOK: 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.hr = 13 +POSTHOOK: type: QUERY +STAGE DEPENDENCIES: + Stage-1 is a root stage + Stage-0 depends on stages: Stage-1 + +STAGE PLANS: + Stage: Stage-1 + Tez + Edges: + Reducer 2 <- Map 1 (SIMPLE_EDGE), Map 6 (SIMPLE_EDGE) + Reducer 3 <- Map 5 (SIMPLE_EDGE), Reducer 2 (SIMPLE_EDGE) + Reducer 4 <- Reducer 3 (SIMPLE_EDGE) +#### A masked pattern was here #### + Vertices: + Map 1 + Map 5 + Map Operator Tree: + TableScan + alias: srcpart_hour + filterExpr: (hr is not null and (hr = 13)) (type: boolean) + Statistics: Num rows: 2 Data size: 344 Basic stats: COMPLETE Column stats: NONE + Filter Operator + predicate: (hr is not null and (hr = 13)) (type: boolean) + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Reduce Output Operator + key expressions: '13' (type: string) + sort order: + + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Map 6 + Map Operator Tree: + TableScan + alias: srcpart_date + filterExpr: (ds is not null and (date = '2008-04-08')) (type: boolean) + Statistics: Num rows: 2 Data size: 376 Basic stats: COMPLETE Column stats: NONE + Filter Operator + predicate: (ds is not null and (date = '2008-04-08')) (type: boolean) + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Reduce Output Operator + key expressions: ds (type: string) + sort order: + + Map-reduce partition columns: ds (type: string) + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Execution mode: vectorized + Reducer 2 + Reduce Operator Tree: + Join Operator + condition map: + Inner Join 0 to 1 + condition expressions: + 0 + 1 + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Reduce Output Operator + key expressions: '13' (type: string) + sort order: + + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Reducer 3 + Reduce Operator Tree: + Join Operator + condition map: + Inner Join 0 to 1 + condition expressions: + 0 + 1 + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Select Operator + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE 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 4 + 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 + Select Operator + expressions: _col0 (type: bigint) + 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.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + Execution mode: vectorized + + Stage: Stage-0 + Fetch Operator + limit: -1 + Processor Tree: + ListSink + +PREHOOK: query: 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.hr = 13 +PREHOOK: type: QUERY +PREHOOK: Input: default@srcpart +PREHOOK: Input: default@srcpart_date +PREHOOK: Input: default@srcpart_hour +#### A masked pattern was here #### +POSTHOOK: query: 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.hr = 13 +POSTHOOK: type: QUERY +POSTHOOK: Input: default@srcpart +POSTHOOK: Input: default@srcpart_date +POSTHOOK: Input: default@srcpart_hour +#### A masked pattern was here #### +0 +PREHOOK: query: -- union + subquery +EXPLAIN select count(*) from srcpart where srcpart.ds in (select max(srcpart.ds) from srcpart union all select min(srcpart.ds) from srcpart) +PREHOOK: type: QUERY +POSTHOOK: query: -- union + subquery +EXPLAIN select count(*) from srcpart where srcpart.ds in (select max(srcpart.ds) from srcpart union all select min(srcpart.ds) from srcpart) +POSTHOOK: type: QUERY +STAGE DEPENDENCIES: + Stage-1 is a root stage + Stage-0 depends on stages: Stage-1 + +STAGE PLANS: + Stage: Stage-1 + Tez + Edges: + Reducer 2 <- Map 1 (SIMPLE_EDGE), Union 3 (CONTAINS) + Reducer 4 <- Map 6 (SIMPLE_EDGE), Union 3 (SIMPLE_EDGE) + Reducer 5 <- Reducer 4 (SIMPLE_EDGE) + Reducer 8 <- Map 7 (SIMPLE_EDGE), Union 3 (CONTAINS) +#### 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: COMPLETE + Select Operator + expressions: ds (type: string) + outputColumnNames: ds + Statistics: Num rows: 2000 Data size: 21248 Basic stats: COMPLETE Column stats: COMPLETE + Group By Operator + aggregations: min(ds) + mode: hash + outputColumnNames: _col0 + Statistics: Num rows: 1 Data size: 84 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + sort order: + Statistics: Num rows: 1 Data size: 84 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: _col0 (type: string) + Map 6 + 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: COMPLETE + Reduce Output Operator + key expressions: ds (type: string) + sort order: + + Map-reduce partition columns: ds (type: string) + Statistics: Num rows: 2000 Data size: 21248 Basic stats: COMPLETE Column stats: COMPLETE + Map 7 + Map Operator Tree: + TableScan + alias: srcpart + Statistics: Num rows: 2000 Data size: 21248 Basic stats: COMPLETE Column stats: COMPLETE + Select Operator + expressions: ds (type: string) + outputColumnNames: ds + Statistics: Num rows: 2000 Data size: 21248 Basic stats: COMPLETE Column stats: COMPLETE + Group By Operator + aggregations: max(ds) + mode: hash + outputColumnNames: _col0 + Statistics: Num rows: 1 Data size: 84 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + sort order: + Statistics: Num rows: 1 Data size: 84 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: _col0 (type: string) + Reducer 2 + Reduce Operator Tree: + Group By Operator + aggregations: min(VALUE._col0) + mode: mergepartial + outputColumnNames: _col0 + Filter Operator + predicate: _col0 is not null (type: boolean) + Select Operator + expressions: _col0 (type: string) + outputColumnNames: _col0 + Select Operator + expressions: _col0 (type: string) + outputColumnNames: _col0 + Group By Operator + keys: _col0 (type: string) + mode: hash + outputColumnNames: _col0 + Reduce Output Operator + key expressions: _col0 (type: string) + sort order: + + Map-reduce partition columns: _col0 (type: string) + Select Operator + expressions: _col0 (type: string) + outputColumnNames: _col0 + Group By Operator + keys: _col0 (type: string) + mode: hash + outputColumnNames: _col0 + Dynamic Partitioning Event Operator + Target Input: srcpart + Partition key expr: ds + Target column: ds + Target Vertex: Map 6 + Reducer 4 + Reduce Operator Tree: + Join Operator + condition map: + Left Semi Join 0 to 1 + condition expressions: + 0 + 1 + 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: COMPLETE Column stats: COMPLETE + Reduce Output Operator + sort order: + Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: _col0 (type: bigint) + Reducer 5 + 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: COMPLETE + Select Operator + expressions: _col0 (type: bigint) + outputColumnNames: _col0 + Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE + File Output Operator + compressed: false + Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE + table: + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + Execution mode: vectorized + Reducer 8 + Reduce Operator Tree: + Group By Operator + aggregations: max(VALUE._col0) + mode: mergepartial + outputColumnNames: _col0 + Filter Operator + predicate: _col0 is not null (type: boolean) + Select Operator + expressions: _col0 (type: string) + outputColumnNames: _col0 + Select Operator + expressions: _col0 (type: string) + outputColumnNames: _col0 + Group By Operator + keys: _col0 (type: string) + mode: hash + outputColumnNames: _col0 + Reduce Output Operator + key expressions: _col0 (type: string) + sort order: + + Map-reduce partition columns: _col0 (type: string) + Select Operator + expressions: _col0 (type: string) + outputColumnNames: _col0 + Group By Operator + keys: _col0 (type: string) + mode: hash + outputColumnNames: _col0 + Dynamic Partitioning Event Operator + Target Input: srcpart + Partition key expr: ds + Target column: ds + Target Vertex: Map 6 + Union 3 + Vertex: Union 3 + + Stage: Stage-0 + Fetch Operator + limit: -1 + Processor Tree: + ListSink + +PREHOOK: query: select count(*) from srcpart where srcpart.ds in (select max(srcpart.ds) from srcpart union all select min(srcpart.ds) from srcpart) +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 +#### A masked pattern was here #### +POSTHOOK: query: select count(*) from srcpart where srcpart.ds in (select max(srcpart.ds) from srcpart union all select min(srcpart.ds) from srcpart) +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 +#### A masked pattern was here #### +2000 +PREHOOK: query: EXPLAIN select distinct(ds) from srcpart where srcpart.ds in (select max(srcpart.ds) from srcpart union all select min(srcpart.ds) from srcpart) +PREHOOK: type: QUERY +POSTHOOK: query: EXPLAIN select distinct(ds) from srcpart where srcpart.ds in (select max(srcpart.ds) from srcpart union all select min(srcpart.ds) from srcpart) +POSTHOOK: type: QUERY +STAGE DEPENDENCIES: + Stage-1 is a root stage + Stage-0 depends on stages: Stage-1 + +STAGE PLANS: + Stage: Stage-1 + Tez + Edges: + Reducer 2 <- Map 1 (SIMPLE_EDGE), Union 3 (CONTAINS) + Reducer 4 <- Map 6 (SIMPLE_EDGE), Union 3 (SIMPLE_EDGE) + Reducer 5 <- Reducer 4 (SIMPLE_EDGE) + Reducer 8 <- Map 7 (SIMPLE_EDGE), Union 3 (CONTAINS) +#### 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: COMPLETE + Select Operator + expressions: ds (type: string) + outputColumnNames: ds + Statistics: Num rows: 2000 Data size: 21248 Basic stats: COMPLETE Column stats: COMPLETE + Group By Operator + aggregations: min(ds) + mode: hash + outputColumnNames: _col0 + Statistics: Num rows: 1 Data size: 84 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + sort order: + Statistics: Num rows: 1 Data size: 84 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: _col0 (type: string) + Map 6 + 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: COMPLETE + Reduce Output Operator + key expressions: ds (type: string) + sort order: + + Map-reduce partition columns: ds (type: string) + Statistics: Num rows: 2000 Data size: 21248 Basic stats: COMPLETE Column stats: COMPLETE + Map 7 + Map Operator Tree: + TableScan + alias: srcpart + Statistics: Num rows: 2000 Data size: 21248 Basic stats: COMPLETE Column stats: COMPLETE + Select Operator + expressions: ds (type: string) + outputColumnNames: ds + Statistics: Num rows: 2000 Data size: 21248 Basic stats: COMPLETE Column stats: COMPLETE + Group By Operator + aggregations: max(ds) + mode: hash + outputColumnNames: _col0 + Statistics: Num rows: 1 Data size: 84 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + sort order: + Statistics: Num rows: 1 Data size: 84 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: _col0 (type: string) + Reducer 2 + Reduce Operator Tree: + Group By Operator + aggregations: min(VALUE._col0) + mode: mergepartial + outputColumnNames: _col0 + Filter Operator + predicate: _col0 is not null (type: boolean) + Select Operator + expressions: _col0 (type: string) + outputColumnNames: _col0 + Select Operator + expressions: _col0 (type: string) + outputColumnNames: _col0 + Group By Operator + keys: _col0 (type: string) + mode: hash + outputColumnNames: _col0 + Reduce Output Operator + key expressions: _col0 (type: string) + sort order: + + Map-reduce partition columns: _col0 (type: string) + Select Operator + expressions: _col0 (type: string) + outputColumnNames: _col0 + Group By Operator + keys: _col0 (type: string) + mode: hash + outputColumnNames: _col0 + Dynamic Partitioning Event Operator + Target Input: srcpart + Partition key expr: ds + Target column: ds + Target Vertex: Map 6 + Reducer 4 + Reduce Operator Tree: + Join Operator + condition map: + Left Semi Join 0 to 1 + condition expressions: + 0 {KEY.reducesinkkey0} + 1 + outputColumnNames: _col2 + Statistics: Num rows: 2 Data size: 368 Basic stats: COMPLETE Column stats: COMPLETE + Select Operator + expressions: _col2 (type: string) + outputColumnNames: _col2 + Statistics: Num rows: 2 Data size: 368 Basic stats: COMPLETE Column stats: COMPLETE + Group By Operator + keys: _col2 (type: string) + mode: hash + outputColumnNames: _col0 + Statistics: Num rows: 2 Data size: 368 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + key expressions: _col0 (type: string) + sort order: + + Map-reduce partition columns: _col0 (type: string) + Statistics: Num rows: 2 Data size: 368 Basic stats: COMPLETE Column stats: COMPLETE + Reducer 5 + Reduce Operator Tree: + Group By Operator + keys: KEY._col0 (type: string) + mode: mergepartial + outputColumnNames: _col0 + Statistics: Num rows: 1 Data size: 184 Basic stats: COMPLETE Column stats: COMPLETE + Select Operator + expressions: _col0 (type: string) + outputColumnNames: _col0 + Statistics: Num rows: 1 Data size: 184 Basic stats: COMPLETE Column stats: COMPLETE + File Output Operator + compressed: false + Statistics: Num rows: 1 Data size: 184 Basic stats: COMPLETE Column stats: COMPLETE + table: + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + Execution mode: vectorized + Reducer 8 + Reduce Operator Tree: + Group By Operator + aggregations: max(VALUE._col0) + mode: mergepartial + outputColumnNames: _col0 + Filter Operator + predicate: _col0 is not null (type: boolean) + Select Operator + expressions: _col0 (type: string) + outputColumnNames: _col0 + Select Operator + expressions: _col0 (type: string) + outputColumnNames: _col0 + Group By Operator + keys: _col0 (type: string) + mode: hash + outputColumnNames: _col0 + Reduce Output Operator + key expressions: _col0 (type: string) + sort order: + + Map-reduce partition columns: _col0 (type: string) + Select Operator + expressions: _col0 (type: string) + outputColumnNames: _col0 + Group By Operator + keys: _col0 (type: string) + mode: hash + outputColumnNames: _col0 + Dynamic Partitioning Event Operator + Target Input: srcpart + Partition key expr: ds + Target column: ds + Target Vertex: Map 6 + Union 3 + Vertex: Union 3 + + Stage: Stage-0 + Fetch Operator + limit: -1 + Processor Tree: + ListSink + +PREHOOK: query: select distinct(ds) from srcpart where srcpart.ds in (select max(srcpart.ds) from srcpart union all select min(srcpart.ds) from srcpart) +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 +#### A masked pattern was here #### +POSTHOOK: query: select distinct(ds) from srcpart where srcpart.ds in (select max(srcpart.ds) from srcpart union all select min(srcpart.ds) from srcpart) +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 +#### A masked pattern was here #### +2008-04-08 +2008-04-09 +PREHOOK: query: EXPLAIN select ds from (select distinct(ds) as ds from srcpart union all select distinct(ds) as ds from srcpart) s where s.ds in (select max(srcpart.ds) from srcpart union all select min(srcpart.ds) from srcpart) +PREHOOK: type: QUERY +POSTHOOK: query: EXPLAIN select ds from (select distinct(ds) as ds from srcpart union all select distinct(ds) as ds from srcpart) s where s.ds in (select max(srcpart.ds) from srcpart union all select min(srcpart.ds) from srcpart) +POSTHOOK: type: QUERY +STAGE DEPENDENCIES: + Stage-1 is a root stage + Stage-0 depends on stages: Stage-1 + +STAGE PLANS: + Stage: Stage-1 + Tez + Edges: + Reducer 11 <- Map 10 (SIMPLE_EDGE), Union 3 (CONTAINS) + Reducer 2 <- Map 1 (SIMPLE_EDGE), Union 3 (CONTAINS) + Reducer 4 <- Union 3 (SIMPLE_EDGE), Union 7 (SIMPLE_EDGE) + Reducer 6 <- Map 5 (SIMPLE_EDGE), Union 7 (CONTAINS) + Reducer 9 <- Map 8 (SIMPLE_EDGE), Union 7 (CONTAINS) +#### 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: COMPLETE + Select Operator + expressions: ds (type: string) + outputColumnNames: ds + Statistics: Num rows: 2000 Data size: 21248 Basic stats: COMPLETE Column stats: COMPLETE + Group By Operator + aggregations: min(ds) + mode: hash + outputColumnNames: _col0 + Statistics: Num rows: 1 Data size: 84 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + sort order: + Statistics: Num rows: 1 Data size: 84 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: _col0 (type: string) + Map 10 + Map Operator Tree: + TableScan + alias: srcpart + Statistics: Num rows: 2000 Data size: 21248 Basic stats: COMPLETE Column stats: COMPLETE + Select Operator + expressions: ds (type: string) + outputColumnNames: ds + Statistics: Num rows: 2000 Data size: 21248 Basic stats: COMPLETE Column stats: COMPLETE + Group By Operator + aggregations: max(ds) + mode: hash + outputColumnNames: _col0 + Statistics: Num rows: 1 Data size: 84 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + sort order: + Statistics: Num rows: 1 Data size: 84 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: _col0 (type: string) + Map 5 + 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: COMPLETE + Select Operator + expressions: ds (type: string) + outputColumnNames: ds + Statistics: Num rows: 2000 Data size: 21248 Basic stats: COMPLETE Column stats: COMPLETE + Group By Operator + keys: ds (type: string) + mode: hash + outputColumnNames: _col0 + Statistics: Num rows: 2000 Data size: 368000 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + key expressions: _col0 (type: string) + sort order: + + Map-reduce partition columns: _col0 (type: string) + Statistics: Num rows: 2000 Data size: 368000 Basic stats: COMPLETE Column stats: COMPLETE + Map 8 + 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: COMPLETE + Select Operator + expressions: ds (type: string) + outputColumnNames: ds + Statistics: Num rows: 2000 Data size: 21248 Basic stats: COMPLETE Column stats: COMPLETE + Group By Operator + keys: ds (type: string) + mode: hash + outputColumnNames: _col0 + Statistics: Num rows: 2000 Data size: 368000 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + key expressions: _col0 (type: string) + sort order: + + Map-reduce partition columns: _col0 (type: string) + Statistics: Num rows: 2000 Data size: 368000 Basic stats: COMPLETE Column stats: COMPLETE + Reducer 11 + Reduce Operator Tree: + Group By Operator + aggregations: max(VALUE._col0) + mode: mergepartial + outputColumnNames: _col0 + Filter Operator + predicate: _col0 is not null (type: boolean) + Select Operator + expressions: _col0 (type: string) + outputColumnNames: _col0 + Select Operator + expressions: _col0 (type: string) + outputColumnNames: _col0 + Group By Operator + keys: _col0 (type: string) + mode: hash + outputColumnNames: _col0 + Reduce Output Operator + key expressions: _col0 (type: string) + sort order: + + Map-reduce partition columns: _col0 (type: string) + Select Operator + expressions: _col0 (type: string) + outputColumnNames: _col0 + Group By Operator + keys: _col0 (type: string) + mode: hash + outputColumnNames: _col0 + Dynamic Partitioning Event Operator + Target Input: srcpart + Partition key expr: ds + Target column: ds + Target Vertex: Map 5 + Select Operator + expressions: _col0 (type: string) + outputColumnNames: _col0 + Group By Operator + keys: _col0 (type: string) + mode: hash + outputColumnNames: _col0 + Dynamic Partitioning Event Operator + Target Input: srcpart + Partition key expr: ds + Target column: ds + Target Vertex: Map 8 + Reducer 2 + Reduce Operator Tree: + Group By Operator + aggregations: min(VALUE._col0) + mode: mergepartial + outputColumnNames: _col0 + Filter Operator + predicate: _col0 is not null (type: boolean) + Select Operator + expressions: _col0 (type: string) + outputColumnNames: _col0 + Select Operator + expressions: _col0 (type: string) + outputColumnNames: _col0 + Group By Operator + keys: _col0 (type: string) + mode: hash + outputColumnNames: _col0 + Reduce Output Operator + key expressions: _col0 (type: string) + sort order: + + Map-reduce partition columns: _col0 (type: string) + Select Operator + expressions: _col0 (type: string) + outputColumnNames: _col0 + Group By Operator + keys: _col0 (type: string) + mode: hash + outputColumnNames: _col0 + Dynamic Partitioning Event Operator + Target Input: srcpart + Partition key expr: ds + Target column: ds + Target Vertex: Map 5 + Select Operator + expressions: _col0 (type: string) + outputColumnNames: _col0 + Group By Operator + keys: _col0 (type: string) + mode: hash + outputColumnNames: _col0 + Dynamic Partitioning Event Operator + Target Input: srcpart + Partition key expr: ds + Target column: ds + Target Vertex: Map 8 + Reducer 4 + Reduce Operator Tree: + Join Operator + condition map: + Left Semi Join 0 to 1 + condition expressions: + 0 {KEY.reducesinkkey0} + 1 + outputColumnNames: _col0 + Statistics: Num rows: 4000 Data size: 336000 Basic stats: COMPLETE Column stats: COMPLETE + Select Operator + expressions: _col0 (type: string) + outputColumnNames: _col0 + Statistics: Num rows: 4000 Data size: 336000 Basic stats: COMPLETE Column stats: COMPLETE + File Output Operator + compressed: false + Statistics: Num rows: 4000 Data size: 336000 Basic stats: COMPLETE Column stats: COMPLETE + table: + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + Reducer 6 + Reduce Operator Tree: + Group By Operator + keys: KEY._col0 (type: string) + mode: mergepartial + outputColumnNames: _col0 + Select Operator + expressions: _col0 (type: string) + outputColumnNames: _col0 + Reduce Output Operator + key expressions: _col0 (type: string) + sort order: + + Map-reduce partition columns: _col0 (type: string) + Execution mode: vectorized + Reducer 9 + Reduce Operator Tree: + Group By Operator + keys: KEY._col0 (type: string) + mode: mergepartial + outputColumnNames: _col0 + Select Operator + expressions: _col0 (type: string) + outputColumnNames: _col0 + Reduce Output Operator + key expressions: _col0 (type: string) + sort order: + + Map-reduce partition columns: _col0 (type: string) + Execution mode: vectorized + Union 3 + Vertex: Union 3 + Union 7 + Vertex: Union 7 + + Stage: Stage-0 + Fetch Operator + limit: -1 + Processor Tree: + ListSink + +PREHOOK: query: select ds from (select distinct(ds) as ds from srcpart union all select distinct(ds) as ds from srcpart) s where s.ds in (select max(srcpart.ds) from srcpart union all select min(srcpart.ds) from srcpart) +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 +#### A masked pattern was here #### +POSTHOOK: query: select ds from (select distinct(ds) as ds from srcpart union all select distinct(ds) as ds from srcpart) s where s.ds in (select max(srcpart.ds) from srcpart union all select min(srcpart.ds) from srcpart) +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 +#### A masked pattern was here #### +2008-04-08 +2008-04-08 +2008-04-09 +2008-04-09 +PREHOOK: query: -- single column, single key +EXPLAIN select count(*) from srcpart join srcpart_date on (srcpart.ds = srcpart_date.ds) where srcpart_date.date = '2008-04-08' +PREHOOK: type: QUERY +POSTHOOK: query: -- single column, single key +EXPLAIN select count(*) from srcpart join srcpart_date on (srcpart.ds = 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 + Tez + Edges: + Map 1 <- Map 3 (BROADCAST_EDGE) + Reducer 2 <- Map 1 (SIMPLE_EDGE) +#### 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: COMPLETE + Map Join Operator + condition map: + Inner Join 0 to 1 + condition expressions: + 0 + 1 + keys: + 0 ds (type: string) + 1 ds (type: string) + Statistics: Num rows: 2200 Data size: 23372 Basic stats: COMPLETE Column stats: NONE + Select Operator + 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) + Map 3 + Map Operator Tree: + TableScan + alias: srcpart_date + filterExpr: (ds is not null and (date = '2008-04-08')) (type: boolean) + Statistics: Num rows: 2 Data size: 376 Basic stats: COMPLETE Column stats: NONE + Filter Operator + predicate: (ds is not null and (date = '2008-04-08')) (type: boolean) + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Reduce Output Operator + key expressions: ds (type: string) + sort order: + + Map-reduce partition columns: ds (type: string) + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Select Operator + expressions: ds (type: string) + outputColumnNames: _col0 + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Group By Operator + keys: _col0 (type: string) + mode: hash + outputColumnNames: _col0 + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Dynamic Partitioning Event Operator + Target Input: srcpart + Partition key expr: ds + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Target column: ds + Target Vertex: Map 1 + Execution mode: vectorized + 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 + Select Operator + expressions: _col0 (type: bigint) + 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.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + Execution mode: vectorized + + Stage: Stage-0 + Fetch Operator + limit: -1 + Processor Tree: + ListSink + +PREHOOK: query: select count(*) from srcpart join srcpart_date on (srcpart.ds = 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 (srcpart.ds = 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: select count(*) from srcpart where ds = '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 +#### A masked pattern was here #### +POSTHOOK: query: select count(*) from srcpart where ds = '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 +#### A masked pattern was here #### +1000 +PREHOOK: query: -- 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 +PREHOOK: type: QUERY +POSTHOOK: query: -- 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 +POSTHOOK: type: QUERY +STAGE DEPENDENCIES: + Stage-1 is a root stage + Stage-0 depends on stages: Stage-1 + +STAGE PLANS: + Stage: Stage-1 + Tez + Edges: + Map 1 <- Map 3 (BROADCAST_EDGE), Map 4 (BROADCAST_EDGE) + Reducer 2 <- Map 1 (SIMPLE_EDGE) +#### 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: COMPLETE + Map Join Operator + condition map: + Inner Join 0 to 1 + condition expressions: + 0 {hr} + 1 + keys: + 0 ds (type: string) + 1 ds (type: string) + outputColumnNames: _col3 + Statistics: Num rows: 2200 Data size: 23372 Basic stats: COMPLETE Column stats: NONE + Map Join Operator + condition map: + Inner Join 0 to 1 + condition expressions: + 0 + 1 + keys: + 0 _col3 (type: string) + 1 hr (type: string) + Statistics: Num rows: 2420 Data size: 25709 Basic stats: COMPLETE Column stats: NONE + Select Operator + Statistics: Num rows: 2420 Data size: 25709 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) + Map 3 + Map Operator Tree: + TableScan + alias: srcpart_hour + filterExpr: (hr is not null and (hour = 11)) (type: boolean) + Statistics: Num rows: 2 Data size: 344 Basic stats: COMPLETE Column stats: NONE + Filter Operator + predicate: (hr is not null and (hour = 11)) (type: boolean) + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Reduce Output Operator + key expressions: hr (type: string) + sort order: + + Map-reduce partition columns: hr (type: string) + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Select Operator + expressions: hr (type: string) + outputColumnNames: _col0 + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Group By Operator + keys: _col0 (type: string) + mode: hash + outputColumnNames: _col0 + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Dynamic Partitioning Event Operator + Target Input: srcpart + Partition key expr: hr + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Target column: hr + Target Vertex: Map 1 + Map 4 + Map Operator Tree: + TableScan + alias: srcpart_date + filterExpr: (ds is not null and (date = '2008-04-08')) (type: boolean) + Statistics: Num rows: 2 Data size: 376 Basic stats: COMPLETE Column stats: NONE + Filter Operator + predicate: (ds is not null and (date = '2008-04-08')) (type: boolean) + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Reduce Output Operator + key expressions: ds (type: string) + sort order: + + Map-reduce partition columns: ds (type: string) + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Select Operator + expressions: ds (type: string) + outputColumnNames: _col0 + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Group By Operator + keys: _col0 (type: string) + mode: hash + outputColumnNames: _col0 + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Dynamic Partitioning Event Operator + Target Input: srcpart + Partition key expr: ds + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Target column: ds + Target Vertex: Map 1 + Execution mode: vectorized + 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 + Select Operator + expressions: _col0 (type: bigint) + 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.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + Execution mode: vectorized + + Stage: Stage-0 + Fetch Operator + limit: -1 + Processor Tree: + ListSink + +PREHOOK: query: 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 +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 +PREHOOK: Input: default@srcpart_hour +#### A masked pattern was here #### +POSTHOOK: query: 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 +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 +POSTHOOK: Input: default@srcpart_hour +#### A masked pattern was here #### +500 +PREHOOK: query: select count(*) from srcpart where hr = 11 and ds = '2008-04-08' +PREHOOK: type: QUERY +PREHOOK: Input: default@srcpart +PREHOOK: Input: default@srcpart@ds=2008-04-08/hr=11 +#### A masked pattern was here #### +POSTHOOK: query: select count(*) from srcpart where hr = 11 and ds = '2008-04-08' +POSTHOOK: type: QUERY +POSTHOOK: Input: default@srcpart +POSTHOOK: Input: default@srcpart@ds=2008-04-08/hr=11 +#### A masked pattern was here #### +500 +PREHOOK: query: -- 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 +PREHOOK: type: QUERY +POSTHOOK: query: -- 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 +POSTHOOK: type: QUERY +STAGE DEPENDENCIES: + Stage-1 is a root stage + Stage-0 depends on stages: Stage-1 + +STAGE PLANS: + Stage: Stage-1 + Tez + Edges: + Map 1 <- Map 3 (BROADCAST_EDGE) + Reducer 2 <- Map 1 (SIMPLE_EDGE) +#### 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: COMPLETE + Map Join Operator + condition map: + Inner Join 0 to 1 + condition expressions: + 0 + 1 + keys: + 0 ds (type: string), hr (type: string) + 1 ds (type: string), hr (type: string) + Statistics: Num rows: 2200 Data size: 23372 Basic stats: COMPLETE Column stats: NONE + Select Operator + 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) + Map 3 + Map Operator Tree: + TableScan + alias: srcpart_date_hour + filterExpr: (((ds is not null and hr is not null) and (date = '2008-04-08')) and (hour = 11)) (type: boolean) + Statistics: Num rows: 4 Data size: 1440 Basic stats: COMPLETE Column stats: NONE + Filter Operator + predicate: (((ds is not null and hr is not null) and (date = '2008-04-08')) and (hour = 11)) (type: boolean) + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Reduce Output Operator + key expressions: ds (type: string), hr (type: string) + sort order: ++ + Map-reduce partition columns: ds (type: string), hr (type: string) + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Select Operator + expressions: ds (type: string) + outputColumnNames: _col0 + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Group By Operator + keys: _col0 (type: string) + mode: hash + outputColumnNames: _col0 + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Dynamic Partitioning Event Operator + Target Input: srcpart + Partition key expr: ds + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Target column: ds + Target Vertex: Map 1 + Select Operator + expressions: hr (type: string) + outputColumnNames: _col0 + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Group By Operator + keys: _col0 (type: string) + mode: hash + outputColumnNames: _col0 + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Dynamic Partitioning Event Operator + Target Input: srcpart + Partition key expr: hr + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Target column: hr + Target Vertex: Map 1 + 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 + Select Operator + expressions: _col0 (type: bigint) + 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.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + Execution mode: vectorized + + 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: select count(*) from srcpart where ds = '2008-04-08' and hr = 11 +PREHOOK: type: QUERY +PREHOOK: Input: default@srcpart +PREHOOK: Input: default@srcpart@ds=2008-04-08/hr=11 +#### A masked pattern was here #### +POSTHOOK: query: select count(*) from srcpart where ds = '2008-04-08' and hr = 11 +POSTHOOK: type: QUERY +POSTHOOK: Input: default@srcpart +POSTHOOK: Input: default@srcpart@ds=2008-04-08/hr=11 +#### A masked pattern was here #### +500 +PREHOOK: query: -- empty set +EXPLAIN select count(*) from srcpart join srcpart_date on (srcpart.ds = srcpart_date.ds) where srcpart_date.date = 'I DONT EXIST' +PREHOOK: type: QUERY +POSTHOOK: query: -- empty set +EXPLAIN select count(*) from srcpart join srcpart_date on (srcpart.ds = srcpart_date.ds) where srcpart_date.date = 'I DONT EXIST' +POSTHOOK: type: QUERY +STAGE DEPENDENCIES: + Stage-1 is a root stage + Stage-0 depends on stages: Stage-1 + +STAGE PLANS: + Stage: Stage-1 + Tez + Edges: + Map 1 <- Map 3 (BROADCAST_EDGE) + Reducer 2 <- Map 1 (SIMPLE_EDGE) +#### 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: COMPLETE + Map Join Operator + condition map: + Inner Join 0 to 1 + condition expressions: + 0 + 1 + keys: + 0 ds (type: string) + 1 ds (type: string) + Statistics: Num rows: 2200 Data size: 23372 Basic stats: COMPLETE Column stats: NONE + Select Operator + 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) + Map 3 + Map Operator Tree: + TableScan + alias: srcpart_date + filterExpr: (ds is not null and (date = 'I DONT EXIST')) (type: boolean) + Statistics: Num rows: 2 Data size: 376 Basic stats: COMPLETE Column stats: NONE + Filter Operator + predicate: (ds is not null and (date = 'I DONT EXIST')) (type: boolean) + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Reduce Output Operator + key expressions: ds (type: string) + sort order: + + Map-reduce partition columns: ds (type: string) + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Select Operator + expressions: ds (type: string) + outputColumnNames: _col0 + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Group By Operator + keys: _col0 (type: string) + mode: hash + outputColumnNames: _col0 + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Dynamic Partitioning Event Operator + Target Input: srcpart + Partition key expr: ds + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Target column: ds + Target Vertex: Map 1 + Execution mode: vectorized + 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 + Select Operator + expressions: _col0 (type: bigint) + 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.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + Execution mode: vectorized + + Stage: Stage-0 + Fetch Operator + limit: -1 + Processor Tree: + ListSink + +PREHOOK: query: -- Disabled until TEZ-1486 is fixed +-- select count(*) from srcpart join srcpart_date on (srcpart.ds = srcpart_date.ds) where srcpart_date.date = 'I DONT EXIST'; + +-- expressions +EXPLAIN select count(*) from srcpart join srcpart_double_hour on (srcpart.hr = cast(srcpart_double_hour.hr/2 as int)) where srcpart_double_hour.hour = 11 +PREHOOK: type: QUERY +POSTHOOK: query: -- Disabled until TEZ-1486 is fixed +-- select count(*) from srcpart join srcpart_date on (srcpart.ds = srcpart_date.ds) where srcpart_date.date = 'I DONT EXIST'; + +-- expressions +EXPLAIN select count(*) from srcpart join srcpart_double_hour on (srcpart.hr = cast(srcpart_double_hour.hr/2 as int)) where srcpart_double_hour.hour = 11 +POSTHOOK: type: QUERY +STAGE DEPENDENCIES: + Stage-1 is a root stage + Stage-0 depends on stages: Stage-1 + +STAGE PLANS: + Stage: Stage-1 + Tez + Edges: + Map 1 <- Map 3 (BROADCAST_EDGE) + Reducer 2 <- Map 1 (SIMPLE_EDGE) +#### A masked pattern was here #### + Vertices: + Map 1 + Map Operator Tree: + TableScan + alias: srcpart + filterExpr: UDFToDouble(hr) is not null (type: boolean) + Statistics: Num rows: 2000 Data size: 21248 Basic stats: COMPLETE Column stats: COMPLETE + Filter Operator + predicate: UDFToDouble(hr) is not null (type: boolean) + Statistics: Num rows: 1000 Data size: 0 Basic stats: PARTIAL Column stats: COMPLETE + Map Join Operator + condition map: + Inner Join 0 to 1 + condition expressions: + 0 + 1 + keys: + 0 UDFToDouble(hr) (type: double) + 1 UDFToDouble(UDFToInteger((hr / 2))) (type: double) + Statistics: Num rows: 1100 Data size: 0 Basic stats: PARTIAL Column stats: NONE + Select Operator + Statistics: Num rows: 1100 Data size: 0 Basic stats: PARTIAL 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) + Map 3 + Map Operator Tree: + TableScan + alias: srcpart_double_hour + filterExpr: (UDFToDouble(UDFToInteger((hr / 2))) is not null and (hour = 11)) (type: boolean) + Statistics: Num rows: 2 Data size: 188 Basic stats: COMPLETE Column stats: NONE + Filter Operator + predicate: (UDFToDouble(UDFToInteger((hr / 2))) is not null and (hour = 11)) (type: boolean) + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Reduce Output Operator + key expressions: UDFToDouble(UDFToInteger((hr / 2))) (type: double) + sort order: + + Map-reduce partition columns: UDFToDouble(UDFToInteger((hr / 2))) (type: double) + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Select Operator + expressions: UDFToDouble(UDFToInteger((hr / 2))) (type: double) + outputColumnNames: _col0 + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Group By Operator + keys: _col0 (type: double) + mode: hash + outputColumnNames: _col0 + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Dynamic Partitioning Event Operator + Target Input: srcpart + Partition key expr: UDFToDouble(hr) + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Target column: hr + Target Vertex: Map 1 + 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 + Select Operator + expressions: _col0 (type: bigint) + 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.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + Execution mode: vectorized + + Stage: Stage-0 + Fetch Operator + limit: -1 + Processor Tree: + ListSink + +PREHOOK: query: select count(*) from srcpart join srcpart_double_hour on (srcpart.hr = cast(srcpart_double_hour.hr/2 as int)) where srcpart_double_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_double_hour +#### A masked pattern was here #### +POSTHOOK: query: select count(*) from srcpart join srcpart_double_hour on (srcpart.hr = cast(srcpart_double_hour.hr/2 as int)) where srcpart_double_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_double_hour +#### A masked pattern was here #### +1000 +PREHOOK: query: EXPLAIN select count(*) from srcpart join srcpart_double_hour on (srcpart.hr*2 = srcpart_double_hour.hr) where srcpart_double_hour.hour = 11 +PREHOOK: type: QUERY +POSTHOOK: query: EXPLAIN select count(*) from srcpart join srcpart_double_hour on (srcpart.hr*2 = srcpart_double_hour.hr) where srcpart_double_hour.hour = 11 +POSTHOOK: type: QUERY +STAGE DEPENDENCIES: + Stage-1 is a root stage + Stage-0 depends on stages: Stage-1 + +STAGE PLANS: + Stage: Stage-1 + Tez + Edges: + Map 1 <- Map 3 (BROADCAST_EDGE) + Reducer 2 <- Map 1 (SIMPLE_EDGE) +#### A masked pattern was here #### + Vertices: + Map 1 + Map Operator Tree: + TableScan + alias: srcpart + filterExpr: (hr * 2) is not null (type: boolean) + Statistics: Num rows: 2000 Data size: 21248 Basic stats: COMPLETE Column stats: COMPLETE + Filter Operator + predicate: (hr * 2) is not null (type: boolean) + Statistics: Num rows: 1000 Data size: 0 Basic stats: PARTIAL Column stats: COMPLETE + Map Join Operator + condition map: + Inner Join 0 to 1 + condition expressions: + 0 + 1 + keys: + 0 (hr * 2) (type: double) + 1 hr (type: double) + Statistics: Num rows: 1100 Data size: 0 Basic stats: PARTIAL Column stats: NONE + Select Operator + Statistics: Num rows: 1100 Data size: 0 Basic stats: PARTIAL 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) + Map 3 + Map Operator Tree: + TableScan + alias: srcpart_double_hour + filterExpr: (hr is not null and (hour = 11)) (type: boolean) + Statistics: Num rows: 2 Data size: 188 Basic stats: COMPLETE Column stats: NONE + Filter Operator + predicate: (hr is not null and (hour = 11)) (type: boolean) + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Reduce Output Operator + key expressions: hr (type: double) + sort order: + + Map-reduce partition columns: hr (type: double) + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Select Operator + expressions: hr (type: double) + outputColumnNames: _col0 + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Group By Operator + keys: _col0 (type: double) + mode: hash + outputColumnNames: _col0 + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Dynamic Partitioning Event Operator + Target Input: srcpart + Partition key expr: (hr * 2) + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Target column: hr + Target Vertex: Map 1 + 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 + Select Operator + expressions: _col0 (type: bigint) + 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.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + Execution mode: vectorized + + Stage: Stage-0 + Fetch Operator + limit: -1 + Processor Tree: + ListSink + +PREHOOK: query: select count(*) from srcpart join srcpart_double_hour on (srcpart.hr*2 = srcpart_double_hour.hr) where srcpart_double_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_double_hour +#### A masked pattern was here #### +POSTHOOK: query: select count(*) from srcpart join srcpart_double_hour on (srcpart.hr*2 = srcpart_double_hour.hr) where srcpart_double_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_double_hour +#### A masked pattern was here #### +1000 +PREHOOK: query: select count(*) from srcpart where hr = 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-09/hr=11 +#### A masked pattern was here #### +POSTHOOK: query: select count(*) from srcpart where hr = 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-09/hr=11 +#### A masked pattern was here #### +1000 +PREHOOK: query: -- parent is reduce tasks +EXPLAIN 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' +PREHOOK: type: QUERY +POSTHOOK: query: -- parent is reduce tasks +EXPLAIN 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' +POSTHOOK: type: QUERY +STAGE DEPENDENCIES: + Stage-1 is a root stage + Stage-0 depends on stages: Stage-1 + +STAGE PLANS: + Stage: Stage-1 + Tez + Edges: + Reducer 3 <- Map 1 (BROADCAST_EDGE), Map 2 (SIMPLE_EDGE) + Reducer 4 <- Reducer 3 (SIMPLE_EDGE) +#### 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: COMPLETE + Reduce Output Operator + key expressions: ds (type: string) + sort order: + + Map-reduce partition columns: ds (type: string) + Statistics: Num rows: 2000 Data size: 21248 Basic stats: COMPLETE Column stats: COMPLETE + Select Operator + expressions: ds (type: string) + outputColumnNames: _col0 + Statistics: Num rows: 2000 Data size: 0 Basic stats: PARTIAL Column stats: COMPLETE + Group By Operator + keys: _col0 (type: string) + mode: hash + outputColumnNames: _col0 + Statistics: Num rows: 2000 Data size: 0 Basic stats: PARTIAL Column stats: COMPLETE + Dynamic Partitioning Event Operator + Target Input: srcpart + Partition key expr: ds + Statistics: Num rows: 2000 Data size: 0 Basic stats: PARTIAL Column stats: COMPLETE + Target column: ds + Target Vertex: Map 2 + Map 2 + Map Operator Tree: + TableScan + alias: srcpart + filterExpr: (ds is not null and (ds = '2008-04-08')) (type: boolean) + Statistics: Num rows: 1000 Data size: 10624 Basic stats: COMPLETE Column stats: COMPLETE + Select Operator + expressions: ds (type: string) + outputColumnNames: ds + Statistics: Num rows: 1000 Data size: 10624 Basic stats: COMPLETE Column stats: COMPLETE + Group By Operator + keys: ds (type: string) + mode: hash + outputColumnNames: _col0 + Statistics: Num rows: 1000 Data size: 184000 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + key expressions: _col0 (type: string) + sort order: + + Map-reduce partition columns: _col0 (type: string) + Statistics: Num rows: 1000 Data size: 184000 Basic stats: COMPLETE Column stats: COMPLETE + Reducer 3 + Reduce Operator Tree: + Group By Operator + keys: KEY._col0 (type: string) + mode: mergepartial + outputColumnNames: _col0 + Statistics: Num rows: 500 Data size: 92000 Basic stats: COMPLETE Column stats: COMPLETE + Select Operator + expressions: _col0 (type: string) + outputColumnNames: _col0 + Statistics: Num rows: 500 Data size: 92000 Basic stats: COMPLETE Column stats: COMPLETE + Map Join Operator + condition map: + Inner Join 0 to 1 + condition expressions: + 0 + 1 + keys: + 0 ds (type: string) + 1 _col0 (type: string) + Statistics: Num rows: 500 Data size: 0 Basic stats: PARTIAL Column stats: COMPLETE + Select Operator + Statistics: Num rows: 500 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: COMPLETE Column stats: COMPLETE + Reduce Output Operator + sort order: + Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: _col0 (type: bigint) + Reducer 4 + 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: COMPLETE + Select Operator + expressions: _col0 (type: bigint) + outputColumnNames: _col0 + Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE + File Output Operator + compressed: false + Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE + table: + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + Execution mode: vectorized + + Stage: Stage-0 + Fetch Operator + limit: -1 + Processor Tree: + ListSink + +PREHOOK: query: 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' +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 +#### A masked pattern was here #### +POSTHOOK: query: 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' +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 +#### A masked pattern was here #### +1000 +PREHOOK: query: select count(*) from srcpart where ds = '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 +#### A masked pattern was here #### +POSTHOOK: query: select count(*) from srcpart where ds = '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 +#### A masked pattern was here #### +1000 +PREHOOK: query: -- left join +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: -- left join +EXPLAIN select count(*) from srcpart left join srcpart_date on (srcpart.ds = 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 + Tez + Edges: + Map 1 <- Map 3 (BROADCAST_EDGE) + Reducer 2 <- Map 1 (SIMPLE_EDGE) +#### 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: COMPLETE + Map Join Operator + condition map: + Left Outer Join0 to 1 + condition expressions: + 0 + 1 {date} + keys: + 0 ds (type: string) + 1 ds (type: string) + outputColumnNames: _col8 + Statistics: Num rows: 2200 Data size: 23372 Basic stats: COMPLETE Column stats: NONE + Filter Operator + predicate: (_col8 = '2008-04-08') (type: boolean) + Statistics: Num rows: 1100 Data size: 11686 Basic stats: COMPLETE Column stats: NONE + Select Operator + Statistics: Num rows: 1100 Data size: 11686 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) + Map 3 + Map Operator Tree: + TableScan + alias: srcpart_date + Statistics: Num rows: 2 Data size: 376 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + key expressions: ds (type: string) + sort order: + + Map-reduce partition columns: ds (type: string) + Statistics: Num rows: 2 Data size: 376 Basic stats: COMPLETE Column stats: NONE + value expressions: date (type: string) + Execution mode: vectorized + 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 + Select Operator + expressions: _col0 (type: bigint) + 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.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + Execution mode: vectorized + + Stage: Stage-0 + Fetch Operator + limit: -1 + Processor Tree: + ListSink + +PREHOOK: query: EXPLAIN select count(*) from srcpart_date left join srcpart on (srcpart.ds = srcpart_date.ds) where srcpart_date.date = '2008-04-08' +PREHOOK: type: QUERY +POSTHOOK: query: EXPLAIN select count(*) from srcpart_date left join srcpart on (srcpart.ds = 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 + Tez + Edges: + Map 2 <- Map 1 (BROADCAST_EDGE) + Reducer 3 <- Map 2 (SIMPLE_EDGE) +#### 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: COMPLETE + Reduce Output Operator + key expressions: ds (type: string) + sort order: + + Map-reduce partition columns: ds (type: string) + Statistics: Num rows: 2000 Data size: 21248 Basic stats: COMPLETE Column stats: COMPLETE + Map 2 + Map Operator Tree: + TableScan + alias: srcpart_date + filterExpr: (date = '2008-04-08') (type: boolean) + Statistics: Num rows: 2 Data size: 376 Basic stats: COMPLETE Column stats: NONE + Filter Operator + predicate: (date = '2008-04-08') (type: boolean) + Statistics: Num rows: 1 Data size: 188 Basic stats: COMPLETE Column stats: NONE + Map Join Operator + condition map: + Left Outer Join0 to 1 + condition expressions: + 0 + 1 + keys: + 0 ds (type: string) + 1 ds (type: string) + Statistics: Num rows: 2200 Data size: 23372 Basic stats: COMPLETE Column stats: NONE + Select Operator + 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) + Execution mode: vectorized + 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 + Select Operator + expressions: _col0 (type: bigint) + 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.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + Execution mode: vectorized + + Stage: Stage-0 + Fetch Operator + limit: -1 + Processor Tree: + ListSink + +PREHOOK: query: -- full outer +EXPLAIN select count(*) from srcpart full outer join srcpart_date on (srcpart.ds = srcpart_date.ds) where srcpart_date.date = '2008-04-08' +PREHOOK: type: QUERY +POSTHOOK: query: -- full outer +EXPLAIN select count(*) from srcpart full outer join srcpart_date on (srcpart.ds = 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 + Tez + Edges: + Reducer 2 <- Map 1 (SIMPLE_EDGE), Map 4 (SIMPLE_EDGE) + Reducer 3 <- Reducer 2 (SIMPLE_EDGE) +#### 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: COMPLETE + Reduce Output Operator + key expressions: ds (type: string) + sort order: + + Map-reduce partition columns: ds (type: string) + Statistics: Num rows: 2000 Data size: 21248 Basic stats: COMPLETE Column stats: COMPLETE + Map 4 + Map Operator Tree: + TableScan + alias: srcpart_date + Statistics: Num rows: 2 Data size: 376 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + key expressions: ds (type: string) + sort order: + + Map-reduce partition columns: ds (type: string) + Statistics: Num rows: 2 Data size: 376 Basic stats: COMPLETE Column stats: NONE + value expressions: date (type: string) + Execution mode: vectorized + Reducer 2 + Reduce Operator Tree: + Join Operator + condition map: + Outer Join 0 to 1 + condition expressions: + 0 + 1 {VALUE._col0} + outputColumnNames: _col8 + Statistics: Num rows: 2200 Data size: 23372 Basic stats: COMPLETE Column stats: NONE + Filter Operator + predicate: (_col8 = '2008-04-08') (type: boolean) + Statistics: Num rows: 1100 Data size: 11686 Basic stats: COMPLETE Column stats: NONE + Select Operator + Statistics: Num rows: 1100 Data size: 11686 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 + Select Operator + expressions: _col0 (type: bigint) + 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.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + Execution mode: vectorized + + Stage: Stage-0 + Fetch Operator + limit: -1 + Processor Tree: + ListSink + +PREHOOK: query: -- with static pruning +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 and srcpart.hr = 11 +PREHOOK: type: QUERY +POSTHOOK: query: -- with static pruning +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 and srcpart.hr = 11 +POSTHOOK: type: QUERY +STAGE DEPENDENCIES: + Stage-1 is a root stage + Stage-0 depends on stages: Stage-1 + +STAGE PLANS: + Stage: Stage-1 + Tez + Edges: + Map 1 <- Map 3 (BROADCAST_EDGE), Map 4 (BROADCAST_EDGE) + Reducer 2 <- Map 1 (SIMPLE_EDGE) +#### A masked pattern was here #### + Vertices: + Map 1 + Map Operator Tree: + TableScan + alias: srcpart + filterExpr: (hr = 11) (type: boolean) + Statistics: Num rows: 1000 Data size: 10624 Basic stats: COMPLETE Column stats: COMPLETE + Map Join Operator + condition map: + Inner Join 0 to 1 + condition expressions: + 0 {hr} + 1 + keys: + 0 ds (type: string) + 1 ds (type: string) + outputColumnNames: _col3 + Statistics: Num rows: 1100 Data size: 11686 Basic stats: COMPLETE Column stats: NONE + Map Join Operator + condition map: + Inner Join 0 to 1 + condition expressions: + 0 + 1 + keys: + 0 _col3 (type: string) + 1 '11' (type: string) + Statistics: Num rows: 1210 Data size: 12854 Basic stats: COMPLETE Column stats: NONE + Select Operator + Statistics: Num rows: 1210 Data size: 12854 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) + Map 3 + Map Operator Tree: + TableScan + alias: srcpart_hour + filterExpr: ((hr is not null and (hour = 11)) and (hr = 11)) (type: boolean) + Statistics: Num rows: 2 Data size: 344 Basic stats: COMPLETE Column stats: NONE + Filter Operator + predicate: ((hr is not null and (hour = 11)) and (hr = 11)) (type: boolean) + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Reduce Output Operator + key expressions: '11' (type: string) + sort order: + + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Select Operator + expressions: '11' (type: string) + outputColumnNames: _col0 + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Group By Operator + keys: _col0 (type: string) + mode: hash + outputColumnNames: _col0 + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Dynamic Partitioning Event Operator + Target Input: srcpart + Partition key expr: hr + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Target column: hr + Target Vertex: Map 1 + Map 4 + Map Operator Tree: + TableScan + alias: srcpart_date + filterExpr: (ds is not null and (date = '2008-04-08')) (type: boolean) + Statistics: Num rows: 2 Data size: 376 Basic stats: COMPLETE Column stats: NONE + Filter Operator + predicate: (ds is not null and (date = '2008-04-08')) (type: boolean) + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Reduce Output Operator + key expressions: ds (type: string) + sort order: + + Map-reduce partition columns: ds (type: string) + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Select Operator + expressions: ds (type: string) + outputColumnNames: _col0 + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Group By Operator + keys: _col0 (type: string) + mode: hash + outputColumnNames: _col0 + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Dynamic Partitioning Event Operator + Target Input: srcpart + Partition key expr: ds + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Target column: ds + Target Vertex: Map 1 + Execution mode: vectorized + 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 + Select Operator + expressions: _col0 (type: bigint) + 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.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + Execution mode: vectorized + + Stage: Stage-0 + Fetch Operator + limit: -1 + Processor Tree: + ListSink + +PREHOOK: query: 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 and srcpart.hr = 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-09/hr=11 +PREHOOK: Input: default@srcpart_date +PREHOOK: Input: default@srcpart_hour +#### A masked pattern was here #### +POSTHOOK: query: 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 and srcpart.hr = 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-09/hr=11 +POSTHOOK: Input: default@srcpart_date +POSTHOOK: Input: default@srcpart_hour +#### A masked pattern was here #### +500 +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.hr = 13 +PREHOOK: type: QUERY +POSTHOOK: 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.hr = 13 +POSTHOOK: type: QUERY +STAGE DEPENDENCIES: + Stage-1 is a root stage + Stage-0 depends on stages: Stage-1 + +STAGE PLANS: + Stage: Stage-1 + Tez + Edges: + Map 1 <- Map 3 (BROADCAST_EDGE), Map 4 (BROADCAST_EDGE) + Reducer 2 <- Map 1 (SIMPLE_EDGE) +#### A masked pattern was here #### + Vertices: + Map 1 + Map 3 + Map Operator Tree: + TableScan + alias: srcpart_hour + filterExpr: (hr is not null and (hr = 13)) (type: boolean) + Statistics: Num rows: 2 Data size: 344 Basic stats: COMPLETE Column stats: NONE + Filter Operator + predicate: (hr is not null and (hr = 13)) (type: boolean) + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Reduce Output Operator + key expressions: '13' (type: string) + sort order: + + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Map 4 + Map Operator Tree: + TableScan + alias: srcpart_date + filterExpr: (ds is not null and (date = '2008-04-08')) (type: boolean) + Statistics: Num rows: 2 Data size: 376 Basic stats: COMPLETE Column stats: NONE + Filter Operator + predicate: (ds is not null and (date = '2008-04-08')) (type: boolean) + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Reduce Output Operator + key expressions: ds (type: string) + sort order: + + Map-reduce partition columns: ds (type: string) + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Execution mode: vectorized + 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 + Select Operator + expressions: _col0 (type: bigint) + 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.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + Execution mode: vectorized + + Stage: Stage-0 + Fetch Operator + limit: -1 + Processor Tree: + ListSink + +PREHOOK: query: -- Disabled until TEZ-1486 is fixed +-- 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.hr = 13; + +-- union + subquery +EXPLAIN select distinct(ds) from srcpart where srcpart.ds in (select max(srcpart.ds) from srcpart union all select min(srcpart.ds) from srcpart) +PREHOOK: type: QUERY +POSTHOOK: query: -- Disabled until TEZ-1486 is fixed +-- 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.hr = 13; + +-- union + subquery +EXPLAIN select distinct(ds) from srcpart where srcpart.ds in (select max(srcpart.ds) from srcpart union all select min(srcpart.ds) from srcpart) +POSTHOOK: type: QUERY +STAGE DEPENDENCIES: + Stage-1 is a root stage + Stage-0 depends on stages: Stage-1 + +STAGE PLANS: + Stage: Stage-1 + Tez + Edges: + Map 4 <- Union 3 (BROADCAST_EDGE) + Reducer 2 <- Map 1 (SIMPLE_EDGE), Union 3 (CONTAINS) + Reducer 5 <- Map 4 (SIMPLE_EDGE) + Reducer 7 <- Map 6 (SIMPLE_EDGE), Union 3 (CONTAINS) +#### 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: COMPLETE + Select Operator + expressions: ds (type: string) + outputColumnNames: ds + Statistics: Num rows: 2000 Data size: 21248 Basic stats: COMPLETE Column stats: COMPLETE + Group By Operator + aggregations: min(ds) + mode: hash + outputColumnNames: _col0 + Statistics: Num rows: 1 Data size: 84 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + sort order: + Statistics: Num rows: 1 Data size: 84 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: _col0 (type: string) + Map 4 + 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: COMPLETE + Map Join Operator + condition map: + Left Semi Join 0 to 1 + condition expressions: + 0 {ds} + 1 + keys: + 0 ds (type: string) + 1 _col0 (type: string) + outputColumnNames: _col2 + Statistics: Num rows: 2 Data size: 368 Basic stats: COMPLETE Column stats: COMPLETE + Select Operator + expressions: _col2 (type: string) + outputColumnNames: _col2 + Statistics: Num rows: 2 Data size: 368 Basic stats: COMPLETE Column stats: COMPLETE + Group By Operator + keys: _col2 (type: string) + mode: hash + outputColumnNames: _col0 + Statistics: Num rows: 2 Data size: 368 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + key expressions: _col0 (type: string) + sort order: + + Map-reduce partition columns: _col0 (type: string) + Statistics: Num rows: 2 Data size: 368 Basic stats: COMPLETE Column stats: COMPLETE + Map 6 + Map Operator Tree: + TableScan + alias: srcpart + Statistics: Num rows: 2000 Data size: 21248 Basic stats: COMPLETE Column stats: COMPLETE + Select Operator + expressions: ds (type: string) + outputColumnNames: ds + Statistics: Num rows: 2000 Data size: 21248 Basic stats: COMPLETE Column stats: COMPLETE + Group By Operator + aggregations: max(ds) + mode: hash + outputColumnNames: _col0 + Statistics: Num rows: 1 Data size: 84 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + sort order: + Statistics: Num rows: 1 Data size: 84 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: _col0 (type: string) + Reducer 2 + Reduce Operator Tree: + Group By Operator + aggregations: min(VALUE._col0) + mode: mergepartial + outputColumnNames: _col0 + Filter Operator + predicate: _col0 is not null (type: boolean) + Select Operator + expressions: _col0 (type: string) + outputColumnNames: _col0 + Select Operator + expressions: _col0 (type: string) + outputColumnNames: _col0 + Group By Operator + keys: _col0 (type: string) + mode: hash + outputColumnNames: _col0 + Reduce Output Operator + key expressions: _col0 (type: string) + sort order: + + Map-reduce partition columns: _col0 (type: string) + Select Operator + expressions: _col0 (type: string) + outputColumnNames: _col0 + Group By Operator + keys: _col0 (type: string) + mode: hash + outputColumnNames: _col0 + Dynamic Partitioning Event Operator + Target Input: srcpart + Partition key expr: ds + Target column: ds + Target Vertex: Map 4 + Reducer 5 + Reduce Operator Tree: + Group By Operator + keys: KEY._col0 (type: string) + mode: mergepartial + outputColumnNames: _col0 + Statistics: Num rows: 1 Data size: 184 Basic stats: COMPLETE Column stats: COMPLETE + Select Operator + expressions: _col0 (type: string) + outputColumnNames: _col0 + Statistics: Num rows: 1 Data size: 184 Basic stats: COMPLETE Column stats: COMPLETE + File Output Operator + compressed: false + Statistics: Num rows: 1 Data size: 184 Basic stats: COMPLETE Column stats: COMPLETE + table: + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + Execution mode: vectorized + Reducer 7 + Reduce Operator Tree: + Group By Operator + aggregations: max(VALUE._col0) + mode: mergepartial + outputColumnNames: _col0 + Filter Operator + predicate: _col0 is not null (type: boolean) + Select Operator + expressions: _col0 (type: string) + outputColumnNames: _col0 + Select Operator + expressions: _col0 (type: string) + outputColumnNames: _col0 + Group By Operator + keys: _col0 (type: string) + mode: hash + outputColumnNames: _col0 + Reduce Output Operator + key expressions: _col0 (type: string) + sort order: + + Map-reduce partition columns: _col0 (type: string) + Select Operator + expressions: _col0 (type: string) + outputColumnNames: _col0 + Group By Operator + keys: _col0 (type: string) + mode: hash + outputColumnNames: _col0 + Dynamic Partitioning Event Operator + Target Input: srcpart + Partition key expr: ds + Target column: ds + Target Vertex: Map 4 + Union 3 + Vertex: Union 3 + + Stage: Stage-0 + Fetch Operator + limit: -1 + Processor Tree: + ListSink + +PREHOOK: query: select distinct(ds) from srcpart where srcpart.ds in (select max(srcpart.ds) from srcpart union all select min(srcpart.ds) from srcpart) +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 +#### A masked pattern was here #### +POSTHOOK: query: select distinct(ds) from srcpart where srcpart.ds in (select max(srcpart.ds) from srcpart union all select min(srcpart.ds) from srcpart) +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 +#### A masked pattern was here #### +2008-04-08 +2008-04-09 +PREHOOK: query: -- different file format +create table srcpart_orc (key int, value string) partitioned by (ds string, hr int) stored as orc +PREHOOK: type: CREATETABLE +PREHOOK: Output: database:default +PREHOOK: Output: default@srcpart_orc +POSTHOOK: query: -- different file format +create table srcpart_orc (key int, value string) partitioned by (ds string, hr int) stored as orc +POSTHOOK: type: CREATETABLE +POSTHOOK: Output: database:default +POSTHOOK: Output: default@srcpart_orc +PREHOOK: query: insert into table srcpart_orc partition (ds, hr) select key, value, ds, hr from srcpart +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: Output: default@srcpart_orc +POSTHOOK: query: insert into table srcpart_orc partition (ds, hr) select key, value, ds, hr from srcpart +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: Output: default@srcpart_orc@ds=2008-04-08/hr=11 +POSTHOOK: Output: default@srcpart_orc@ds=2008-04-08/hr=12 +POSTHOOK: Output: default@srcpart_orc@ds=2008-04-09/hr=11 +POSTHOOK: Output: default@srcpart_orc@ds=2008-04-09/hr=12 +POSTHOOK: Lineage: srcpart_orc PARTITION(ds=2008-04-08,hr=11).key EXPRESSION [(srcpart)srcpart.FieldSchema(name:key, type:string, comment:default), ] +POSTHOOK: Lineage: srcpart_orc PARTITION(ds=2008-04-08,hr=11).value SIMPLE [(srcpart)srcpart.FieldSchema(name:value, type:string, comment:default), ] +POSTHOOK: Lineage: srcpart_orc PARTITION(ds=2008-04-08,hr=12).key EXPRESSION [(srcpart)srcpart.FieldSchema(name:key, type:string, comment:default), ] +POSTHOOK: Lineage: srcpart_orc PARTITION(ds=2008-04-08,hr=12).value SIMPLE [(srcpart)srcpart.FieldSchema(name:value, type:string, comment:default), ] +POSTHOOK: Lineage: srcpart_orc PARTITION(ds=2008-04-09,hr=11).key EXPRESSION [(srcpart)srcpart.FieldSchema(name:key, type:string, comment:default), ] +POSTHOOK: Lineage: srcpart_orc PARTITION(ds=2008-04-09,hr=11).value SIMPLE [(srcpart)srcpart.FieldSchema(name:value, type:string, comment:default), ] +POSTHOOK: Lineage: srcpart_orc PARTITION(ds=2008-04-09,hr=12).key EXPRESSION [(srcpart)srcpart.FieldSchema(name:key, type:string, comment:default), ] +POSTHOOK: Lineage: srcpart_orc PARTITION(ds=2008-04-09,hr=12).value SIMPLE [(srcpart)srcpart.FieldSchema(name:value, type:string, comment:default), ] +PREHOOK: query: EXPLAIN select count(*) from srcpart_orc join srcpart_date_hour on (srcpart_orc.ds = srcpart_date_hour.ds and srcpart_orc.hr = srcpart_date_hour.hr) where srcpart_date_hour.hour = 11 and (srcpart_date_hour.date = '2008-04-08' or srcpart_date_hour.date = '2008-04-09') +PREHOOK: type: QUERY +POSTHOOK: query: EXPLAIN select count(*) from srcpart_orc join srcpart_date_hour on (srcpart_orc.ds = srcpart_date_hour.ds and srcpart_orc.hr = srcpart_date_hour.hr) where srcpart_date_hour.hour = 11 and (srcpart_date_hour.date = '2008-04-08' or srcpart_date_hour.date = '2008-04-09') +POSTHOOK: type: QUERY +STAGE DEPENDENCIES: + Stage-1 is a root stage + Stage-0 depends on stages: Stage-1 + +STAGE PLANS: + Stage: Stage-1 + Tez + Edges: + Map 1 <- Map 3 (BROADCAST_EDGE) + Reducer 2 <- Map 1 (SIMPLE_EDGE) +#### A masked pattern was here #### + Vertices: + Map 1 + Map Operator Tree: + TableScan + alias: srcpart_orc + filterExpr: UDFToDouble(hr) is not null (type: boolean) + Statistics: Num rows: 2000 Data size: 188000 Basic stats: COMPLETE Column stats: COMPLETE + Filter Operator + predicate: UDFToDouble(hr) is not null (type: boolean) + Statistics: Num rows: 1000 Data size: 0 Basic stats: PARTIAL Column stats: COMPLETE + Map Join Operator + condition map: + Inner Join 0 to 1 + condition expressions: + 0 + 1 + keys: + 0 ds (type: string), UDFToDouble(hr) (type: double) + 1 ds (type: string), UDFToDouble(hr) (type: double) + Statistics: Num rows: 1100 Data size: 0 Basic stats: PARTIAL Column stats: NONE + Select Operator + Statistics: Num rows: 1100 Data size: 0 Basic stats: PARTIAL 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) + Map 3 + Map Operator Tree: + TableScan + alias: srcpart_date_hour + filterExpr: (((ds is not null and UDFToDouble(hr) is not null) and (hour = 11)) and ((date = '2008-04-08') or (date = '2008-04-09'))) (type: boolean) + Statistics: Num rows: 4 Data size: 1440 Basic stats: COMPLETE Column stats: NONE + Filter Operator + predicate: (((ds is not null and UDFToDouble(hr) is not null) and (hour = 11)) and ((date = '2008-04-08') or (date = '2008-04-09'))) (type: boolean) + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Reduce Output Operator + key expressions: ds (type: string), UDFToDouble(hr) (type: double) + sort order: ++ + Map-reduce partition columns: ds (type: string), UDFToDouble(hr) (type: double) + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Select Operator + expressions: ds (type: string) + outputColumnNames: _col0 + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Group By Operator + keys: _col0 (type: string) + mode: hash + outputColumnNames: _col0 + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Dynamic Partitioning Event Operator + Target Input: srcpart_orc + Partition key expr: ds + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Target column: ds + Target Vertex: Map 1 + Select Operator + expressions: UDFToDouble(hr) (type: double) + outputColumnNames: _col0 + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Group By Operator + keys: _col0 (type: double) + mode: hash + outputColumnNames: _col0 + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Dynamic Partitioning Event Operator + Target Input: srcpart_orc + Partition key expr: UDFToDouble(hr) + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Target column: hr + Target Vertex: Map 1 + 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 + Select Operator + expressions: _col0 (type: bigint) + 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.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + + Stage: Stage-0 + Fetch Operator + limit: -1 + Processor Tree: + ListSink + +PREHOOK: query: select count(*) from srcpart_orc join srcpart_date_hour on (srcpart_orc.ds = srcpart_date_hour.ds and srcpart_orc.hr = srcpart_date_hour.hr) where srcpart_date_hour.hour = 11 and (srcpart_date_hour.date = '2008-04-08' or srcpart_date_hour.date = '2008-04-09') +PREHOOK: type: QUERY +PREHOOK: Input: default@srcpart_date_hour +PREHOOK: Input: default@srcpart_orc +PREHOOK: Input: default@srcpart_orc@ds=2008-04-08/hr=11 +PREHOOK: Input: default@srcpart_orc@ds=2008-04-08/hr=12 +PREHOOK: Input: default@srcpart_orc@ds=2008-04-09/hr=11 +PREHOOK: Input: default@srcpart_orc@ds=2008-04-09/hr=12 +#### A masked pattern was here #### +POSTHOOK: query: select count(*) from srcpart_orc join srcpart_date_hour on (srcpart_orc.ds = srcpart_date_hour.ds and srcpart_orc.hr = srcpart_date_hour.hr) where srcpart_date_hour.hour = 11 and (srcpart_date_hour.date = '2008-04-08' or srcpart_date_hour.date = '2008-04-09') +POSTHOOK: type: QUERY +POSTHOOK: Input: default@srcpart_date_hour +POSTHOOK: Input: default@srcpart_orc +POSTHOOK: Input: default@srcpart_orc@ds=2008-04-08/hr=11 +POSTHOOK: Input: default@srcpart_orc@ds=2008-04-08/hr=12 +POSTHOOK: Input: default@srcpart_orc@ds=2008-04-09/hr=11 +POSTHOOK: Input: default@srcpart_orc@ds=2008-04-09/hr=12 +#### A masked pattern was here #### +1000 +PREHOOK: query: select count(*) from srcpart where (ds = '2008-04-08' or ds = '2008-04-09') and hr = 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-09/hr=11 +#### A masked pattern was here #### +POSTHOOK: query: select count(*) from srcpart where (ds = '2008-04-08' or ds = '2008-04-09') and hr = 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-09/hr=11 +#### A masked pattern was here #### +1000 +PREHOOK: query: drop table srcpart_orc +PREHOOK: type: DROPTABLE +PREHOOK: Input: default@srcpart_orc +PREHOOK: Output: default@srcpart_orc +POSTHOOK: query: drop table srcpart_orc +POSTHOOK: type: DROPTABLE +POSTHOOK: Input: default@srcpart_orc +POSTHOOK: Output: default@srcpart_orc +PREHOOK: query: drop table srcpart_date +PREHOOK: type: DROPTABLE +PREHOOK: Input: default@srcpart_date +PREHOOK: Output: default@srcpart_date +POSTHOOK: query: drop table srcpart_date +POSTHOOK: type: DROPTABLE +POSTHOOK: Input: default@srcpart_date +POSTHOOK: Output: default@srcpart_date +PREHOOK: query: drop table srcpart_hour +PREHOOK: type: DROPTABLE +PREHOOK: Input: default@srcpart_hour +PREHOOK: Output: default@srcpart_hour +POSTHOOK: query: drop table srcpart_hour +POSTHOOK: type: DROPTABLE +POSTHOOK: Input: default@srcpart_hour +POSTHOOK: Output: default@srcpart_hour +PREHOOK: query: drop table srcpart_date_hour +PREHOOK: type: DROPTABLE +PREHOOK: Input: default@srcpart_date_hour +PREHOOK: Output: default@srcpart_date_hour +POSTHOOK: query: drop table srcpart_date_hour +POSTHOOK: type: DROPTABLE +POSTHOOK: Input: default@srcpart_date_hour +POSTHOOK: Output: default@srcpart_date_hour +PREHOOK: query: drop table srcpart_double_hour +PREHOOK: type: DROPTABLE +PREHOOK: Input: default@srcpart_double_hour +PREHOOK: Output: default@srcpart_double_hour +POSTHOOK: query: drop table srcpart_double_hour +POSTHOOK: type: DROPTABLE +POSTHOOK: Input: default@srcpart_double_hour +POSTHOOK: Output: default@srcpart_double_hour