diff --git common/src/java/org/apache/hadoop/hive/conf/HiveConf.java common/src/java/org/apache/hadoop/hive/conf/HiveConf.java index 7f4afd9..191e6ef 100644 --- common/src/java/org/apache/hadoop/hive/conf/HiveConf.java +++ common/src/java/org/apache/hadoop/hive/conf/HiveConf.java @@ -366,7 +366,6 @@ METASTORECONNECTURLKEY("javax.jdo.option.ConnectionURL", "jdbc:derby:;databaseName=metastore_db;create=true", "JDBC connect string for a JDBC metastore"), - HMSHANDLERATTEMPTS("hive.hmshandler.retry.attempts", 1, "The number of times to retry a HMSHandler call if there were a connection error"), HMSHANDLERINTERVAL("hive.hmshandler.retry.interval", 1000, @@ -1729,7 +1728,15 @@ "When auto reducer parallelism is enabled this factor will be used to over-partition data in shuffle edges."), TEZ_MIN_PARTITION_FACTOR("hive.tez.min.partition.factor", 0.25f, "When auto reducer parallelism is enabled this factor will be used to put a lower limit to the number\n" + - "of reducers that tez specifies.") + "of reducers that tez specifies."), + TEZ_DYNAMIC_PARTITION_PRUNING( + "hive.tez.dynamic.partition.pruning", true, + "When dynamic pruning is enabled, joins on partition keys will be processed by sending events from the processing " + + "vertices to the tez application master. These events will be used to prune unnecessary partitions."), + TEZ_DYNAMIC_PARTITION_PRUNING_MAX_EVENT_SIZE("hive.tez.dynamic.partition.pruning.max.event.size", 1*1024*1024L, + "Maximum size of events sent by processors in dynamic pruning. If this size is crossed no pruning will take place."), + TEZ_DYNAMIC_PARTITION_PRUNING_MAX_DATA_SIZE("hive.tez.dynamic.parition.pruning.max.data.size", 100*1024*1024L, + "Maximum total data size of events in dynamic pruning.") ; public final String varname; diff --git itests/qtest/testconfiguration.properties itests/qtest/testconfiguration.properties index d0df420..6fd1096 100644 --- itests/qtest/testconfiguration.properties +++ itests/qtest/testconfiguration.properties @@ -1,5 +1,5 @@ minimr.query.files=stats_counter_partitioned.q,list_bucket_dml_10.q,input16_cc.q,scriptfile1.q,scriptfile1_win.q,bucket4.q,bucketmapjoin6.q,disable_merge_for_bucketing.q,reduce_deduplicate.q,smb_mapjoin_8.q,join1.q,groupby2.q,bucketizedhiveinputformat.q,bucketmapjoin7.q,optrstat_groupby.q,bucket_num_reducers.q,bucket5.q,load_fs2.q,bucket_num_reducers2.q,infer_bucket_sort_merge.q,infer_bucket_sort_reducers_power_two.q,infer_bucket_sort_dyn_part.q,infer_bucket_sort_bucketed_table.q,infer_bucket_sort_map_operators.q,infer_bucket_sort_num_buckets.q,leftsemijoin_mr.q,schemeAuthority.q,schemeAuthority2.q,truncate_column_buckets.q,remote_script.q,,load_hdfs_file_with_space_in_the_name.q,parallel_orderby.q,import_exported_table.q,stats_counter.q,auto_sortmerge_join_16.q,quotedid_smb.q,file_with_header_footer.q,external_table_with_space_in_location_path.q,root_dir_external_table.q,index_bitmap3.q,ql_rewrite_gbtoidx.q,index_bitmap_auto.q,udf_using.q,empty_dir_in_table.q,temp_table_external.q minimr.query.negative.files=cluster_tasklog_retrieval.q,minimr_broken_pipe.q,mapreduce_stack_trace.q,mapreduce_stack_trace_turnoff.q,mapreduce_stack_trace_hadoop20.q,mapreduce_stack_trace_turnoff_hadoop20.q,file_with_header_footer_negative.q,udf_local_resource.q -minitez.query.files=tez_fsstat.q,mapjoin_decimal.q,tez_join_tests.q,tez_joins_explain.q,mrr.q,tez_dml.q,tez_insert_overwrite_local_directory_1.q,tez_union.q,bucket_map_join_tez1.q,bucket_map_join_tez2.q,tez_schema_evolution.q,tez_join_hash.q,tez_bmj_schema_evolution.q +minitez.query.files=tez_fsstat.q,mapjoin_decimal.q,tez_join_tests.q,tez_joins_explain.q,mrr.q,tez_dml.q,tez_insert_overwrite_local_directory_1.q,tez_union.q,bucket_map_join_tez1.q,bucket_map_join_tez2.q,tez_schema_evolution.q,tez_join_hash.q,tez_bmj_schema_evolution.q,dynamic_partition_pruning.q minitez.query.files.shared=cross_product_check_1.q,cross_product_check_2.q,dynpart_sort_opt_vectorization.q,dynpart_sort_optimization.q,orc_analyze.q,join0.q,join1.q,auto_join0.q,auto_join1.q,bucket2.q,bucket3.q,bucket4.q,count.q,create_merge_compressed.q,cross_join.q,ctas.q,custom_input_output_format.q,disable_merge_for_bucketing.q,enforce_order.q,filter_join_breaktask.q,filter_join_breaktask2.q,groupby1.q,groupby2.q,groupby3.q,having.q,insert1.q,insert_into1.q,insert_into2.q,leftsemijoin.q,limit_pushdown.q,load_dyn_part1.q,load_dyn_part2.q,load_dyn_part3.q,mapjoin_mapjoin.q,mapreduce1.q,mapreduce2.q,merge1.q,merge2.q,metadata_only_queries.q,sample1.q,subquery_in.q,subquery_exists.q,vectorization_15.q,ptf.q,stats_counter.q,stats_noscan_1.q,stats_counter_partitioned.q,union2.q,union3.q,union4.q,union5.q,union6.q,union7.q,union8.q,union9.q,transform1.q,transform2.q,transform_ppr1.q,transform_ppr2.q,script_env_var1.q,script_env_var2.q,script_pipe.q,scriptfile1.q,metadataonly1.q,temp_table.q,vectorized_ptf.q,optimize_nullscan.q beeline.positive.exclude=add_part_exist.q,alter1.q,alter2.q,alter4.q,alter5.q,alter_rename_partition.q,alter_rename_partition_authorization.q,archive.q,archive_corrupt.q,archive_multi.q,archive_mr_1806.q,archive_multi_mr_1806.q,authorization_1.q,authorization_2.q,authorization_4.q,authorization_5.q,authorization_6.q,authorization_7.q,ba_table1.q,ba_table2.q,ba_table3.q,ba_table_udfs.q,binary_table_bincolserde.q,binary_table_colserde.q,cluster.q,columnarserde_create_shortcut.q,combine2.q,constant_prop.q,create_nested_type.q,create_or_replace_view.q,create_struct_table.q,create_union_table.q,database.q,database_location.q,database_properties.q,ddltime.q,describe_database_json.q,drop_database_removes_partition_dirs.q,escape1.q,escape2.q,exim_00_nonpart_empty.q,exim_01_nonpart.q,exim_02_00_part_empty.q,exim_02_part.q,exim_03_nonpart_over_compat.q,exim_04_all_part.q,exim_04_evolved_parts.q,exim_05_some_part.q,exim_06_one_part.q,exim_07_all_part_over_nonoverlap.q,exim_08_nonpart_rename.q,exim_09_part_spec_nonoverlap.q,exim_10_external_managed.q,exim_11_managed_external.q,exim_12_external_location.q,exim_13_managed_location.q,exim_14_managed_location_over_existing.q,exim_15_external_part.q,exim_16_part_external.q,exim_17_part_managed.q,exim_18_part_external.q,exim_19_00_part_external_location.q,exim_19_part_external_location.q,exim_20_part_managed_location.q,exim_21_export_authsuccess.q,exim_22_import_exist_authsuccess.q,exim_23_import_part_authsuccess.q,exim_24_import_nonexist_authsuccess.q,global_limit.q,groupby_complex_types.q,groupby_complex_types_multi_single_reducer.q,index_auth.q,index_auto.q,index_auto_empty.q,index_bitmap.q,index_bitmap1.q,index_bitmap2.q,index_bitmap3.q,index_bitmap_auto.q,index_bitmap_rc.q,index_compact.q,index_compact_1.q,index_compact_2.q,index_compact_3.q,index_stale_partitioned.q,init_file.q,input16.q,input16_cc.q,input46.q,input_columnarserde.q,input_dynamicserde.q,input_lazyserde.q,input_testxpath3.q,input_testxpath4.q,insert2_overwrite_partitions.q,insertexternal1.q,join_thrift.q,lateral_view.q,load_binary_data.q,load_exist_part_authsuccess.q,load_nonpart_authsuccess.q,load_part_authsuccess.q,loadpart_err.q,lock1.q,lock2.q,lock3.q,lock4.q,merge_dynamic_partition.q,multi_insert.q,multi_insert_move_tasks_share_dependencies.q,null_column.q,ppd_clusterby.q,query_with_semi.q,rename_column.q,sample6.q,sample_islocalmode_hook.q,set_processor_namespaces.q,show_tables.q,source.q,split_sample.q,str_to_map.q,transform1.q,udaf_collect_set.q,udaf_context_ngrams.q,udaf_histogram_numeric.q,udaf_ngrams.q,udaf_percentile_approx.q,udf_array.q,udf_bitmap_and.q,udf_bitmap_or.q,udf_explode.q,udf_format_number.q,udf_map.q,udf_map_keys.q,udf_map_values.q,udf_max.q,udf_min.q,udf_named_struct.q,udf_percentile.q,udf_printf.q,udf_sentences.q,udf_sort_array.q,udf_split.q,udf_struct.q,udf_substr.q,udf_translate.q,udf_union.q,udf_xpath.q,udtf_stack.q,view.q,virtual_column.q diff --git metastore/src/gen/thrift/gen-py/hive_metastore/ThriftHiveMetastore-remote metastore/src/gen/thrift/gen-py/hive_metastore/ThriftHiveMetastore-remote old mode 100644 new mode 100755 diff --git ql/if/queryplan.thrift ql/if/queryplan.thrift index eafbe5a..6026d94 100644 --- ql/if/queryplan.thrift +++ ql/if/queryplan.thrift @@ -56,6 +56,7 @@ enum OperatorType { PTF, MUX, DEMUX, + EVENT, } struct Operator { diff --git ql/src/gen/thrift/gen-cpp/queryplan_types.cpp ql/src/gen/thrift/gen-cpp/queryplan_types.cpp index 96dbb29..b9e04e2 100644 --- ql/src/gen/thrift/gen-cpp/queryplan_types.cpp +++ ql/src/gen/thrift/gen-cpp/queryplan_types.cpp @@ -51,7 +51,8 @@ int _kOperatorTypeValues[] = { OperatorType::HASHTABLEDUMMY, OperatorType::PTF, OperatorType::MUX, - OperatorType::DEMUX + OperatorType::DEMUX, + OperatorType::EVENT }; const char* _kOperatorTypeNames[] = { "JOIN", @@ -74,9 +75,10 @@ const char* _kOperatorTypeNames[] = { "HASHTABLEDUMMY", "PTF", "MUX", - "DEMUX" + "DEMUX", + "EVENT" }; -const std::map _OperatorType_VALUES_TO_NAMES(::apache::thrift::TEnumIterator(21, _kOperatorTypeValues, _kOperatorTypeNames), ::apache::thrift::TEnumIterator(-1, NULL, NULL)); +const std::map _OperatorType_VALUES_TO_NAMES(::apache::thrift::TEnumIterator(22, _kOperatorTypeValues, _kOperatorTypeNames), ::apache::thrift::TEnumIterator(-1, NULL, NULL)); int _kTaskTypeValues[] = { TaskType::MAP, diff --git ql/src/gen/thrift/gen-cpp/queryplan_types.h ql/src/gen/thrift/gen-cpp/queryplan_types.h index 634dd55..30ef711 100644 --- ql/src/gen/thrift/gen-cpp/queryplan_types.h +++ ql/src/gen/thrift/gen-cpp/queryplan_types.h @@ -56,7 +56,8 @@ struct OperatorType { HASHTABLEDUMMY = 17, PTF = 18, MUX = 19, - DEMUX = 20 + DEMUX = 20, + EVENT = 21 }; }; diff --git ql/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/ql/plan/api/OperatorType.java ql/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/ql/plan/api/OperatorType.java index aa094ee..6f23575 100644 --- ql/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/ql/plan/api/OperatorType.java +++ ql/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/ql/plan/api/OperatorType.java @@ -32,7 +32,8 @@ HASHTABLEDUMMY(17), PTF(18), MUX(19), - DEMUX(20); + DEMUX(20), + EVENT(21); private final int value; @@ -95,6 +96,8 @@ public static OperatorType findByValue(int value) { return MUX; case 20: return DEMUX; + case 21: + return EVENT; default: return null; } diff --git ql/src/gen/thrift/gen-php/Types.php ql/src/gen/thrift/gen-php/Types.php index 5164b2c..4ed7fcc 100644 --- ql/src/gen/thrift/gen-php/Types.php +++ ql/src/gen/thrift/gen-php/Types.php @@ -56,6 +56,7 @@ final class OperatorType { const PTF = 18; const MUX = 19; const DEMUX = 20; + const EVENT = 21; static public $__names = array( 0 => 'JOIN', 1 => 'MAPJOIN', @@ -78,6 +79,7 @@ final class OperatorType { 18 => 'PTF', 19 => 'MUX', 20 => 'DEMUX', + 21 => 'EVENT', ); } diff --git ql/src/gen/thrift/gen-py/queryplan/ttypes.py ql/src/gen/thrift/gen-py/queryplan/ttypes.py index 2a3f745..96777fa 100644 --- ql/src/gen/thrift/gen-py/queryplan/ttypes.py +++ ql/src/gen/thrift/gen-py/queryplan/ttypes.py @@ -66,6 +66,7 @@ class OperatorType: PTF = 18 MUX = 19 DEMUX = 20 + EVENT = 21 _VALUES_TO_NAMES = { 0: "JOIN", @@ -89,6 +90,7 @@ class OperatorType: 18: "PTF", 19: "MUX", 20: "DEMUX", + 21: "EVENT", } _NAMES_TO_VALUES = { @@ -113,6 +115,7 @@ class OperatorType: "PTF": 18, "MUX": 19, "DEMUX": 20, + "EVENT": 21, } class TaskType: diff --git ql/src/gen/thrift/gen-rb/queryplan_types.rb ql/src/gen/thrift/gen-rb/queryplan_types.rb index 35e1f0f..449becf 100644 --- ql/src/gen/thrift/gen-rb/queryplan_types.rb +++ ql/src/gen/thrift/gen-rb/queryplan_types.rb @@ -42,8 +42,9 @@ module OperatorType PTF = 18 MUX = 19 DEMUX = 20 - VALUE_MAP = {0 => "JOIN", 1 => "MAPJOIN", 2 => "EXTRACT", 3 => "FILTER", 4 => "FORWARD", 5 => "GROUPBY", 6 => "LIMIT", 7 => "SCRIPT", 8 => "SELECT", 9 => "TABLESCAN", 10 => "FILESINK", 11 => "REDUCESINK", 12 => "UNION", 13 => "UDTF", 14 => "LATERALVIEWJOIN", 15 => "LATERALVIEWFORWARD", 16 => "HASHTABLESINK", 17 => "HASHTABLEDUMMY", 18 => "PTF", 19 => "MUX", 20 => "DEMUX"} - VALID_VALUES = Set.new([JOIN, MAPJOIN, EXTRACT, FILTER, FORWARD, GROUPBY, LIMIT, SCRIPT, SELECT, TABLESCAN, FILESINK, REDUCESINK, UNION, UDTF, LATERALVIEWJOIN, LATERALVIEWFORWARD, HASHTABLESINK, HASHTABLEDUMMY, PTF, MUX, DEMUX]).freeze + EVENT = 21 + VALUE_MAP = {0 => "JOIN", 1 => "MAPJOIN", 2 => "EXTRACT", 3 => "FILTER", 4 => "FORWARD", 5 => "GROUPBY", 6 => "LIMIT", 7 => "SCRIPT", 8 => "SELECT", 9 => "TABLESCAN", 10 => "FILESINK", 11 => "REDUCESINK", 12 => "UNION", 13 => "UDTF", 14 => "LATERALVIEWJOIN", 15 => "LATERALVIEWFORWARD", 16 => "HASHTABLESINK", 17 => "HASHTABLEDUMMY", 18 => "PTF", 19 => "MUX", 20 => "DEMUX", 21 => "EVENT"} + VALID_VALUES = Set.new([JOIN, MAPJOIN, EXTRACT, FILTER, FORWARD, GROUPBY, LIMIT, SCRIPT, SELECT, TABLESCAN, FILESINK, REDUCESINK, UNION, UDTF, LATERALVIEWJOIN, LATERALVIEWFORWARD, HASHTABLESINK, HASHTABLEDUMMY, PTF, MUX, DEMUX, EVENT]).freeze end module TaskType diff --git ql/src/java/org/apache/hadoop/hive/ql/exec/AppMasterEventOperator.java ql/src/java/org/apache/hadoop/hive/ql/exec/AppMasterEventOperator.java new file mode 100644 index 0000000..7315be5 --- /dev/null +++ ql/src/java/org/apache/hadoop/hive/ql/exec/AppMasterEventOperator.java @@ -0,0 +1,145 @@ +/** + * 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; + +import java.io.IOException; +import java.nio.ByteBuffer; +import java.util.Collections; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.hive.conf.HiveConf; +import org.apache.hadoop.hive.conf.HiveConf.ConfVars; +import org.apache.hadoop.hive.ql.exec.tez.TezContext; +import org.apache.hadoop.hive.ql.metadata.HiveException; +import org.apache.hadoop.hive.ql.plan.AppMasterEventDesc; +import org.apache.hadoop.hive.ql.plan.api.OperatorType; +import org.apache.hadoop.hive.serde2.Serializer; +import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspector; +import org.apache.hadoop.io.DataOutputBuffer; +import org.apache.hadoop.io.Writable; +import org.apache.hadoop.util.ReflectionUtils; +import org.apache.tez.runtime.api.Event; +import org.apache.tez.runtime.api.events.InputInitializerEvent; + +/** + * AppMasterEventOperator sends any rows it receives to the Tez AM. This can be + * used to control execution dynamically. + */ +@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; + + @Override + public void initializeOp(Configuration hconf) throws HiveException { + MAX_SIZE = HiveConf.getLongVar(hconf, ConfVars.TEZ_DYNAMIC_PARTITION_PRUNING_MAX_EVENT_SIZE); + serializer = + (Serializer) ReflectionUtils.newInstance(conf.getTable().getDeserializerClass(), null); + initDataBuffer(false); + } + + private void initDataBuffer(boolean skipPruning) throws HiveException { + buffer = new DataOutputBuffer(); + try { + // where does this go to? + buffer.writeUTF(((TezContext) TezContext.get()).getTezProcessorContext().getTaskVertexName()); + + // add any other header info + getConf().writeEventHeader(buffer); + + // write byte to say whether to skip pruning or not + buffer.writeBoolean(skipPruning); + } catch (IOException e) { + throw new HiveException(e); + } + } + + @Override + public void processOp(Object row, int tag) throws HiveException { + if (hasReachedMaxSize) { + return; + } + + ObjectInspector rowInspector = inputObjInspectors[0]; + try { + Writable writableRow = serializer.serialize(row, rowInspector); + writableRow.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); + } + + if (LOG.isDebugEnabled()) { + LOG.debug("AppMasterEvent: " + row); + } + forward(row, rowInspector); + } + + @Override + public void closeOp(boolean abort) throws HiveException { + if (!abort) { + TezContext context = (TezContext) TezContext.get(); + + String vertexName = getConf().getVertexName(); + String inputName = getConf().getInputName(); + + byte[] payload = null; + + if (hasReachedMaxSize) { + initDataBuffer(true); + } + + payload = new byte[buffer.getLength()]; + System.arraycopy(buffer.getData(), 0, payload, 0, buffer.getLength()); + + Event event = + InputInitializerEvent.create(vertexName, inputName, + ByteBuffer.wrap(payload, 0, payload.length)); + + LOG.info("Sending Tez event to vertex = " + vertexName + ", input = " + inputName + + ". Payload size = " + payload.length); + + context.getTezProcessorContext().sendEvents(Collections.singletonList(event)); + } + } + + @Override + public OperatorType getType() { + return OperatorType.EVENT; + } + + /** + * @return the name of the operator + */ + @Override + public String getName() { + return getOperatorName(); + } + + static public String getOperatorName() { + return "EVENT"; + } +} 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 2bcb481..8946221 100644 --- ql/src/java/org/apache/hadoop/hive/ql/exec/OperatorFactory.java +++ ql/src/java/org/apache/hadoop/hive/ql/exec/OperatorFactory.java @@ -29,13 +29,15 @@ import org.apache.hadoop.hive.ql.exec.vector.VectorLimitOperator; import org.apache.hadoop.hive.ql.exec.vector.VectorMapJoinOperator; import org.apache.hadoop.hive.ql.exec.vector.VectorReduceSinkOperator; -import org.apache.hadoop.hive.ql.exec.vector.VectorSelectOperator; import org.apache.hadoop.hive.ql.exec.vector.VectorSMBMapJoinOperator; +import org.apache.hadoop.hive.ql.exec.vector.VectorSelectOperator; import org.apache.hadoop.hive.ql.exec.vector.VectorizationContext; import org.apache.hadoop.hive.ql.metadata.HiveException; +import org.apache.hadoop.hive.ql.plan.AppMasterEventDesc; import org.apache.hadoop.hive.ql.plan.CollectDesc; import org.apache.hadoop.hive.ql.plan.DemuxDesc; import org.apache.hadoop.hive.ql.plan.DummyStoreDesc; +import org.apache.hadoop.hive.ql.plan.DynamicPruningEventDesc; import org.apache.hadoop.hive.ql.plan.ExprNodeDesc; import org.apache.hadoop.hive.ql.plan.ExtractDesc; import org.apache.hadoop.hive.ql.plan.FileSinkDesc; @@ -64,6 +66,7 @@ * OperatorFactory. * */ +@SuppressWarnings({ "rawtypes", "unchecked" }) public final class OperatorFactory { private static final List opvec; private static final List vectorOpvec; @@ -101,6 +104,10 @@ DemuxOperator.class)); opvec.add(new OpTuple(MuxDesc.class, MuxOperator.class)); + opvec.add(new OpTuple(AppMasterEventDesc.class, + AppMasterEventOperator.class)); + opvec.add(new OpTuple(DynamicPruningEventDesc.class, + AppMasterEventOperator.class)); } static { @@ -119,9 +126,9 @@ private static final class OpTuple { private final Class descClass; - private final Class> opClass; + private final Class> opClass; - public OpTuple(Class descClass, Class> opClass) { + public OpTuple(Class descClass, Class> opClass) { this.descClass = descClass; this.opClass = opClass; } diff --git ql/src/java/org/apache/hadoop/hive/ql/exec/Utilities.java ql/src/java/org/apache/hadoop/hive/ql/exec/Utilities.java index 1d6a93a..b3405a7 100644 --- ql/src/java/org/apache/hadoop/hive/ql/exec/Utilities.java +++ ql/src/java/org/apache/hadoop/hive/ql/exec/Utilities.java @@ -818,10 +818,12 @@ public Path read(Kryo kryo, Input input, Class type) { } } - public static Set> cloneOperatorTree(Configuration conf, Set> roots) { + public static List> cloneOperatorTree(Configuration conf, List> roots) { ByteArrayOutputStream baos = new ByteArrayOutputStream(4096); serializePlan(roots, baos, conf, true); - Set> result = deserializePlan(new ByteArrayInputStream(baos.toByteArray()), + @SuppressWarnings("unchecked") + List> result = + deserializePlan(new ByteArrayInputStream(baos.toByteArray()), roots.getClass(), conf, true); return result; } diff --git ql/src/java/org/apache/hadoop/hive/ql/exec/tez/DynamicPartitionPruner.java ql/src/java/org/apache/hadoop/hive/ql/exec/tez/DynamicPartitionPruner.java new file mode 100644 index 0000000..78d6cf5 --- /dev/null +++ ql/src/java/org/apache/hadoop/hive/ql/exec/tez/DynamicPartitionPruner.java @@ -0,0 +1,393 @@ +/** + * 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.tez; + +import java.io.DataInputStream; +import java.io.IOException; +import java.io.InputStream; +import java.nio.ByteBuffer; +import java.util.ArrayList; +import java.util.Collections; +import java.util.HashMap; +import java.util.HashSet; +import java.util.Iterator; +import java.util.List; +import java.util.Map; +import java.util.Set; +import java.util.concurrent.BlockingQueue; +import java.util.concurrent.LinkedBlockingQueue; +import java.util.concurrent.atomic.AtomicBoolean; + +import javolution.testing.AssertionException; + +import org.apache.commons.logging.Log; +import org.apache.commons.logging.LogFactory; +import org.apache.hadoop.hive.ql.exec.ExprNodeEvaluator; +import org.apache.hadoop.hive.ql.exec.ExprNodeEvaluatorFactory; +import org.apache.hadoop.hive.ql.metadata.HiveException; +import org.apache.hadoop.hive.ql.plan.ExprNodeDesc; +import org.apache.hadoop.hive.ql.plan.MapWork; +import org.apache.hadoop.hive.ql.plan.PartitionDesc; +import org.apache.hadoop.hive.ql.plan.TableDesc; +import org.apache.hadoop.hive.serde2.Deserializer; +import org.apache.hadoop.hive.serde2.SerDeException; +import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspector; +import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspectorConverters; +import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspectorConverters.Converter; +import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspectorFactory; +import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspectorUtils; +import org.apache.hadoop.hive.serde2.objectinspector.StructField; +import org.apache.hadoop.hive.serde2.objectinspector.StructObjectInspector; +import org.apache.hadoop.hive.serde2.objectinspector.primitive.PrimitiveObjectInspectorFactory; +import org.apache.hadoop.hive.serde2.typeinfo.TypeInfoFactory; +import org.apache.hadoop.io.BytesWritable; +import org.apache.hadoop.mapred.JobConf; +import org.apache.hadoop.util.ReflectionUtils; +import org.apache.tez.runtime.api.InputInitializerContext; +import org.apache.tez.runtime.api.events.InputInitializerEvent; + +/** + * DynamicPartitionPruner takes a list of assigned partitions at runtime (split + * generation) and prunes them using events generated during execution of the + * dag. + * + */ +public class DynamicPartitionPruner { + + private static final Log LOG = LogFactory.getLog(DynamicPartitionPruner.class); + + private final Map> sourceInfoMap = + new HashMap>(); + + private final BytesWritable writable = new BytesWritable(); + + private final BlockingQueue queue = + new LinkedBlockingQueue(); + + private int sourceInfoCount = 0; + + private InputInitializerContext context; + + public DynamicPartitionPruner() { + } + + public void prune(MapWork work, JobConf jobConf, InputInitializerContext context) + throws SerDeException, IOException, + InterruptedException, HiveException { + + this.context = context; + this.initialize(work, jobConf); + + LOG.info("Waiting for events (" + sourceInfoCount + " items) ..."); + // synchronous event processing loop. Won't return until all events have + // been processed. + this.processEvents(); + this.prunePartitions(work); + LOG.info("Ok to proceed."); + } + + public BlockingQueue getQueue() { + return queue; + } + + private void clear() { + sourceInfoMap.clear(); + sourceInfoCount = 0; + } + + private void initialize(MapWork work, JobConf jobConf) throws SerDeException { + this.clear(); + Map columnMap = new HashMap(); + + for (String s : work.getEventSourceTableDescMap().keySet()) { + List tables = work.getEventSourceTableDescMap().get(s); + List columnNames = work.getEventSourceColumnNameMap().get(s); + List partKeyExprs = work.getEventSourcePartKeyExprMap().get(s); + + Iterator cit = columnNames.iterator(); + Iterator pit = partKeyExprs.iterator(); + for (TableDesc t : tables) { + ++sourceInfoCount; + String columnName = cit.next(); + ExprNodeDesc partKeyExpr = pit.next(); + SourceInfo si = new SourceInfo(t, partKeyExpr, columnName, jobConf); + if (!sourceInfoMap.containsKey(s)) { + sourceInfoMap.put(s, new ArrayList()); + } + List sis = sourceInfoMap.get(s); + sis.add(si); + + // We could have multiple sources restrict the same column, need to take + // the union of the values in that case. + if (columnMap.containsKey(columnName)) { + si.values = columnMap.get(columnName).values; + si.skipPruning = columnMap.get(columnName).skipPruning; + } + columnMap.put(columnName, si); + } + } + } + + private void prunePartitions(MapWork work) throws HiveException { + for (String source : this.sourceInfoMap.keySet()) { + for (SourceInfo si : this.sourceInfoMap.get(source)) { + prunePartitionSingleSource(source, si, work); + } + } + } + + private void prunePartitionSingleSource(String source, SourceInfo si, MapWork work) + throws HiveException { + + if (si.skipPruning.get()) { + // in this case we've determined that there's too much data + // to prune dynamically. + LOG.info("Skip pruning on " + source + ", column " + si.columnName); + return; + } + + Set values = si.values; + String columnName = si.columnName; + + if (LOG.isDebugEnabled()) { + StringBuilder sb = new StringBuilder("Pruning "); + sb.append(columnName); + sb.append(" with "); + for (Object value : values) { + sb.append(value == null ? null : value.toString()); + sb.append(", "); + } + LOG.debug(sb.toString()); + } + + ObjectInspector oi = + PrimitiveObjectInspectorFactory.getPrimitiveWritableObjectInspector(TypeInfoFactory + .getPrimitiveTypeInfo(si.fieldInspector.getTypeName())); + + Converter converter = + ObjectInspectorConverters.getConverter( + PrimitiveObjectInspectorFactory.javaStringObjectInspector, oi); + + StructObjectInspector soi = + ObjectInspectorFactory.getStandardStructObjectInspector( + Collections.singletonList(columnName), Collections.singletonList(oi)); + + @SuppressWarnings("rawtypes") + ExprNodeEvaluator eval = ExprNodeEvaluatorFactory.get(si.partKey); + eval.initialize(soi); + + applyFilterToPartitions(work, converter, eval, columnName, values); + } + + @SuppressWarnings("rawtypes") + private void applyFilterToPartitions(MapWork work, Converter converter, ExprNodeEvaluator eval, + String columnName, Set values) throws HiveException { + + Object[] row = new Object[1]; + + Iterator it = work.getPathToPartitionInfo().keySet().iterator(); + while (it.hasNext()) { + String p = it.next(); + PartitionDesc desc = work.getPathToPartitionInfo().get(p); + Map spec = desc.getPartSpec(); + if (spec == null) { + throw new AssertionException("No partition spec found in dynamic pruning"); + } + + String partValueString = spec.get(columnName); + if (partValueString == null) { + throw new AssertionException("Could not find partition value for column: " + columnName); + } + + Object partValue = converter.convert(partValueString); + if (LOG.isDebugEnabled()) { + LOG.debug("Converted partition value: " + partValue + " original (" + partValueString + ")"); + } + + row[0] = partValue; + partValue = eval.evaluate(row); + if (LOG.isDebugEnabled()) { + LOG.debug("part key expr applied: " + partValue); + } + + if (!values.contains(partValue)) { + LOG.info("Pruning path: " + p); + it.remove(); + work.getPathToAliases().remove(p); + work.getPaths().remove(p); + work.getPartitionDescs().remove(desc); + } + } + } + + @SuppressWarnings("deprecation") + private static class SourceInfo { + public final ExprNodeDesc partKey; + public final Deserializer deserializer; + public final StructObjectInspector soi; + public final StructField field; + public final ObjectInspector fieldInspector; + public Set values = new HashSet(); + public AtomicBoolean skipPruning = new AtomicBoolean(); + public final String columnName; + + public SourceInfo(TableDesc table, ExprNodeDesc partKey, String columnName, JobConf jobConf) + throws SerDeException { + + this.skipPruning.set(false); + + this.partKey = partKey; + + this.columnName = columnName; + + deserializer = ReflectionUtils.newInstance(table.getDeserializerClass(), null); + deserializer.initialize(jobConf, table.getProperties()); + + ObjectInspector inspector = deserializer.getObjectInspector(); + LOG.debug("Type of obj insp: " + inspector.getTypeName()); + + soi = (StructObjectInspector) inspector; + List fields = soi.getAllStructFieldRefs(); + if (fields.size() > 1) { + LOG.error("expecting single field in input"); + } + + field = fields.get(0); + + fieldInspector = + ObjectInspectorUtils.getStandardObjectInspector(field.getFieldObjectInspector()); + } + } + + private void processEvents() throws SerDeException, IOException, InterruptedException { + int eventCount = 0; + int neededEvents = getExpectedNumberOfEvents(); + + while (neededEvents > eventCount) { + InputInitializerEvent event = queue.take(); + LOG.info("Input event: " + event.getTargetInputName() + ", " + event.getTargetVertexName() + + ", " + (event.getUserPayload().limit() - event.getUserPayload().position())); + processPayload(event.getUserPayload()); + eventCount += 1; + neededEvents = getExpectedNumberOfEvents(); + LOG.info("Needed events: " + neededEvents + ", received events: " + eventCount); + } + } + + private int getExpectedNumberOfEvents() throws InterruptedException { + int neededEvents = 0; + + boolean notInitialized; + do { + neededEvents = 0; + notInitialized = false; + for (String s : sourceInfoMap.keySet()) { + int multiplier = sourceInfoMap.get(s).size(); + int taskNum = context.getVertexNumTasks(s); + LOG.info("Vertex " + s + " has " + taskNum + " events."); + if (taskNum < 0) { + notInitialized = true; + Thread.sleep(10); + continue; + } + neededEvents += (taskNum * multiplier); + } + } while (notInitialized); + + return neededEvents; + } + + @SuppressWarnings("deprecation") + private String processPayload(ByteBuffer payload) throws SerDeException, IOException { + DataInputStream in = new DataInputStream(new ByteBufferBackedInputStream(payload)); + String sourceName = in.readUTF(); + String columnName = in.readUTF(); + boolean skip = in.readBoolean(); + + LOG.info("Source of event: " + sourceName); + + List infos = this.sourceInfoMap.get(sourceName); + if (infos == null) { + in.close(); + throw new AssertionException("no source info for event source: " + sourceName); + } + + SourceInfo info = null; + for (SourceInfo si : infos) { + if (columnName.equals(si.columnName)) { + info = si; + break; + } + } + + if (info == null) { + in.close(); + throw new AssertionException("no source info for column: " + columnName); + } + + if (skip) { + info.skipPruning.set(true); + } + + while (payload.hasRemaining()) { + writable.readFields(in); + + Object row = info.deserializer.deserialize(writable); + + Object value = info.soi.getStructFieldData(row, info.field); + value = ObjectInspectorUtils.copyToStandardObject(value, info.fieldInspector); + + if (LOG.isDebugEnabled()) { + LOG.debug("Adding: " + value + " to list of required partitions"); + } + info.values.add(value); + } + in.close(); + return sourceName; + } + + private static class ByteBufferBackedInputStream extends InputStream { + + ByteBuffer buf; + + public ByteBufferBackedInputStream(ByteBuffer buf) { + this.buf = buf; + } + + @Override + public int read() throws IOException { + if (!buf.hasRemaining()) { + return -1; + } + return buf.get() & 0xFF; + } + + @Override + public int read(byte[] bytes, int off, int len) throws IOException { + if (!buf.hasRemaining()) { + return -1; + } + + len = Math.min(len, buf.remaining()); + buf.get(bytes, off, len); + return len; + } + } + +} diff --git ql/src/java/org/apache/hadoop/hive/ql/exec/tez/HiveSplitGenerator.java ql/src/java/org/apache/hadoop/hive/ql/exec/tez/HiveSplitGenerator.java index d540e27..3df73ee 100644 --- ql/src/java/org/apache/hadoop/hive/ql/exec/tez/HiveSplitGenerator.java +++ ql/src/java/org/apache/hadoop/hive/ql/exec/tez/HiveSplitGenerator.java @@ -46,12 +46,12 @@ import org.apache.tez.mapreduce.protos.MRRuntimeProtos.MRSplitProto; import org.apache.tez.mapreduce.protos.MRRuntimeProtos.MRSplitsProto; import org.apache.tez.runtime.api.Event; -import org.apache.tez.runtime.api.events.InputConfigureVertexTasksEvent; -import org.apache.tez.runtime.api.events.InputDataInformationEvent; -import org.apache.tez.runtime.api.events.InputInitializerEvent; import org.apache.tez.runtime.api.InputInitializer; import org.apache.tez.runtime.api.InputInitializerContext; import org.apache.tez.runtime.api.InputSpecUpdate; +import org.apache.tez.runtime.api.events.InputConfigureVertexTasksEvent; +import org.apache.tez.runtime.api.events.InputDataInformationEvent; +import org.apache.tez.runtime.api.events.InputInitializerEvent; import com.google.common.collect.ArrayListMultimap; import com.google.common.collect.Lists; @@ -63,11 +63,14 @@ * making sure that splits from different partitions are only grouped if they * are of the same schema, format and serde */ +@SuppressWarnings("deprecation") public class HiveSplitGenerator extends InputInitializer { private static final Log LOG = LogFactory.getLog(HiveSplitGenerator.class); private static final SplitGrouper grouper = new SplitGrouper(); + private final DynamicPartitionPruner pruner = new DynamicPartitionPruner(); + private InputInitializerContext context; public HiveSplitGenerator(InputInitializerContext initializerContext) { super(initializerContext); @@ -77,6 +80,8 @@ public HiveSplitGenerator(InputInitializerContext initializerContext) { public List initialize() throws Exception { InputInitializerContext rootInputContext = getContext(); + context = rootInputContext; + MRInputUserPayloadProto userPayloadProto = MRInputHelpers.parseMRInputPayload(rootInputContext.getInputUserPayload()); @@ -90,6 +95,11 @@ public HiveSplitGenerator(InputInitializerContext initializerContext) { JobConf jobConf = new JobConf(conf); ShimLoader.getHadoopShims().getMergedCredentials(jobConf); + MapWork work = Utilities.getMapWork(jobConf); + + // perform dynamic partition pruning + pruner.prune(work, jobConf, context); + InputSplitInfoMem inputSplitInfo = null; String realInputFormatName = conf.get("mapred.input.format.class"); boolean groupingEnabled = userPayloadProto.getGroupingEnabled(); @@ -138,6 +148,7 @@ public HiveSplitGenerator(InputInitializerContext initializerContext) { return createEventList(sendSerializedEvents, inputSplitInfo); } + public static Multimap generateGroupedSplits(JobConf jobConf, Configuration conf, InputSplit[] splits, float waves, int availableSlots) throws Exception { @@ -189,10 +200,6 @@ public HiveSplitGenerator(InputInitializerContext initializerContext) { return groupedSplits; } - @Override - public void handleInputInitializerEvent(List events) throws Exception { - } - private List createEventList(boolean sendSerializedEvents, InputSplitInfoMem inputSplitInfo) { List events = Lists.newArrayListWithCapacity(inputSplitInfo.getNumTasks() + 1); @@ -221,4 +228,11 @@ public void handleInputInitializerEvent(List events) thro } return events; } + + @Override + public void handleInputInitializerEvent(List events) throws Exception { + for (InputInitializerEvent e : events) { + pruner.getQueue().put(e); + } + } } diff --git ql/src/java/org/apache/hadoop/hive/ql/exec/tez/MapRecordProcessor.java ql/src/java/org/apache/hadoop/hive/ql/exec/tez/MapRecordProcessor.java index 7556d7b..37b7bbd 100644 --- ql/src/java/org/apache/hadoop/hive/ql/exec/tez/MapRecordProcessor.java +++ ql/src/java/org/apache/hadoop/hive/ql/exec/tez/MapRecordProcessor.java @@ -117,7 +117,8 @@ void init(JobConf jconf, ProcessorContext processorContext, MRTaskReporter mrRep l4j.info(mapOp.dump(0)); MapredContext.init(true, new JobConf(jconf)); - ((TezContext)MapredContext.get()).setInputs(inputs); + ((TezContext) MapredContext.get()).setInputs(inputs); + ((TezContext) MapredContext.get()).setTezProcessorContext(processorContext); mapOp.setExecContext(execContext); mapOp.initializeLocalWork(jconf); mapOp.initialize(jconf, null); diff --git ql/src/java/org/apache/hadoop/hive/ql/exec/tez/ReduceRecordProcessor.java ql/src/java/org/apache/hadoop/hive/ql/exec/tez/ReduceRecordProcessor.java index e278572..990a4f1 100644 --- ql/src/java/org/apache/hadoop/hive/ql/exec/tez/ReduceRecordProcessor.java +++ ql/src/java/org/apache/hadoop/hive/ql/exec/tez/ReduceRecordProcessor.java @@ -138,7 +138,7 @@ void init(JobConf jconf, ProcessorContext processorContext, MRTaskReporter mrRep try { keyTableDesc = redWork.getKeyDesc(); - inputKeyDeserializer = (SerDe) ReflectionUtils.newInstance(keyTableDesc + inputKeyDeserializer = ReflectionUtils.newInstance(keyTableDesc .getDeserializerClass(), null); SerDeUtils.initializeSerDe(inputKeyDeserializer, null, keyTableDesc.getProperties(), null); keyObjectInspector = inputKeyDeserializer.getObjectInspector(); @@ -150,7 +150,7 @@ void init(JobConf jconf, ProcessorContext processorContext, MRTaskReporter mrRep keyStructInspector = (StructObjectInspector)keyObjectInspector; batches = new VectorizedRowBatch[maxTags]; valueStructInspectors = new StructObjectInspector[maxTags]; - valueStringWriters = (List[])new List[maxTags]; + valueStringWriters = new List[maxTags]; keysColumnOffset = keyStructInspector.getAllStructFieldRefs().size(); buffer = new DataOutputBuffer(); } @@ -213,7 +213,8 @@ void init(JobConf jconf, ProcessorContext processorContext, MRTaskReporter mrRep } MapredContext.init(false, new JobConf(jconf)); - ((TezContext)MapredContext.get()).setInputs(inputs); + ((TezContext) MapredContext.get()).setInputs(inputs); + ((TezContext) MapredContext.get()).setTezProcessorContext(processorContext); // initialize reduce operator tree try { @@ -304,7 +305,7 @@ void run() throws Exception { Map tag2input = redWork.getTagToInput(); ArrayList shuffleInputs = new ArrayList(); for(String inpStr : tag2input.values()){ - shuffleInputs.add((LogicalInput)inputs.get(inpStr)); + shuffleInputs.add(inputs.get(inpStr)); } return shuffleInputs; } diff --git ql/src/java/org/apache/hadoop/hive/ql/exec/tez/TezContext.java ql/src/java/org/apache/hadoop/hive/ql/exec/tez/TezContext.java index 01d68ab..010a6f4 100644 --- ql/src/java/org/apache/hadoop/hive/ql/exec/tez/TezContext.java +++ ql/src/java/org/apache/hadoop/hive/ql/exec/tez/TezContext.java @@ -23,6 +23,7 @@ import org.apache.hadoop.mapred.JobConf; import org.apache.tez.runtime.api.LogicalInput; import org.apache.tez.runtime.api.LogicalOutput; +import org.apache.tez.runtime.api.ProcessorContext; /** * TezContext contains additional context only available with Tez @@ -31,9 +32,11 @@ // all the inputs for the tez processor private Map inputs; - + private Map outputs; + private ProcessorContext processorContext; + public TezContext(boolean isMap, JobConf jobConf) { super(isMap, jobConf); } @@ -41,7 +44,7 @@ public TezContext(boolean isMap, JobConf jobConf) { public void setInputs(Map inputs) { this.inputs = inputs; } - + public void setOutputs(Map outputs) { this.outputs = outputs; } @@ -52,11 +55,19 @@ public LogicalInput getInput(String name) { } return inputs.get(name); } - + public LogicalOutput getOutput(String name) { if (outputs == null) { return null; } return outputs.get(name); } + + public void setTezProcessorContext(ProcessorContext processorContext) { + this.processorContext = processorContext; + } + + public ProcessorContext getTezProcessorContext() { + return processorContext; + } } diff --git ql/src/java/org/apache/hadoop/hive/ql/optimizer/ConstantPropagateProcFactory.java ql/src/java/org/apache/hadoop/hive/ql/optimizer/ConstantPropagateProcFactory.java index b15aedc..36c5757 100644 --- ql/src/java/org/apache/hadoop/hive/ql/optimizer/ConstantPropagateProcFactory.java +++ ql/src/java/org/apache/hadoop/hive/ql/optimizer/ConstantPropagateProcFactory.java @@ -941,12 +941,12 @@ public Object process(Node nd, Stack stack, NodeProcessorCtx ctx, Object.. return null; } - List newChildren = new ArrayList(); - for (ExprNodeDesc expr : pred.getChildren()) { - ExprNodeDesc constant = foldExpr(expr, constants, cppCtx, op, 0, false); - newChildren.add(constant); + ExprNodeDesc constant = foldExpr(pred, constants, cppCtx, op, 0, false); + if (constant instanceof ExprNodeGenericFuncDesc) { + conf.setFilterExpr((ExprNodeGenericFuncDesc) constant); + } else { + conf.setFilterExpr(null); } - pred.setChildren(newChildren); return null; } } diff --git ql/src/java/org/apache/hadoop/hive/ql/optimizer/ConvertJoinMapJoin.java ql/src/java/org/apache/hadoop/hive/ql/optimizer/ConvertJoinMapJoin.java index d42e1f7..8e9d3cc 100644 --- ql/src/java/org/apache/hadoop/hive/ql/optimizer/ConvertJoinMapJoin.java +++ ql/src/java/org/apache/hadoop/hive/ql/optimizer/ConvertJoinMapJoin.java @@ -19,6 +19,7 @@ package org.apache.hadoop.hive.ql.optimizer; import java.util.HashMap; +import java.util.HashSet; import java.util.List; import java.util.Map; import java.util.Set; @@ -27,6 +28,7 @@ import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; import org.apache.hadoop.hive.conf.HiveConf; +import org.apache.hadoop.hive.ql.exec.AppMasterEventOperator; import org.apache.hadoop.hive.ql.exec.GroupByOperator; import org.apache.hadoop.hive.ql.exec.JoinOperator; import org.apache.hadoop.hive.ql.exec.MapJoinOperator; @@ -39,6 +41,7 @@ import org.apache.hadoop.hive.ql.parse.OptimizeTezProcContext; import org.apache.hadoop.hive.ql.parse.ParseContext; import org.apache.hadoop.hive.ql.parse.SemanticException; +import org.apache.hadoop.hive.ql.plan.DynamicPruningEventDesc; import org.apache.hadoop.hive.ql.plan.ExprNodeColumnDesc; import org.apache.hadoop.hive.ql.plan.ExprNodeDesc; import org.apache.hadoop.hive.ql.plan.MapJoinDesc; @@ -363,6 +366,19 @@ public MapJoinOperator convertJoinMapJoin(JoinOperator joinOp, OptimizeTezProcCo Operator parentBigTableOp = mapJoinOp.getParentOperators().get(bigTablePosition); if (parentBigTableOp instanceof ReduceSinkOperator) { + for (Operator p : parentBigTableOp.getParentOperators()) { + // we might have generated a dynamic partition operator chain. Since + // we're removing the reduce sink we need do remove that too. + Set> dynamicPartitionOperators = new HashSet>(); + for (Operator c : p.getChildOperators()) { + if (hasDynamicPartitionBroadcast(c)) { + dynamicPartitionOperators.add(c); + } + } + for (Operator c : dynamicPartitionOperators) { + p.removeChild(c); + } + } mapJoinOp.getParentOperators().remove(bigTablePosition); if (!(mapJoinOp.getParentOperators().contains( parentBigTableOp.getParentOperators().get(0)))) { @@ -380,4 +396,16 @@ public MapJoinOperator convertJoinMapJoin(JoinOperator joinOp, OptimizeTezProcCo return mapJoinOp; } + + private boolean hasDynamicPartitionBroadcast(Operator op) { + if (op instanceof AppMasterEventOperator && op.getConf() instanceof DynamicPruningEventDesc) { + return true; + } + for (Operator c : op.getChildOperators()) { + if (hasDynamicPartitionBroadcast(c)) { + return true; + } + } + return false; + } } diff --git ql/src/java/org/apache/hadoop/hive/ql/optimizer/DynamicPartitionPruningOptimization.java ql/src/java/org/apache/hadoop/hive/ql/optimizer/DynamicPartitionPruningOptimization.java new file mode 100644 index 0000000..7492f52 --- /dev/null +++ ql/src/java/org/apache/hadoop/hive/ql/optimizer/DynamicPartitionPruningOptimization.java @@ -0,0 +1,343 @@ +/** + * 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.optimizer; + +import java.util.ArrayList; +import java.util.HashMap; +import java.util.Iterator; +import java.util.LinkedHashMap; +import java.util.List; +import java.util.Map; +import java.util.Stack; + +import org.apache.commons.logging.Log; +import org.apache.commons.logging.LogFactory; +import org.apache.hadoop.hive.conf.HiveConf; +import org.apache.hadoop.hive.conf.HiveConf.ConfVars; +import org.apache.hadoop.hive.ql.exec.FilterOperator; +import org.apache.hadoop.hive.ql.exec.GroupByOperator; +import org.apache.hadoop.hive.ql.exec.Operator; +import org.apache.hadoop.hive.ql.exec.OperatorFactory; +import org.apache.hadoop.hive.ql.exec.ReduceSinkOperator; +import org.apache.hadoop.hive.ql.exec.SelectOperator; +import org.apache.hadoop.hive.ql.exec.TableScanOperator; +import org.apache.hadoop.hive.ql.lib.DefaultGraphWalker; +import org.apache.hadoop.hive.ql.lib.DefaultRuleDispatcher; +import org.apache.hadoop.hive.ql.lib.Dispatcher; +import org.apache.hadoop.hive.ql.lib.GraphWalker; +import org.apache.hadoop.hive.ql.lib.Node; +import org.apache.hadoop.hive.ql.lib.NodeProcessor; +import org.apache.hadoop.hive.ql.lib.NodeProcessorCtx; +import org.apache.hadoop.hive.ql.lib.Rule; +import org.apache.hadoop.hive.ql.lib.RuleRegExp; +import org.apache.hadoop.hive.ql.metadata.Partition; +import org.apache.hadoop.hive.ql.metadata.Table; +import org.apache.hadoop.hive.ql.parse.OptimizeTezProcContext; +import org.apache.hadoop.hive.ql.parse.ParseContext; +import org.apache.hadoop.hive.ql.parse.PrunedPartitionList; +import org.apache.hadoop.hive.ql.parse.SemanticException; +import org.apache.hadoop.hive.ql.plan.AggregationDesc; +import org.apache.hadoop.hive.ql.plan.DynamicPruningEventDesc; +import org.apache.hadoop.hive.ql.plan.ExprNodeColumnDesc; +import org.apache.hadoop.hive.ql.plan.ExprNodeConstantDesc; +import org.apache.hadoop.hive.ql.plan.ExprNodeDesc; +import org.apache.hadoop.hive.ql.plan.ExprNodeDynamicListDesc; +import org.apache.hadoop.hive.ql.plan.FilterDesc; +import org.apache.hadoop.hive.ql.plan.GroupByDesc; +import org.apache.hadoop.hive.ql.plan.OperatorDesc; +import org.apache.hadoop.hive.ql.plan.PlanUtils; +import org.apache.hadoop.hive.ql.plan.SelectDesc; + +/** + * This optimization looks for expressions of the kind "x IN (RS[n])". If such + * an expression made it to a table scan operator and x is a partition column we + * can use an existing join to dynamically prune partitions. This class sets up + * the infrastructure for that. + */ +public class DynamicPartitionPruningOptimization implements NodeProcessor { + + static final private Log LOG = LogFactory.getLog(DynamicPartitionPruningOptimization.class + .getName()); + + public static class DynamicPartitionPrunerProc implements NodeProcessor { + + /** + * process simply remembers all the dynamic partition pruning expressions + * found + */ + @Override + public Object process(Node nd, Stack stack, NodeProcessorCtx procCtx, + Object... nodeOutputs) throws SemanticException { + ExprNodeDynamicListDesc desc = (ExprNodeDynamicListDesc) nd; + DynamicPartitionPrunerContext context = (DynamicPartitionPrunerContext) procCtx; + + // Rule is searching for dynamic pruning expr. There's at least an IN + // expression wrapping it. + ExprNodeDesc parent = (ExprNodeDesc) stack.get(stack.size() - 2); + ExprNodeDesc grandParent = stack.size() >= 3 ? (ExprNodeDesc) stack.get(stack.size() - 3) : null; + + context.addDynamicList(desc, parent, grandParent, (ReduceSinkOperator) desc.getSource()); + + return context; + } + } + + private static class DynamicListContext { + public ExprNodeDynamicListDesc desc; + public ExprNodeDesc parent; + public ExprNodeDesc grandParent; + public ReduceSinkOperator generator; + + public DynamicListContext(ExprNodeDynamicListDesc desc, ExprNodeDesc parent, + ExprNodeDesc grandParent, ReduceSinkOperator generator) { + this.desc = desc; + this.parent = parent; + this.grandParent = grandParent; + this.generator = generator; + } + } + + private static class DynamicPartitionPrunerContext implements NodeProcessorCtx, + Iterable { + public List dynLists = new ArrayList(); + + public void addDynamicList(ExprNodeDynamicListDesc desc, ExprNodeDesc parent, + ExprNodeDesc grandParent, ReduceSinkOperator generator) { + dynLists.add(new DynamicListContext(desc, parent, grandParent, generator)); + } + + @Override + public Iterator iterator() { + return dynLists.iterator(); + } + } + + private String extractColName(ExprNodeDesc root) { + if (root instanceof ExprNodeColumnDesc) { + return ((ExprNodeColumnDesc) root).getColumn(); + } else { + if (root.getChildren() == null) { + return null; + } + + String column = null; + for (ExprNodeDesc d: root.getChildren()) { + String candidate = extractColName(d); + if (column != null && candidate != null) { + return null; + } else if (candidate != null) { + column = candidate; + } + } + return column; + } + } + + @Override + public Object process(Node nd, Stack stack, NodeProcessorCtx procCtx, Object... nodeOutputs) + throws SemanticException { + OptimizeTezProcContext context = (OptimizeTezProcContext) procCtx; + ParseContext parseContext = context.parseContext; + + FilterOperator filter = (FilterOperator) nd; + FilterDesc desc = filter.getConf(); + + TableScanOperator ts = null; + + if (!parseContext.getConf().getBoolVar(ConfVars.TEZ_DYNAMIC_PARTITION_PRUNING)) { + // nothing to do when the optimization is off + return null; + } + + DynamicPartitionPrunerContext removerContext = new DynamicPartitionPrunerContext(); + + if (filter.getParentOperators().size() == 1 + && filter.getParentOperators().get(0) instanceof TableScanOperator) { + ts = (TableScanOperator) filter.getParentOperators().get(0); + } + + if (LOG.isDebugEnabled()) { + LOG.debug("Parent: " + filter.getParentOperators().get(0)); + LOG.debug("Filter: " + desc.getPredicateString()); + LOG.debug("TableScan: " + ts); + } + + // collect the dynamic pruning conditions + removerContext.dynLists.clear(); + walkExprTree(desc.getPredicate(), removerContext); + + for (DynamicListContext ctx : removerContext) { + String column = extractColName(ctx.parent); + + if (ts != null && column != null) { + Table table = parseContext.getTopToTable().get(ts); + + if (table != null && table.isPartitionKey(column)) { + String alias = ts.getConf().getAlias(); + PrunedPartitionList plist = parseContext.getPrunedPartitions(alias, ts); + if (LOG.isDebugEnabled()) { + LOG.debug("alias: " + alias); + LOG.debug("pruned partition list: "); + if (plist != null) { + for (Partition p : plist.getPartitions()) { + LOG.debug(p.getCompleteName()); + } + } + } + if (plist == null || plist.getPartitions().size() != 0) { + LOG.info("Dynamic partitioning: " + table.getCompleteName() + "." + column); + generateEventOperatorPlan(ctx, parseContext, ts, column); + } else { + // all partitions have been statically removed + LOG.debug("No partition pruning necessary."); + } + } else { + LOG.debug("Column " + column + " is not a partition column"); + } + } + + // we always remove the condition by replacing it with "true" + ExprNodeDesc constNode = new ExprNodeConstantDesc(ctx.parent.getTypeInfo(), true); + if (ctx.grandParent == null) { + desc.setPredicate(constNode); + } else { + int i = ctx.grandParent.getChildren().indexOf(ctx.parent); + ctx.grandParent.getChildren().remove(i); + ctx.grandParent.getChildren().add(i, constNode); + } + } + + // if we pushed the predicate into the table scan we need to remove the + // synthetic conditions there. + cleanTableScanFilters(ts); + + return false; + } + + private void cleanTableScanFilters(TableScanOperator ts) throws SemanticException { + + if (ts == null || ts.getConf() == null || ts.getConf().getFilterExpr() == null) { + // nothing to do + return; + } + + DynamicPartitionPrunerContext removerContext = new DynamicPartitionPrunerContext(); + + // collect the dynamic pruning conditions + removerContext.dynLists.clear(); + walkExprTree(ts.getConf().getFilterExpr(), removerContext); + + for (DynamicListContext ctx : removerContext) { + // remove the condition by replacing it with "true" + ExprNodeDesc constNode = new ExprNodeConstantDesc(ctx.parent.getTypeInfo(), true); + if (ctx.grandParent == null) { + // we're the only node, just clear out the expression + ts.getConf().setFilterExpr(null); + } else { + int i = ctx.grandParent.getChildren().indexOf(ctx.parent); + ctx.grandParent.getChildren().remove(i); + ctx.grandParent.getChildren().add(i, constNode); + } + } + } + + private void generateEventOperatorPlan(DynamicListContext ctx, ParseContext parseContext, + TableScanOperator ts, String column) { + + // we will put a fork in the plan at the source of the reduce sink + Operator parentOfRS = ctx.generator.getParentOperators().get(0); + + // we need the expr that generated the key of the reduce sink + ExprNodeDesc key = ctx.generator.getConf().getKeyCols().get(ctx.desc.getKeyIndex()); + + // we also need the expr for the partitioned table + ExprNodeDesc partKey = ctx.parent.getChildren().get(0); + + if (LOG.isDebugEnabled()) { + LOG.debug("key expr: " + key); + LOG.debug("partition key expr: " + partKey); + } + + List keyExprs = new ArrayList(); + keyExprs.add(key); + + // group by requires "ArrayList", don't ask. + ArrayList outputNames = new ArrayList(); + outputNames.add(HiveConf.getColumnInternalName(0)); + + // project the relevant key column + SelectDesc select = new SelectDesc(keyExprs, outputNames); + SelectOperator selectOp = + (SelectOperator) OperatorFactory.getAndMakeChild(select, parentOfRS); + + // do a group by on the list to dedup + float groupByMemoryUsage = + HiveConf.getFloatVar(parseContext.getConf(), HiveConf.ConfVars.HIVEMAPAGGRHASHMEMORY); + float memoryThreshold = + HiveConf.getFloatVar(parseContext.getConf(), + HiveConf.ConfVars.HIVEMAPAGGRMEMORYTHRESHOLD); + + ArrayList groupByExprs = new ArrayList(); + ExprNodeDesc groupByExpr = + new ExprNodeColumnDesc(key.getTypeInfo(), outputNames.get(0), null, false); + groupByExprs.add(groupByExpr); + + GroupByDesc groupBy = + new GroupByDesc(GroupByDesc.Mode.HASH, outputNames, groupByExprs, + new ArrayList(), false, groupByMemoryUsage, memoryThreshold, + null, false, 0, true); + + GroupByOperator groupByOp = + (GroupByOperator) OperatorFactory.getAndMakeChild(groupBy, selectOp); + + // finally add the event broadcast operator + DynamicPruningEventDesc eventDesc = new DynamicPruningEventDesc(); + eventDesc.setTableScan(ts); + eventDesc.setTable(PlanUtils.getReduceValueTableDesc(PlanUtils + .getFieldSchemasFromColumnList(keyExprs, "key"))); + eventDesc.setTargetColumnName(column); + eventDesc.setPartKey(partKey); + + OperatorFactory.getAndMakeChild(eventDesc, groupByOp); + } + + private Map walkExprTree(ExprNodeDesc pred, NodeProcessorCtx ctx) + throws SemanticException { + + // create a walker which walks the tree in a DFS manner while maintaining + // the operator stack. The dispatcher + // generates the plan from the operator tree + Map exprRules = new LinkedHashMap(); + exprRules.put(new RuleRegExp("R1", ExprNodeDynamicListDesc.class.getName() + "%"), + new DynamicPartitionPrunerProc()); + + // The dispatcher fires the processor corresponding to the closest matching + // rule and passes the context along + Dispatcher disp = new DefaultRuleDispatcher(null, exprRules, ctx); + GraphWalker egw = new DefaultGraphWalker(disp); + + List startNodes = new ArrayList(); + startNodes.add(pred); + + HashMap outputMap = new HashMap(); + egw.startWalking(startNodes, outputMap); + return outputMap; + } + +} diff --git ql/src/java/org/apache/hadoop/hive/ql/optimizer/Optimizer.java ql/src/java/org/apache/hadoop/hive/ql/optimizer/Optimizer.java index d7e1fbf..20ef64d 100644 --- ql/src/java/org/apache/hadoop/hive/ql/optimizer/Optimizer.java +++ ql/src/java/org/apache/hadoop/hive/ql/optimizer/Optimizer.java @@ -36,6 +36,7 @@ import org.apache.hadoop.hive.ql.parse.SemanticException; import org.apache.hadoop.hive.ql.ppd.PredicatePushDown; import org.apache.hadoop.hive.ql.ppd.PredicateTransitivePropagate; +import org.apache.hadoop.hive.ql.ppd.SyntheticJoinPredicate; /** * Implementation of the optimizer. @@ -55,6 +56,7 @@ public void initialize(HiveConf hiveConf) { transformations.add(new Generator()); if (HiveConf.getBoolVar(hiveConf, HiveConf.ConfVars.HIVEOPTPPD)) { transformations.add(new PredicateTransitivePropagate()); + transformations.add(new SyntheticJoinPredicate()); transformations.add(new PredicatePushDown()); transformations.add(new PartitionPruner()); transformations.add(new PartitionConditionRemover()); @@ -125,8 +127,8 @@ public void initialize(HiveConf hiveConf) { if(HiveConf.getBoolVar(hiveConf, HiveConf.ConfVars.HIVEOPTIMIZEMETADATAQUERIES)) { transformations.add(new StatsOptimizer()); } - if (pctx.getContext().getExplain() || - HiveConf.getVar(hiveConf, HiveConf.ConfVars.HIVE_EXECUTION_ENGINE).equals("tez")) { + if (pctx.getContext().getExplain() + || !HiveConf.getVar(hiveConf, HiveConf.ConfVars.HIVE_EXECUTION_ENGINE).equals("tez")) { transformations.add(new AnnotateWithStatistics()); transformations.add(new AnnotateWithOpTraits()); } diff --git ql/src/java/org/apache/hadoop/hive/ql/optimizer/PrunerExpressionOperatorFactory.java ql/src/java/org/apache/hadoop/hive/ql/optimizer/PrunerExpressionOperatorFactory.java index b639a2a..e633fdc 100644 --- ql/src/java/org/apache/hadoop/hive/ql/optimizer/PrunerExpressionOperatorFactory.java +++ ql/src/java/org/apache/hadoop/hive/ql/optimizer/PrunerExpressionOperatorFactory.java @@ -186,8 +186,7 @@ public Object process(Node nd, Stack stack, NodeProcessorCtx procCtx, return ((ExprNodeNullDesc) nd).clone(); } - assert (false); - return null; + return new ExprNodeConstantDesc(((ExprNodeDesc)nd).getTypeInfo(), null); } } diff --git ql/src/java/org/apache/hadoop/hive/ql/optimizer/ReduceSinkMapJoinProc.java ql/src/java/org/apache/hadoop/hive/ql/optimizer/ReduceSinkMapJoinProc.java index 9eb81aa..eeef609 100644 --- ql/src/java/org/apache/hadoop/hive/ql/optimizer/ReduceSinkMapJoinProc.java +++ ql/src/java/org/apache/hadoop/hive/ql/optimizer/ReduceSinkMapJoinProc.java @@ -26,8 +26,6 @@ import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; -import org.apache.hadoop.hive.metastore.api.ColumnStatistics; -import org.apache.hadoop.hive.ql.exec.FileSinkOperator; import org.apache.hadoop.hive.ql.exec.HashTableDummyOperator; import org.apache.hadoop.hive.ql.exec.MapJoinOperator; import org.apache.hadoop.hive.ql.exec.Operator; @@ -42,7 +40,6 @@ import org.apache.hadoop.hive.ql.parse.SemanticException; import org.apache.hadoop.hive.ql.plan.BaseWork; import org.apache.hadoop.hive.ql.plan.ColStatistics; -import org.apache.hadoop.hive.ql.plan.ExprNodeColumnDesc; import org.apache.hadoop.hive.ql.plan.ExprNodeDesc; import org.apache.hadoop.hive.ql.plan.HashTableDummyDesc; import org.apache.hadoop.hive.ql.plan.MapJoinDesc; @@ -134,7 +131,8 @@ public Object process(Node nd, Stack stack, NodeProcessorCtx procContext, String prefix = Utilities.ReduceField.KEY.toString(); for (String keyCol : keyCols) { ExprNodeDesc realCol = parentRS.getColumnExprMap().get(prefix + "." + keyCol); - ColStatistics cs = StatsUtils.getColStatisticsFromExpression(null, stats, realCol); + ColStatistics cs = + StatsUtils.getColStatisticsFromExpression(context.conf, stats, realCol); if (cs == null || cs.getCountDistint() <= 0) { maxKeyCount = Long.MAX_VALUE; break; diff --git ql/src/java/org/apache/hadoop/hive/ql/optimizer/RemoveDynamicPruningBySize.java ql/src/java/org/apache/hadoop/hive/ql/optimizer/RemoveDynamicPruningBySize.java new file mode 100644 index 0000000..4803959 --- /dev/null +++ ql/src/java/org/apache/hadoop/hive/ql/optimizer/RemoveDynamicPruningBySize.java @@ -0,0 +1,72 @@ +/** + * 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.optimizer; + +import java.util.Stack; + +import org.apache.commons.logging.Log; +import org.apache.commons.logging.LogFactory; +import org.apache.hadoop.hive.conf.HiveConf.ConfVars; +import org.apache.hadoop.hive.ql.exec.AppMasterEventOperator; +import org.apache.hadoop.hive.ql.exec.Operator; +import org.apache.hadoop.hive.ql.lib.Node; +import org.apache.hadoop.hive.ql.lib.NodeProcessor; +import org.apache.hadoop.hive.ql.lib.NodeProcessorCtx; +import org.apache.hadoop.hive.ql.parse.OptimizeTezProcContext; +import org.apache.hadoop.hive.ql.parse.SemanticException; +import org.apache.hadoop.hive.ql.plan.AppMasterEventDesc; +import org.apache.hadoop.hive.ql.plan.DynamicPruningEventDesc; + +/** + * If we expect the number of keys for dynamic pruning to be too large we + * disable it. + */ +public class RemoveDynamicPruningBySize implements NodeProcessor { + + static final private Log LOG = LogFactory.getLog(RemoveDynamicPruningBySize.class.getName()); + + @Override + public Object process(Node nd, Stack stack, NodeProcessorCtx procContext, + Object... nodeOutputs) + throws SemanticException { + + OptimizeTezProcContext context = (OptimizeTezProcContext) procContext; + + AppMasterEventOperator event = (AppMasterEventOperator) nd; + AppMasterEventDesc desc = event.getConf(); + + if (desc.getStatistics().getDataSize() > context.conf + .getLongVar(ConfVars.TEZ_DYNAMIC_PARTITION_PRUNING_MAX_DATA_SIZE)) { + Operator child = event; + Operator curr = event; + + while (curr.getChildOperators().size() <= 1) { + child = curr; + curr = curr.getParentOperators().get(0); + } + // at this point we've found the fork in the op pipeline that has the + // pruning as a child plan. + LOG.info("Disabling dynamic pruning for: " + + ((DynamicPruningEventDesc) desc).getTableScan().getName() + + ". Expected data size is too big: " + desc.getStatistics().getDataSize()); + curr.removeChild(child); + } + return false; + } +} diff --git ql/src/java/org/apache/hadoop/hive/ql/optimizer/pcr/PcrExprProcFactory.java ql/src/java/org/apache/hadoop/hive/ql/optimizer/pcr/PcrExprProcFactory.java index e46e144..cbd4e6c 100644 --- ql/src/java/org/apache/hadoop/hive/ql/optimizer/pcr/PcrExprProcFactory.java +++ ql/src/java/org/apache/hadoop/hive/ql/optimizer/pcr/PcrExprProcFactory.java @@ -396,8 +396,7 @@ public Object process(Node nd, Stack stack, NodeProcessorCtx procCtx, return new NodeInfoWrapper(WalkState.CONSTANT, null, (ExprNodeDesc) nd); } - assert (false); - return null; + return new NodeInfoWrapper(WalkState.UNKNOWN, null, (ExprNodeDesc)nd); } } diff --git ql/src/java/org/apache/hadoop/hive/ql/optimizer/stats/annotation/StatsRulesProcFactory.java ql/src/java/org/apache/hadoop/hive/ql/optimizer/stats/annotation/StatsRulesProcFactory.java index 7fdba60..70cf1ba 100644 --- ql/src/java/org/apache/hadoop/hive/ql/optimizer/stats/annotation/StatsRulesProcFactory.java +++ ql/src/java/org/apache/hadoop/hive/ql/optimizer/stats/annotation/StatsRulesProcFactory.java @@ -18,12 +18,18 @@ package org.apache.hadoop.hive.ql.optimizer.stats.annotation; -import com.google.common.collect.Lists; -import com.google.common.collect.Maps; +import java.util.Collections; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.Set; +import java.util.Stack; + import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; import org.apache.hadoop.hive.conf.HiveConf; import org.apache.hadoop.hive.ql.ErrorMsg; +import org.apache.hadoop.hive.ql.exec.AppMasterEventOperator; import org.apache.hadoop.hive.ql.exec.ColumnInfo; import org.apache.hadoop.hive.ql.exec.CommonJoinOperator; import org.apache.hadoop.hive.ql.exec.FilterOperator; @@ -66,12 +72,8 @@ import org.apache.hadoop.hive.ql.udf.generic.GenericUDFOPOr; import org.apache.hadoop.hive.serde.serdeConstants; -import java.util.Collections; -import java.util.HashMap; -import java.util.List; -import java.util.Map; -import java.util.Set; -import java.util.Stack; +import com.google.common.collect.Lists; +import com.google.common.collect.Maps; public class StatsRulesProcFactory { @@ -655,7 +657,8 @@ public Object process(Node nd, Stack stack, NodeProcessorCtx procCtx, if (parentStats != null) { // worst case, in the absence of column statistics assume half the rows are emitted - if (gop.getChildOperators().get(0) instanceof ReduceSinkOperator) { + if (gop.getChildOperators().get(0) instanceof ReduceSinkOperator + || gop.getChildOperators().get(0) instanceof AppMasterEventOperator) { // map side stats = parentStats.clone(); diff --git ql/src/java/org/apache/hadoop/hive/ql/parse/AppMasterEventProcessor.java ql/src/java/org/apache/hadoop/hive/ql/parse/AppMasterEventProcessor.java new file mode 100644 index 0000000..e442f06 --- /dev/null +++ ql/src/java/org/apache/hadoop/hive/ql/parse/AppMasterEventProcessor.java @@ -0,0 +1,62 @@ +/** + * 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.parse; + +import java.util.ArrayList; +import java.util.List; +import java.util.Stack; + +import org.apache.commons.logging.Log; +import org.apache.commons.logging.LogFactory; +import org.apache.hadoop.hive.ql.exec.AppMasterEventOperator; +import org.apache.hadoop.hive.ql.lib.Node; +import org.apache.hadoop.hive.ql.lib.NodeProcessor; +import org.apache.hadoop.hive.ql.lib.NodeProcessorCtx; +import org.apache.hadoop.hive.ql.plan.DynamicPruningEventDesc; + +/** + * FileSinkProcessor is a simple rule to remember seen unions for later + * processing. + * + */ +public class AppMasterEventProcessor implements NodeProcessor { + + static final private Log LOG = LogFactory.getLog(AppMasterEventProcessor.class.getName()); + + @Override + public Object process(Node nd, Stack stack, NodeProcessorCtx procCtx, Object... nodeOutputs) + throws SemanticException { + GenTezProcContext context = (GenTezProcContext) procCtx; + AppMasterEventOperator event = (AppMasterEventOperator) nd; + DynamicPruningEventDesc desc = (DynamicPruningEventDesc) event.getConf(); + + // simply need to remember that we've seen an event operator. + context.eventOperatorSet.add(event); + + // and remember link between event and table scan + List events; + if (context.tsToEventMap.containsKey(desc.getTableScan())) { + events = context.tsToEventMap.get(desc.getTableScan()); + } else { + events = new ArrayList(); + } + context.tsToEventMap.put(desc.getTableScan(), events); + return true; + } +} diff --git ql/src/java/org/apache/hadoop/hive/ql/parse/FileSinkProcessor.java ql/src/java/org/apache/hadoop/hive/ql/parse/FileSinkProcessor.java index 9aed5fd..2f769d4 100644 --- ql/src/java/org/apache/hadoop/hive/ql/parse/FileSinkProcessor.java +++ ql/src/java/org/apache/hadoop/hive/ql/parse/FileSinkProcessor.java @@ -22,16 +22,14 @@ import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; -import org.apache.hadoop.fs.Path; -import org.apache.hadoop.hive.conf.HiveConf; import org.apache.hadoop.hive.ql.exec.FileSinkOperator; import org.apache.hadoop.hive.ql.lib.Node; import org.apache.hadoop.hive.ql.lib.NodeProcessor; import org.apache.hadoop.hive.ql.lib.NodeProcessorCtx; -import org.apache.hadoop.hive.ql.optimizer.GenMapRedUtils; /** - * FileSinkProcessor handles addition of merge, move and stats tasks for filesinks + * FileSinkProcessor is a simple rule to remember seen file sinks for later + * processing. * */ public class FileSinkProcessor implements NodeProcessor { @@ -39,12 +37,6 @@ static final private Log LOG = LogFactory.getLog(FileSinkProcessor.class.getName()); @Override - /* - * (non-Javadoc) - * we should ideally not modify the tree we traverse. - * However, since we need to walk the tree at any time when we modify the - * operator, we might as well do it here. - */ public Object process(Node nd, Stack stack, NodeProcessorCtx procCtx, Object... nodeOutputs) throws SemanticException { diff --git ql/src/java/org/apache/hadoop/hive/ql/parse/GenTezProcContext.java ql/src/java/org/apache/hadoop/hive/ql/parse/GenTezProcContext.java index 8513f99..e34ce28 100644 --- ql/src/java/org/apache/hadoop/hive/ql/parse/GenTezProcContext.java +++ ql/src/java/org/apache/hadoop/hive/ql/parse/GenTezProcContext.java @@ -26,29 +26,28 @@ import java.util.Map; import java.util.Set; -import org.apache.commons.lang3.tuple.ImmutablePair; -import org.apache.commons.lang3.tuple.Pair; import org.apache.hadoop.fs.Path; import org.apache.hadoop.hive.conf.HiveConf; +import org.apache.hadoop.hive.ql.exec.AppMasterEventOperator; import org.apache.hadoop.hive.ql.exec.DependencyCollectionTask; import org.apache.hadoop.hive.ql.exec.FileSinkOperator; import org.apache.hadoop.hive.ql.exec.MapJoinOperator; import org.apache.hadoop.hive.ql.exec.Operator; -import org.apache.hadoop.hive.ql.exec.UnionOperator; import org.apache.hadoop.hive.ql.exec.ReduceSinkOperator; +import org.apache.hadoop.hive.ql.exec.TableScanOperator; import org.apache.hadoop.hive.ql.exec.Task; import org.apache.hadoop.hive.ql.exec.TaskFactory; +import org.apache.hadoop.hive.ql.exec.UnionOperator; import org.apache.hadoop.hive.ql.exec.tez.TezTask; import org.apache.hadoop.hive.ql.hooks.ReadEntity; import org.apache.hadoop.hive.ql.hooks.WriteEntity; import org.apache.hadoop.hive.ql.lib.NodeProcessorCtx; import org.apache.hadoop.hive.ql.plan.BaseWork; import org.apache.hadoop.hive.ql.plan.DependencyCollectionWork; +import org.apache.hadoop.hive.ql.plan.FileSinkDesc; import org.apache.hadoop.hive.ql.plan.MoveWork; import org.apache.hadoop.hive.ql.plan.OperatorDesc; -import org.apache.hadoop.hive.ql.plan.FileSinkDesc; import org.apache.hadoop.hive.ql.plan.TezEdgeProperty; -import org.apache.hadoop.hive.ql.plan.TezEdgeProperty.EdgeType; import org.apache.hadoop.hive.ql.plan.TezWork; /** @@ -134,6 +133,15 @@ // remember which reducesinks we've already connected public final Set connectedReduceSinks; + // remember the event operators we've seen + public final Set eventOperatorSet; + + // remember the event operators we've abandoned. + public final Set abandonedEventOperatorSet; + + // remember the connections between ts and event + public final Map> tsToEventMap; + @SuppressWarnings("unchecked") public GenTezProcContext(HiveConf conf, ParseContext parseContext, List> moveTask, List> rootTasks, @@ -165,6 +173,9 @@ public GenTezProcContext(HiveConf conf, ParseContext parseContext, this.linkedFileSinks = new LinkedHashMap>(); this.fileSinkSet = new LinkedHashSet(); this.connectedReduceSinks = new LinkedHashSet(); + this.eventOperatorSet = new LinkedHashSet(); + this.abandonedEventOperatorSet = new LinkedHashSet(); + this.tsToEventMap = new LinkedHashMap>(); rootTasks.add(currentTask); } diff --git ql/src/java/org/apache/hadoop/hive/ql/parse/GenTezUtils.java ql/src/java/org/apache/hadoop/hive/ql/parse/GenTezUtils.java index deccb16..5057236 100644 --- ql/src/java/org/apache/hadoop/hive/ql/parse/GenTezUtils.java +++ ql/src/java/org/apache/hadoop/hive/ql/parse/GenTezUtils.java @@ -20,38 +20,43 @@ import java.util.ArrayList; import java.util.Deque; -import java.util.HashMap; import java.util.HashSet; import java.util.Iterator; -import java.util.List; import java.util.LinkedList; -import java.util.Map; +import java.util.List; import java.util.Set; -import org.apache.hadoop.fs.Path; import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.Path; import org.apache.hadoop.hive.conf.HiveConf; +import org.apache.hadoop.hive.ql.exec.AppMasterEventOperator; import org.apache.hadoop.hive.ql.exec.FetchTask; -import org.apache.hadoop.hive.ql.exec.Operator; import org.apache.hadoop.hive.ql.exec.FileSinkOperator; import org.apache.hadoop.hive.ql.exec.HashTableDummyOperator; +import org.apache.hadoop.hive.ql.exec.Operator; import org.apache.hadoop.hive.ql.exec.ReduceSinkOperator; import org.apache.hadoop.hive.ql.exec.TableScanOperator; import org.apache.hadoop.hive.ql.exec.UnionOperator; import org.apache.hadoop.hive.ql.exec.Utilities; import org.apache.hadoop.hive.ql.optimizer.GenMapRedUtils; import org.apache.hadoop.hive.ql.plan.BaseWork; +import org.apache.hadoop.hive.ql.plan.DynamicPruningEventDesc; +import org.apache.hadoop.hive.ql.plan.ExprNodeDesc; import org.apache.hadoop.hive.ql.plan.FileSinkDesc; import org.apache.hadoop.hive.ql.plan.MapWork; import org.apache.hadoop.hive.ql.plan.OperatorDesc; import org.apache.hadoop.hive.ql.plan.ReduceWork; +import org.apache.hadoop.hive.ql.plan.TableDesc; import org.apache.hadoop.hive.ql.plan.TezEdgeProperty; import org.apache.hadoop.hive.ql.plan.TezEdgeProperty.EdgeType; import org.apache.hadoop.hive.ql.plan.TezWork; import org.apache.hadoop.hive.ql.plan.UnionWork; +import com.google.common.collect.BiMap; +import com.google.common.collect.HashBiMap; + /** * GenTezUtils is a collection of shared helper methods to produce * TezWork @@ -119,12 +124,12 @@ public ReduceWork createReduceWork(GenTezProcContext context, Operator root, int maxReducers = context.conf.getIntVar(HiveConf.ConfVars.MAXREDUCERS); // min we allow tez to pick - int minPartition = Math.max(1, (int) (reduceSink.getConf().getNumReducers() + int minPartition = Math.max(1, (int) (reduceSink.getConf().getNumReducers() * minPartitionFactor)); minPartition = (minPartition > maxReducers) ? maxReducers : minPartition; // max we allow tez to pick - int maxPartition = (int) (reduceSink.getConf().getNumReducers() * maxPartitionFactor); + int maxPartition = (int) (reduceSink.getConf().getNumReducers() * maxPartitionFactor); maxPartition = (maxPartition > maxReducers) ? maxReducers : maxPartition; reduceWork.setMinReduceTasks(minPartition); @@ -201,18 +206,20 @@ public void removeUnionOperators(Configuration conf, GenTezProcContext context, BaseWork work) throws SemanticException { - Set> roots = work.getAllRootOperators(); + List> roots = new ArrayList>(); + roots.addAll(work.getAllRootOperators()); if (work.getDummyOps() != null) { roots.addAll(work.getDummyOps()); } + roots.addAll(context.eventOperatorSet); // need to clone the plan. - Set> newRoots = Utilities.cloneOperatorTree(conf, roots); + List> newRoots = Utilities.cloneOperatorTree(conf, roots); // we're cloning the operator plan but we're retaining the original work. That means // that root operators have to be replaced with the cloned ops. The replacement map // tells you what that mapping is. - Map, Operator> replacementMap = new HashMap, Operator>(); + BiMap, Operator> replacementMap = HashBiMap.create(); // there's some special handling for dummyOps required. Mapjoins won't be properly // initialized if their dummy parents aren't initialized. Since we cloned the plan @@ -222,11 +229,35 @@ public void removeUnionOperators(Configuration conf, GenTezProcContext context, Iterator> it = newRoots.iterator(); for (Operator orig: roots) { Operator newRoot = it.next(); + + replacementMap.put(orig, newRoot); + if (newRoot instanceof HashTableDummyOperator) { - dummyOps.add((HashTableDummyOperator)newRoot); + // dummy ops need to be updated to the cloned ones. + dummyOps.add((HashTableDummyOperator) newRoot); + it.remove(); + } else if (newRoot instanceof AppMasterEventOperator) { + // event operators point to table scan operators. When cloning these we + // need to restore the original scan. + if (newRoot.getConf() instanceof DynamicPruningEventDesc) { + TableScanOperator ts = ((DynamicPruningEventDesc) orig.getConf()).getTableScan(); + if (ts == null) { + throw new AssertionError("No table scan associated with dynamic event pruning. " + orig); + } + ((DynamicPruningEventDesc) newRoot.getConf()).setTableScan(ts); + } it.remove(); } else { - replacementMap.put(orig,newRoot); + if (newRoot instanceof TableScanOperator) { + if (context.tsToEventMap.containsKey(orig)) { + // we need to update event operators with the cloned table scan + for (AppMasterEventOperator event : context.tsToEventMap.get(orig)) { + ((DynamicPruningEventDesc) event.getConf()).setTableScan((TableScanOperator) newRoot); + } + } + } + context.rootToWorkMap.remove(orig); + context.rootToWorkMap.put(newRoot, work); } } @@ -263,6 +294,15 @@ public void removeUnionOperators(Configuration conf, GenTezProcContext context, desc.setLinkedFileSinkDesc(linked); } + if (current instanceof AppMasterEventOperator) { + // remember for additional processing later + context.eventOperatorSet.add((AppMasterEventOperator) current); + + // mark the original as abandoned. Don't need it anymore. + context.abandonedEventOperatorSet.add((AppMasterEventOperator) replacementMap.inverse() + .get(current)); + } + if (current instanceof UnionOperator) { Operator parent = null; int count = 0; @@ -328,4 +368,87 @@ public void processFileSink(GenTezProcContext context, FileSinkOperator fileSink } } } + + /** + * processAppMasterEvent sets up the event descriptor and the MapWork. + * + * @param procCtx + * @param event + */ + public void processAppMasterEvent(GenTezProcContext procCtx, AppMasterEventOperator event) { + + if (procCtx.abandonedEventOperatorSet.contains(event)) { + // don't need this anymore + return; + } + + DynamicPruningEventDesc eventDesc = (DynamicPruningEventDesc)event.getConf(); + TableScanOperator ts = eventDesc.getTableScan(); + + MapWork work = (MapWork) procCtx.rootToWorkMap.get(ts); + if (work == null) { + throw new AssertionError("No work found for tablescan " + ts); + } + + BaseWork enclosingWork = getEnclosingWork(event, procCtx); + if (enclosingWork == null) { + throw new AssertionError("Cannot find work for operator" + event); + } + String sourceName = enclosingWork.getName(); + + // store the vertex name in the operator pipeline + eventDesc.setVertexName(work.getName()); + eventDesc.setInputName(work.getAliases().get(0)); + + // store table descriptor in map-work + if (!work.getEventSourceTableDescMap().containsKey(sourceName)) { + work.getEventSourceTableDescMap().put(sourceName, new LinkedList()); + } + List tables = work.getEventSourceTableDescMap().get(sourceName); + tables.add(event.getConf().getTable()); + + // store column name in map-work + if (!work.getEventSourceColumnNameMap().containsKey(sourceName)) { + work.getEventSourceColumnNameMap().put(sourceName, new LinkedList()); + } + List columns = work.getEventSourceColumnNameMap().get(sourceName); + columns.add(eventDesc.getTargetColumnName()); + + // store partition key expr in map-work + if (!work.getEventSourcePartKeyExprMap().containsKey(sourceName)) { + work.getEventSourcePartKeyExprMap().put(sourceName, new LinkedList()); + } + List keys = work.getEventSourcePartKeyExprMap().get(sourceName); + keys.add(eventDesc.getPartKey()); + + } + + /** + * getEncosingWork finds the BaseWork any given operator belongs to. + */ + public BaseWork getEnclosingWork(Operator op, GenTezProcContext procCtx) { + List> ops = new ArrayList>(); + findRoots(op, ops); + for (Operator r : ops) { + BaseWork work = procCtx.rootToWorkMap.get(r); + if (work != null) { + return work; + } + } + return null; + } + + /* + * findRoots returns all root operators (in ops) that result in operator op + */ + private void findRoots(Operator op, List> ops) { + List> parents = op.getParentOperators(); + if (parents == null || parents.isEmpty()) { + ops.add(op); + return; + } + for (Operator p : parents) { + findRoots(p, ops); + } + } } diff --git ql/src/java/org/apache/hadoop/hive/ql/parse/OptimizeTezProcContext.java ql/src/java/org/apache/hadoop/hive/ql/parse/OptimizeTezProcContext.java index 0807d65..ee71971 100644 --- ql/src/java/org/apache/hadoop/hive/ql/parse/OptimizeTezProcContext.java +++ ql/src/java/org/apache/hadoop/hive/ql/parse/OptimizeTezProcContext.java @@ -23,13 +23,18 @@ import java.util.Set; import org.apache.hadoop.hive.conf.HiveConf; +import org.apache.hadoop.hive.ql.exec.AppMasterEventOperator; import org.apache.hadoop.hive.ql.exec.Operator; import org.apache.hadoop.hive.ql.exec.ReduceSinkOperator; +import org.apache.hadoop.hive.ql.exec.TableScanOperator; import org.apache.hadoop.hive.ql.hooks.ReadEntity; import org.apache.hadoop.hive.ql.hooks.WriteEntity; import org.apache.hadoop.hive.ql.lib.NodeProcessorCtx; import org.apache.hadoop.hive.ql.plan.OperatorDesc; +import com.google.common.collect.HashMultimap; +import com.google.common.collect.Multimap; + /** * OptimizeTezProcContext. OptimizeTezProcContext maintains information * about the current operator plan as we walk the operator tree @@ -47,19 +52,23 @@ public final Set visitedReduceSinks = new HashSet(); + public final Multimap eventOpToTableScanMap = + HashMultimap.create(); + // rootOperators are all the table scan operators in sequence // of traversal - public final Deque> rootOperators; + public Deque> rootOperators; - @SuppressWarnings("unchecked") - public OptimizeTezProcContext(HiveConf conf, ParseContext parseContext, - Set inputs, Set outputs, - Deque> rootOperators) { + public OptimizeTezProcContext(HiveConf conf, ParseContext parseContext, Set inputs, + Set outputs) { this.conf = conf; this.parseContext = parseContext; this.inputs = inputs; this.outputs = outputs; - this.rootOperators = rootOperators; + } + + public void setRootOperators(Deque> roots) { + this.rootOperators = roots; } } diff --git ql/src/java/org/apache/hadoop/hive/ql/parse/TezCompiler.java ql/src/java/org/apache/hadoop/hive/ql/parse/TezCompiler.java index d58c59d..a40d32f 100644 --- ql/src/java/org/apache/hadoop/hive/ql/parse/TezCompiler.java +++ ql/src/java/org/apache/hadoop/hive/ql/parse/TezCompiler.java @@ -21,20 +21,24 @@ import java.util.ArrayList; import java.util.Deque; import java.util.HashMap; +import java.util.HashSet; import java.util.LinkedHashMap; import java.util.LinkedList; import java.util.List; import java.util.Map; import java.util.Set; import java.util.Stack; +import java.util.concurrent.atomic.AtomicInteger; import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; import org.apache.hadoop.hive.conf.HiveConf; import org.apache.hadoop.hive.conf.HiveConf.ConfVars; import org.apache.hadoop.hive.ql.Context; +import org.apache.hadoop.hive.ql.exec.AppMasterEventOperator; import org.apache.hadoop.hive.ql.exec.ConditionalTask; import org.apache.hadoop.hive.ql.exec.FileSinkOperator; +import org.apache.hadoop.hive.ql.exec.FilterOperator; import org.apache.hadoop.hive.ql.exec.JoinOperator; import org.apache.hadoop.hive.ql.exec.MapJoinOperator; import org.apache.hadoop.hive.ql.exec.Operator; @@ -52,20 +56,24 @@ import org.apache.hadoop.hive.ql.lib.GraphWalker; import org.apache.hadoop.hive.ql.lib.Node; import org.apache.hadoop.hive.ql.lib.NodeProcessor; -import org.apache.hadoop.hive.ql.lib.NodeProcessorCtx; import org.apache.hadoop.hive.ql.lib.Rule; import org.apache.hadoop.hive.ql.lib.RuleRegExp; import org.apache.hadoop.hive.ql.metadata.Hive; +import org.apache.hadoop.hive.ql.optimizer.ConstantPropagate; import org.apache.hadoop.hive.ql.optimizer.ConvertJoinMapJoin; +import org.apache.hadoop.hive.ql.optimizer.DynamicPartitionPruningOptimization; import org.apache.hadoop.hive.ql.optimizer.ReduceSinkMapJoinProc; +import org.apache.hadoop.hive.ql.optimizer.RemoveDynamicPruningBySize; import org.apache.hadoop.hive.ql.optimizer.SetReducerParallelism; import org.apache.hadoop.hive.ql.optimizer.physical.CrossProductCheck; import org.apache.hadoop.hive.ql.optimizer.physical.MetadataOnlyOptimizer; import org.apache.hadoop.hive.ql.optimizer.physical.NullScanOptimizer; import org.apache.hadoop.hive.ql.optimizer.physical.PhysicalContext; -import org.apache.hadoop.hive.ql.optimizer.physical.Vectorizer; import org.apache.hadoop.hive.ql.optimizer.physical.StageIDsRearranger; +import org.apache.hadoop.hive.ql.optimizer.physical.Vectorizer; +import org.apache.hadoop.hive.ql.optimizer.stats.annotation.AnnotateWithStatistics; import org.apache.hadoop.hive.ql.plan.BaseWork; +import org.apache.hadoop.hive.ql.plan.DynamicPruningEventDesc; import org.apache.hadoop.hive.ql.plan.MapWork; import org.apache.hadoop.hive.ql.plan.MoveWork; import org.apache.hadoop.hive.ql.plan.OperatorDesc; @@ -85,7 +93,7 @@ public TezCompiler() { @Override public void init(HiveConf conf, LogHelper console, Hive db) { super.init(conf, console, db); - + // Tez requires us to use RPC for the query plan HiveConf.setBoolVar(conf, ConfVars.HIVE_RPC_QUERY_PLAN, true); @@ -98,33 +106,204 @@ public void init(HiveConf conf, LogHelper console, Hive db) { protected void optimizeOperatorPlan(ParseContext pCtx, Set inputs, Set outputs) throws SemanticException { - // Sequence of TableScan operators to be walked + // Create the context for the walker + OptimizeTezProcContext procCtx = new OptimizeTezProcContext(conf, pCtx, inputs, outputs); + + // setup dynamic partition pruning where possible + runDynamicPartitionPruning(procCtx, inputs, outputs); + + // setup stats in the operator plan + runStatsAnnotation(procCtx); + + // run the optimizations that use stats for optimization + runStatsDependentOptimizations(procCtx, inputs, outputs); + + // after the stats phase we might have some cyclic dependencies that we need + // to take care of. + runCycleAnalysisForPartitionPruning(procCtx, inputs, outputs); + + } + + private void runCycleAnalysisForPartitionPruning(OptimizeTezProcContext procCtx, + Set inputs, Set outputs) throws SemanticException { + + if (!procCtx.conf.getBoolVar(ConfVars.TEZ_DYNAMIC_PARTITION_PRUNING)) { + return; + } + + boolean cycleFree = false; + while (!cycleFree) { + cycleFree = true; + Set>> components = getComponents(procCtx); + for (Set> component : components) { + if (LOG.isDebugEnabled()) { + LOG.debug("Component: "); + for (Operator co : component) { + LOG.debug("Operator: " + co.getName() + ", " + co.getIdentifier()); + } + } + if (component.size() != 1) { + LOG.info("Found cycle in operator plan..."); + cycleFree = false; + removeEventOperator(component); + } + } + LOG.info("Cycle free: " + cycleFree); + } + } + + private void removeEventOperator(Set> component) { + AppMasterEventOperator victim = null; + for (Operator o : component) { + if (o instanceof AppMasterEventOperator) { + if (victim == null + || o.getConf().getStatistics().getDataSize() < victim.getConf().getStatistics() + .getDataSize()) { + victim = (AppMasterEventOperator) o; + } + } + } + + Operator child = victim; + Operator curr = victim; + + while (curr.getChildOperators().size() <= 1) { + child = curr; + curr = curr.getParentOperators().get(0); + } + + // at this point we've found the fork in the op pipeline that has the + // pruning as a child plan. + LOG.info("Disabling dynamic pruning for: " + + ((DynamicPruningEventDesc) victim.getConf()).getTableScan().toString() + + ". Needed to break cyclic dependency"); + curr.removeChild(child); + } + + // Tarjan's algo + private Set>> getComponents(OptimizeTezProcContext procCtx) { Deque> deque = new LinkedList>(); - deque.addAll(pCtx.getTopOps().values()); + deque.addAll(procCtx.parseContext.getTopOps().values()); - // Create the context for the walker - OptimizeTezProcContext procCtx - = new OptimizeTezProcContext(conf, pCtx, inputs, outputs, deque); + AtomicInteger index = new AtomicInteger(); + Map, Integer> indexes = new HashMap, Integer>(); + Map, Integer> lowLinks = new HashMap, Integer>(); + Stack> nodes = new Stack>(); + Set>> components = new HashSet>>(); + + for (Operator o : deque) { + if (!indexes.containsKey(o)) { + connect(o, index, nodes, indexes, lowLinks, components); + } + } + + return components; + } + + private void connect(Operator o, AtomicInteger index, Stack> nodes, + Map, Integer> indexes, Map, Integer> lowLinks, + Set>> components) { + + indexes.put(o, index.get()); + lowLinks.put(o, index.get()); + index.incrementAndGet(); + nodes.push(o); + + List> children; + if (o instanceof AppMasterEventOperator) { + children = new ArrayList>(); + children.addAll(o.getChildOperators()); + TableScanOperator ts = ((DynamicPruningEventDesc) o.getConf()).getTableScan(); + LOG.debug("Adding special edge: " + o.getName() + " --> " + ts.toString()); + children.add(ts); + } else { + children = o.getChildOperators(); + } + + for (Operator child : children) { + if (!indexes.containsKey(child)) { + connect(child, index, nodes, indexes, lowLinks, components); + lowLinks.put(child, Math.min(lowLinks.get(o), lowLinks.get(child))); + } else if (nodes.contains(child)) { + lowLinks.put(o, Math.min(lowLinks.get(o), indexes.get(child))); + } + } + + if (lowLinks.get(o).equals(indexes.get(o))) { + Set> component = new HashSet>(); + components.add(component); + Operator current; + do { + current = nodes.pop(); + component.add(current); + } while (current != o); + } + } + + private void runStatsAnnotation(OptimizeTezProcContext procCtx) throws SemanticException { + new AnnotateWithStatistics().transform(procCtx.parseContext); + } + + private void runStatsDependentOptimizations(OptimizeTezProcContext procCtx, + Set inputs, Set outputs) throws SemanticException { + + // Sequence of TableScan operators to be walked + Deque> deque = new LinkedList>(); + deque.addAll(procCtx.parseContext.getTopOps().values()); // create a walker which walks the tree in a DFS manner while maintaining // the operator stack. Map opRules = new LinkedHashMap(); - opRules.put(new RuleRegExp(new String("Set parallelism - ReduceSink"), + opRules.put(new RuleRegExp("Set parallelism - ReduceSink", ReduceSinkOperator.getOperatorName() + "%"), new SetReducerParallelism()); - opRules.put(new RuleRegExp(new String("Convert Join to Map-join"), + opRules.put(new RuleRegExp("Convert Join to Map-join", JoinOperator.getOperatorName() + "%"), new ConvertJoinMapJoin()); + opRules.put( + new RuleRegExp("Remove dynamic pruning by size", + AppMasterEventOperator.getOperatorName() + "%"), + new RemoveDynamicPruningBySize()); + // The dispatcher fires the processor corresponding to the closest matching // rule and passes the context along Dispatcher disp = new DefaultRuleDispatcher(null, opRules, procCtx); List topNodes = new ArrayList(); - topNodes.addAll(pCtx.getTopOps().values()); + topNodes.addAll(procCtx.parseContext.getTopOps().values()); GraphWalker ogw = new ForwardWalker(disp); ogw.startWalking(topNodes, null); } + private void runDynamicPartitionPruning(OptimizeTezProcContext procCtx, Set inputs, + Set outputs) throws SemanticException { + + if (!procCtx.conf.getBoolVar(ConfVars.TEZ_DYNAMIC_PARTITION_PRUNING)) { + return; + } + + // Sequence of TableScan operators to be walked + Deque> deque = new LinkedList>(); + deque.addAll(procCtx.parseContext.getTopOps().values()); + + Map opRules = new LinkedHashMap(); + opRules.put( + new RuleRegExp(new String("Dynamic Partition Pruning"), FilterOperator.getOperatorName() + + "%"), new DynamicPartitionPruningOptimization()); + + // The dispatcher fires the processor corresponding to the closest matching + // rule and passes the context along + Dispatcher disp = new DefaultRuleDispatcher(null, opRules, procCtx); + List topNodes = new ArrayList(); + topNodes.addAll(procCtx.parseContext.getTopOps().values()); + GraphWalker ogw = new ForwardWalker(disp); + ogw.startWalking(topNodes, null); + + // need a new run of the constant folding because we might have created lots + // of "and true and true" conditions. + new ConstantPropagate().transform(procCtx.parseContext); + } + @Override protected void generateTaskTree(List> rootTasks, ParseContext pCtx, List> mvTask, Set inputs, Set outputs) @@ -158,19 +337,12 @@ protected void generateTaskTree(List> rootTasks, Pa new ProcessAnalyzeTable(GenTezUtils.getUtils())); opRules.put(new RuleRegExp("Remember union", - UnionOperator.getOperatorName() + "%"), new NodeProcessor() - { - @Override - public Object process(Node n, Stack s, - NodeProcessorCtx procCtx, Object... os) throws SemanticException { - GenTezProcContext context = (GenTezProcContext) procCtx; - UnionOperator union = (UnionOperator) n; - - // simply need to remember that we've seen a union. - context.currentUnionOperators.add(union); - return null; - } - }); + UnionOperator.getOperatorName() + "%"), + new UnionProcessor()); + + opRules.put(new RuleRegExp("AppMasterEventOperator", + AppMasterEventOperator.getOperatorName() + "%"), + new AppMasterEventProcessor()); // The dispatcher fires the processor corresponding to the closest matching // rule and passes the context along @@ -185,10 +357,17 @@ public Object process(Node n, Stack s, GenTezUtils.getUtils().removeUnionOperators(conf, procCtx, w); } - // finally make sure the file sink operators are set up right + // then we make sure the file sink operators are set up right for (FileSinkOperator fileSink: procCtx.fileSinkSet) { GenTezUtils.getUtils().processFileSink(procCtx, fileSink); } + + // and finally we hook up any events that need to be sent to the tez AM + LOG.debug("There are " + procCtx.eventOperatorSet.size() + " app master events."); + for (AppMasterEventOperator event : procCtx.eventOperatorSet) { + LOG.debug("Handling AppMasterEventOperator: " + event); + GenTezUtils.getUtils().processAppMasterEvent(procCtx, event); + } } @Override diff --git ql/src/java/org/apache/hadoop/hive/ql/parse/UnionProcessor.java ql/src/java/org/apache/hadoop/hive/ql/parse/UnionProcessor.java new file mode 100644 index 0000000..5ee8b77 --- /dev/null +++ ql/src/java/org/apache/hadoop/hive/ql/parse/UnionProcessor.java @@ -0,0 +1,49 @@ +/** + * 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.parse; + +import java.util.Stack; + +import org.apache.commons.logging.Log; +import org.apache.commons.logging.LogFactory; +import org.apache.hadoop.hive.ql.exec.UnionOperator; +import org.apache.hadoop.hive.ql.lib.Node; +import org.apache.hadoop.hive.ql.lib.NodeProcessor; +import org.apache.hadoop.hive.ql.lib.NodeProcessorCtx; + +/** + * FileSinkProcessor is a simple rule to remember seen unions for later + * processing. + * + */ +public class UnionProcessor implements NodeProcessor { + + static final private Log LOG = LogFactory.getLog(UnionProcessor.class.getName()); + + @Override + public Object process(Node nd, Stack stack, NodeProcessorCtx procCtx, Object... nodeOutputs) + throws SemanticException { + GenTezProcContext context = (GenTezProcContext) procCtx; + UnionOperator union = (UnionOperator) nd; + + // simply need to remember that we've seen a union. + context.currentUnionOperators.add(union); + return null; + } +} diff --git ql/src/java/org/apache/hadoop/hive/ql/plan/AppMasterEventDesc.java ql/src/java/org/apache/hadoop/hive/ql/plan/AppMasterEventDesc.java new file mode 100644 index 0000000..264f959 --- /dev/null +++ ql/src/java/org/apache/hadoop/hive/ql/plan/AppMasterEventDesc.java @@ -0,0 +1,63 @@ +/** + * 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.plan; + +import java.io.IOException; + +import org.apache.hadoop.io.DataOutputBuffer; + + +@SuppressWarnings("serial") +@Explain(displayName = "Application Master Event Operator") +public class AppMasterEventDesc extends AbstractOperatorDesc { + + private TableDesc table; + private String vertexName; + private String inputName; + + @Explain(displayName = "Target Vertex") + public String getVertexName() { + return vertexName; + } + + @Explain(displayName = "Target Input") + public String getInputName() { + return inputName; + } + + public void setInputName(String inputName) { + this.inputName = inputName; + } + + public void setVertexName(String vertexName) { + this.vertexName = vertexName; + } + + public TableDesc getTable() { + return table; + } + + public void setTable(TableDesc table) { + this.table = table; + } + + public void writeEventHeader(DataOutputBuffer buffer) throws IOException { + // nothing to add + } +} diff --git ql/src/java/org/apache/hadoop/hive/ql/plan/DynamicPruningEventDesc.java ql/src/java/org/apache/hadoop/hive/ql/plan/DynamicPruningEventDesc.java new file mode 100644 index 0000000..d6617b5 --- /dev/null +++ ql/src/java/org/apache/hadoop/hive/ql/plan/DynamicPruningEventDesc.java @@ -0,0 +1,74 @@ +/** + * 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.plan; + +import java.io.IOException; + +import org.apache.hadoop.hive.ql.exec.TableScanOperator; +import org.apache.hadoop.io.DataOutputBuffer; + +@SuppressWarnings("serial") +@Explain(displayName = "Dynamic Partitioning Event Operator") +public class DynamicPruningEventDesc extends AppMasterEventDesc { + + // column in the target table that will be pruned against + private String targetColumnName; + + // tableScan is only available during compile + private transient TableScanOperator tableScan; + + // the partition column we're interested in + private ExprNodeDesc partKey; + + public TableScanOperator getTableScan() { + return tableScan; + } + + public void setTableScan(TableScanOperator tableScan) { + this.tableScan = tableScan; + } + + @Explain(displayName = "Target column") + public String getTargetColumnName() { + return targetColumnName; + } + + public void setTargetColumnName(String columnName) { + this.targetColumnName = columnName; + } + + @Override + public void writeEventHeader(DataOutputBuffer buffer) throws IOException { + super.writeEventHeader(buffer); + buffer.writeUTF(targetColumnName); + } + + public void setPartKey(ExprNodeDesc partKey) { + this.partKey = partKey; + } + + @Explain(displayName = "Partition key expr") + public String getPartKeyString() { + return this.partKey.getExprString(); + } + + public ExprNodeDesc getPartKey() { + return this.partKey; + } +} diff --git ql/src/java/org/apache/hadoop/hive/ql/plan/ExprNodeDynamicListDesc.java ql/src/java/org/apache/hadoop/hive/ql/plan/ExprNodeDynamicListDesc.java new file mode 100644 index 0000000..18e4fbd --- /dev/null +++ ql/src/java/org/apache/hadoop/hive/ql/plan/ExprNodeDynamicListDesc.java @@ -0,0 +1,81 @@ +/** + * 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.plan; + +import org.apache.hadoop.hive.ql.exec.Operator; +import org.apache.hadoop.hive.serde2.typeinfo.TypeInfo; + +/** + * This expression represents a list that will be available at runtime. + */ +@SuppressWarnings("serial") +public class ExprNodeDynamicListDesc extends ExprNodeDesc { + + Operator source; + int keyIndex; + + public ExprNodeDynamicListDesc() { + } + + public ExprNodeDynamicListDesc(TypeInfo typeInfo, Operator source, int keyIndex) { + super(typeInfo); + this.source = source; + this.keyIndex = keyIndex; + } + + public void setSource(Operator source) { + this.source = source; + } + + public Operator getSource() { + return source; + } + + public void setKeyIndex(int keyIndex) { + this.keyIndex = keyIndex; + } + + public int getKeyIndex() { + return this.keyIndex; + } + + @Override + public ExprNodeDesc clone() { + ExprNodeDynamicListDesc clone = new ExprNodeDynamicListDesc(typeInfo, source, keyIndex); + return clone; + } + + @Override + public boolean isSame(Object o) { + if (o instanceof ExprNodeDynamicListDesc) { + return source.equals(((ExprNodeDynamicListDesc)o).getSource()); + } + return false; + } + + @Override + public String getExprString() { + return source.toString(); + } + + @Override + public String toString() { + return source.toString(); + } +} diff --git ql/src/java/org/apache/hadoop/hive/ql/plan/MapWork.java ql/src/java/org/apache/hadoop/hive/ql/plan/MapWork.java index b4278d3..eda53da 100644 --- ql/src/java/org/apache/hadoop/hive/ql/plan/MapWork.java +++ ql/src/java/org/apache/hadoop/hive/ql/plan/MapWork.java @@ -26,9 +26,9 @@ import java.util.LinkedHashMap; import java.util.LinkedHashSet; import java.util.List; -import java.util.Set; import java.util.Map; import java.util.Map.Entry; +import java.util.Set; import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; @@ -116,6 +116,14 @@ private boolean useOneNullRowInputFormat; + // used for dynamic partitioning + private Map> eventSourceTableDescMap = + new LinkedHashMap>(); + private Map> eventSourceColumnNameMap = + new LinkedHashMap>(); + private Map> eventSourcePartKeyExprMap = + new LinkedHashMap>(); + public MapWork() {} public MapWork(String name) { @@ -525,4 +533,28 @@ public void logPathToAliases() { } } } + + public void setEventSourceTableDescMap(Map> map) { + this.eventSourceTableDescMap = map; + } + + public Map> getEventSourceTableDescMap() { + return eventSourceTableDescMap; + } + + public void setEventSourceColumnNameMap(Map> map) { + this.eventSourceColumnNameMap = map; + } + + public Map> getEventSourceColumnNameMap() { + return eventSourceColumnNameMap; + } + + public Map> getEventSourcePartKeyExprMap() { + return eventSourcePartKeyExprMap; + } + + public void setEventSourcePartKeyExprMap(Map> map) { + this.eventSourcePartKeyExprMap = map; + } } diff --git ql/src/java/org/apache/hadoop/hive/ql/plan/ReduceWork.java ql/src/java/org/apache/hadoop/hive/ql/plan/ReduceWork.java index 675b70e..53ba031 100644 --- ql/src/java/org/apache/hadoop/hive/ql/plan/ReduceWork.java +++ ql/src/java/org/apache/hadoop/hive/ql/plan/ReduceWork.java @@ -20,8 +20,8 @@ import java.util.ArrayList; import java.util.HashMap; -import java.util.LinkedHashSet; import java.util.LinkedHashMap; +import java.util.LinkedHashSet; import java.util.List; import java.util.Map; import java.util.Set; @@ -32,13 +32,9 @@ import org.apache.hadoop.hive.ql.exec.Operator; import org.apache.hadoop.hive.ql.exec.OperatorUtils; import org.apache.hadoop.hive.ql.exec.Utilities; -import org.apache.hadoop.hive.ql.lib.NodeProcessor; -import org.apache.hadoop.hive.ql.lib.Rule; import org.apache.hadoop.hive.serde2.Deserializer; -import org.apache.hadoop.hive.serde2.SerDe; import org.apache.hadoop.hive.serde2.SerDeUtils; import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspector; -import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspectorFactory; import org.apache.hadoop.hive.serde2.objectinspector.StructField; import org.apache.hadoop.hive.serde2.objectinspector.StructObjectInspector; import org.apache.hadoop.mapred.JobConf; @@ -99,7 +95,7 @@ public ReduceWork(String name) { private ObjectInspector keyObjectInspector = null; private ObjectInspector valueObjectInspector = null; - private Map reduceColumnNameMap = new LinkedHashMap(); + private final Map reduceColumnNameMap = new LinkedHashMap(); /** * If the plan has a reducer and correspondingly a reduce-sink, then store the TableDesc pointing @@ -118,7 +114,7 @@ public TableDesc getKeyDesc() { private ObjectInspector getObjectInspector(TableDesc desc) { ObjectInspector objectInspector; try { - Deserializer deserializer = (SerDe) ReflectionUtils.newInstance(desc + Deserializer deserializer = ReflectionUtils.newInstance(desc .getDeserializerClass(), null); SerDeUtils.initializeSerDe(deserializer, null, desc.getProperties(), null); objectInspector = deserializer.getObjectInspector(); @@ -239,7 +235,6 @@ public void setTagToInput(final Map tagToInput) { @Override public void replaceRoots(Map, Operator> replacementMap) { - assert replacementMap.size() == 1; setReducer(replacementMap.get(getReducer())); } diff --git ql/src/java/org/apache/hadoop/hive/ql/ppd/SyntheticJoinPredicate.java ql/src/java/org/apache/hadoop/hive/ql/ppd/SyntheticJoinPredicate.java new file mode 100644 index 0000000..fe59c13 --- /dev/null +++ ql/src/java/org/apache/hadoop/hive/ql/ppd/SyntheticJoinPredicate.java @@ -0,0 +1,283 @@ +/** + * 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.ppd; + +import java.util.ArrayList; +import java.util.HashSet; +import java.util.LinkedHashMap; +import java.util.List; +import java.util.Map; +import java.util.Set; +import java.util.Stack; + +import org.apache.commons.logging.Log; +import org.apache.commons.logging.LogFactory; +import org.apache.hadoop.hive.conf.HiveConf.ConfVars; +import org.apache.hadoop.hive.ql.exec.CommonJoinOperator; +import org.apache.hadoop.hive.ql.exec.FunctionRegistry; +import org.apache.hadoop.hive.ql.exec.JoinOperator; +import org.apache.hadoop.hive.ql.exec.Operator; +import org.apache.hadoop.hive.ql.exec.OperatorFactory; +import org.apache.hadoop.hive.ql.exec.ReduceSinkOperator; +import org.apache.hadoop.hive.ql.exec.RowSchema; +import org.apache.hadoop.hive.ql.exec.TableScanOperator; +import org.apache.hadoop.hive.ql.lib.DefaultRuleDispatcher; +import org.apache.hadoop.hive.ql.lib.Dispatcher; +import org.apache.hadoop.hive.ql.lib.GraphWalker; +import org.apache.hadoop.hive.ql.lib.Node; +import org.apache.hadoop.hive.ql.lib.NodeProcessor; +import org.apache.hadoop.hive.ql.lib.NodeProcessorCtx; +import org.apache.hadoop.hive.ql.lib.PreOrderWalker; +import org.apache.hadoop.hive.ql.lib.Rule; +import org.apache.hadoop.hive.ql.lib.RuleRegExp; +import org.apache.hadoop.hive.ql.optimizer.Transform; +import org.apache.hadoop.hive.ql.parse.OpParseContext; +import org.apache.hadoop.hive.ql.parse.ParseContext; +import org.apache.hadoop.hive.ql.parse.RowResolver; +import org.apache.hadoop.hive.ql.parse.SemanticException; +import org.apache.hadoop.hive.ql.plan.ExprNodeDesc; +import org.apache.hadoop.hive.ql.plan.ExprNodeDynamicListDesc; +import org.apache.hadoop.hive.ql.plan.ExprNodeGenericFuncDesc; +import org.apache.hadoop.hive.ql.plan.FilterDesc; +import org.apache.hadoop.hive.ql.plan.JoinCondDesc; +import org.apache.hadoop.hive.ql.plan.JoinDesc; +import org.apache.hadoop.hive.ql.plan.OperatorDesc; + +/** + * creates synthetic predicates that represent "IN (keylist other table)" + */ +public class SyntheticJoinPredicate implements Transform { + + private static transient Log LOG = LogFactory.getLog(SyntheticJoinPredicate.class.getName()); + + @Override + public ParseContext transform(ParseContext pctx) throws SemanticException { + + if (!pctx.getConf().getVar(ConfVars.HIVE_EXECUTION_ENGINE).equals("tez") + || !pctx.getConf().getBoolVar(ConfVars.TEZ_DYNAMIC_PARTITION_PRUNING)) { + return pctx; + } + + Map opRules = new LinkedHashMap(); + opRules.put(new RuleRegExp("R1", "(" + + TableScanOperator.getOperatorName() + "%" + ".*" + + ReduceSinkOperator.getOperatorName() + "%" + + JoinOperator.getOperatorName() + "%)"), new JoinSynthetic()); + + // The dispatcher fires the processor corresponding to the closest matching + // rule and passes the context along + SyntheticContext context = new SyntheticContext(pctx); + Dispatcher disp = new DefaultRuleDispatcher(null, opRules, context); + GraphWalker ogw = new PreOrderWalker(disp); + + // Create a list of top op nodes + List topNodes = new ArrayList(); + topNodes.addAll(pctx.getTopOps().values()); + ogw.startWalking(topNodes, null); + + return pctx; + } + + // insert filter operator between target(child) and input(parent) + private static Operator createFilter(Operator target, Operator parent, + RowResolver parentRR, ExprNodeDesc filterExpr) { + Operator filter = OperatorFactory.get(new FilterDesc(filterExpr, false), + new RowSchema(parentRR.getColumnInfos())); + filter.setParentOperators(new ArrayList>()); + filter.setChildOperators(new ArrayList>()); + filter.getParentOperators().add(parent); + filter.getChildOperators().add(target); + parent.replaceChild(target, filter); + target.replaceParent(parent, filter); + return filter; + } + + private static class SyntheticContext implements NodeProcessorCtx { + + ParseContext parseContext; + + public SyntheticContext(ParseContext pCtx) { + parseContext = pCtx; + } + + public ParseContext getParseContext() { + return parseContext; + } + } + + private static class JoinSynthetic implements NodeProcessor { + @Override + public Object process(Node nd, Stack stack, NodeProcessorCtx procCtx, + Object... nodeOutputs) throws SemanticException { + + ParseContext pCtx = ((SyntheticContext) procCtx).getParseContext(); + + @SuppressWarnings("unchecked") + CommonJoinOperator join = (CommonJoinOperator) nd; + + ReduceSinkOperator source = (ReduceSinkOperator) stack.get(stack.size() - 2); + int srcPos = join.getParentOperators().indexOf(source); + + List> parents = join.getParentOperators(); + + int[][] targets = getTargets(join); + + Operator parent = source.getParentOperators().get(0); + RowResolver parentRR = pCtx.getOpParseCtx().get(parent).getRowResolver(); + + // don't generate for null-safes. + if (join.getConf().getNullSafes() != null) { + for (boolean b : join.getConf().getNullSafes()) { + if (b) { + return null; + } + } + } + + for (int targetPos: targets[srcPos]) { + if (srcPos == targetPos) { + continue; + } + + if (LOG.isDebugEnabled()) { + LOG.debug("Synthetic predicate: " + srcPos + " --> " + targetPos); + } + ReduceSinkOperator target = (ReduceSinkOperator) parents.get(targetPos); + List sourceKeys = source.getConf().getKeyCols(); + List targetKeys = target.getConf().getKeyCols(); + + if (sourceKeys.size() < 1) { + continue; + } + + ExprNodeDesc syntheticExpr = null; + + for (int i = 0; i < sourceKeys.size(); ++i) { + List inArgs = new ArrayList(); + inArgs.add(sourceKeys.get(i)); + + ExprNodeDynamicListDesc dynamicExpr = + new ExprNodeDynamicListDesc(targetKeys.get(i).getTypeInfo(), target, i); + + inArgs.add(dynamicExpr); + + ExprNodeDesc syntheticInExpr = + ExprNodeGenericFuncDesc.newInstance(FunctionRegistry.getFunctionInfo("in") + .getGenericUDF(), inArgs); + + if (syntheticExpr != null) { + List andArgs = new ArrayList(); + andArgs.add(syntheticExpr); + andArgs.add(syntheticInExpr); + + syntheticExpr = + ExprNodeGenericFuncDesc.newInstance(FunctionRegistry.getFunctionInfo("and") + .getGenericUDF(), andArgs); + } else { + syntheticExpr = syntheticInExpr; + } + } + + Operator newFilter = createFilter(source, parent, parentRR, syntheticExpr); + pCtx.getOpParseCtx().put(newFilter, new OpParseContext(parentRR)); + parent = newFilter; + } + + return null; + } + + // calculate filter propagation directions for each alias + // L<->R for inner/semi join, L<-R for left outer join, R<-L for right outer + // join + private int[][] getTargets(CommonJoinOperator join) { + JoinCondDesc[] conds = join.getConf().getConds(); + + int aliases = conds.length + 1; + Vectors vector = new Vectors(aliases); + for (JoinCondDesc cond : conds) { + int left = cond.getLeft(); + int right = cond.getRight(); + switch (cond.getType()) { + case JoinDesc.INNER_JOIN: + case JoinDesc.LEFT_SEMI_JOIN: + vector.add(left, right); + vector.add(right, left); + break; + case JoinDesc.LEFT_OUTER_JOIN: + vector.add(right, left); + break; + case JoinDesc.RIGHT_OUTER_JOIN: + vector.add(left, right); + break; + case JoinDesc.FULL_OUTER_JOIN: + break; + } + } + int[][] result = new int[aliases][]; + for (int pos = 0 ; pos < aliases; pos++) { + // find all targets recursively + result[pos] = vector.traverse(pos); + } + return result; + } + } + + private static class Vectors { + + private final Set[] vector; + + @SuppressWarnings("unchecked") + public Vectors(int length) { + vector = new Set[length]; + } + + public void add(int from, int to) { + if (vector[from] == null) { + vector[from] = new HashSet(); + } + vector[from].add(to); + } + + public int[] traverse(int pos) { + Set targets = new HashSet(); + traverse(targets, pos); + return toArray(targets); + } + + private int[] toArray(Set values) { + int index = 0; + int[] result = new int[values.size()]; + for (int value : values) { + result[index++] = value; + } + return result; + } + + private void traverse(Set targets, int pos) { + if (vector[pos] == null) { + return; + } + for (int target : vector[pos]) { + if (targets.add(target)) { + traverse(targets, target); + } + } + } + } +} diff --git ql/src/test/queries/clientpositive/dynamic_partition_pruning.q ql/src/test/queries/clientpositive/dynamic_partition_pruning.q new file mode 100644 index 0000000..f12b2c5 --- /dev/null +++ ql/src/test/queries/clientpositive/dynamic_partition_pruning.q @@ -0,0 +1,191 @@ +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; + + +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 as select ds as ds, ds as date from srcpart group by ds; +create table srcpart_hour as select hr as hr, hr as hour from srcpart group by hr; +create table srcpart_date_hour as select ds as ds, ds as date, hr as hr, hr as hour from srcpart group by ds, hr; +create table srcpart_double_hour 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/dynamic_partition_pruning.q.out ql/src/test/results/clientpositive/tez/dynamic_partition_pruning.q.out new file mode 100644 index 0000000..c0d9392 --- /dev/null +++ ql/src/test/results/clientpositive/tez/dynamic_partition_pruning.q.out @@ -0,0 +1,5351 @@ +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: 0 Data size: 23248 Basic stats: PARTIAL Column stats: COMPLETE + Select Operator + expressions: ds (type: string) + outputColumnNames: ds + Statistics: Num rows: 0 Data size: 23248 Basic stats: PARTIAL Column stats: COMPLETE + Group By Operator + keys: ds (type: string) + mode: hash + outputColumnNames: _col0 + Statistics: Num rows: 0 Data size: 23248 Basic stats: PARTIAL Column stats: COMPLETE + Reduce Output Operator + key expressions: _col0 (type: string) + sort order: + + Map-reduce partition columns: _col0 (type: string) + Statistics: Num rows: 0 Data size: 23248 Basic stats: PARTIAL Column stats: COMPLETE + Reducer 2 + Reduce Operator Tree: + Group By Operator + keys: KEY._col0 (type: string) + mode: mergepartial + outputColumnNames: _col0 + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: COMPLETE + Select Operator + expressions: _col0 (type: string), _col0 (type: string) + outputColumnNames: _col0, _col1 + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: COMPLETE + File Output Operator + compressed: false + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE 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 + + 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: 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 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 +POSTHOOK: query: create table srcpart_date 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: default@srcpart_date +PREHOOK: query: create table srcpart_hour 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 +POSTHOOK: query: create table srcpart_hour 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: default@srcpart_hour +PREHOOK: query: create table srcpart_date_hour as select ds as ds, ds as date, hr as hr, hr as hour from srcpart group by ds, hr +PREHOOK: type: CREATETABLE_AS_SELECT +PREHOOK: Input: default@srcpart +PREHOOK: Input: default@srcpart@ds=2008-04-08/hr=11 +PREHOOK: Input: default@srcpart@ds=2008-04-08/hr=12 +PREHOOK: Input: default@srcpart@ds=2008-04-09/hr=11 +PREHOOK: Input: default@srcpart@ds=2008-04-09/hr=12 +POSTHOOK: query: create table srcpart_date_hour as select ds as ds, ds as date, hr as hr, hr as hour from srcpart group by ds, hr +POSTHOOK: type: CREATETABLE_AS_SELECT +POSTHOOK: Input: default@srcpart +POSTHOOK: Input: default@srcpart@ds=2008-04-08/hr=11 +POSTHOOK: Input: default@srcpart@ds=2008-04-08/hr=12 +POSTHOOK: Input: default@srcpart@ds=2008-04-09/hr=11 +POSTHOOK: Input: default@srcpart@ds=2008-04-09/hr=12 +POSTHOOK: Output: default@srcpart_date_hour +PREHOOK: query: create table srcpart_double_hour 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 +POSTHOOK: query: create table srcpart_double_hour 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: 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: 0 Data size: 23248 Basic stats: PARTIAL Column stats: COMPLETE + Reduce Output Operator + key expressions: ds (type: string) + sort order: + + Map-reduce partition columns: ds (type: string) + Statistics: Num rows: 0 Data size: 23248 Basic stats: PARTIAL 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: 42 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 + Reducer 2 + Reduce Operator Tree: + Join Operator + condition map: + Inner Join 0 to 1 + condition expressions: + 0 + 1 + Statistics: Num rows: 0 Data size: 25572 Basic stats: PARTIAL Column stats: NONE + Select Operator + Statistics: Num rows: 0 Data size: 25572 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 + + 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: 0 Data size: 23248 Basic stats: PARTIAL Column stats: COMPLETE + Reduce Output Operator + key expressions: ds (type: string) + sort order: + + Map-reduce partition columns: ds (type: string) + Statistics: Num rows: 0 Data size: 23248 Basic stats: PARTIAL 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: 42 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 + Reducer 2 + Reduce Operator Tree: + Join Operator + condition map: + Inner Join 0 to 1 + condition expressions: + 0 + 1 + Statistics: Num rows: 0 Data size: 25572 Basic stats: PARTIAL Column stats: NONE + Select Operator + Statistics: Num rows: 0 Data size: 25572 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 + + 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: 0 Data size: 23248 Basic stats: PARTIAL Column stats: COMPLETE + Reduce Output Operator + key expressions: ds (type: string) + sort order: + + Map-reduce partition columns: ds (type: string) + Statistics: Num rows: 0 Data size: 23248 Basic stats: PARTIAL 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: 10 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: 42 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 + 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: 0 Data size: 25572 Basic stats: PARTIAL Column stats: NONE + Reduce Output Operator + key expressions: _col3 (type: string) + sort order: + + Map-reduce partition columns: _col3 (type: string) + Statistics: Num rows: 0 Data size: 25572 Basic stats: PARTIAL 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: 28129 Basic stats: PARTIAL Column stats: NONE + Select Operator + Statistics: Num rows: 0 Data size: 28129 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 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 + + 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: 0 Data size: 23248 Basic stats: PARTIAL Column stats: COMPLETE + Reduce Output Operator + key expressions: ds (type: string) + sort order: + + Map-reduce partition columns: ds (type: string) + Statistics: Num rows: 0 Data size: 23248 Basic stats: PARTIAL 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: 10 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: 42 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 + 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: 0 Data size: 25572 Basic stats: PARTIAL Column stats: NONE + Reduce Output Operator + key expressions: _col3 (type: string) + sort order: + + Map-reduce partition columns: _col3 (type: string) + Statistics: Num rows: 0 Data size: 25572 Basic stats: PARTIAL 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: 28129 Basic stats: PARTIAL Column stats: NONE + Select Operator + Statistics: Num rows: 0 Data size: 28129 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 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 + + 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: 0 Data size: 23248 Basic stats: PARTIAL 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: 0 Data size: 23248 Basic stats: PARTIAL 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: 108 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: 0 Data size: 25572 Basic stats: PARTIAL Column stats: NONE + Select Operator + Statistics: Num rows: 0 Data size: 25572 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 + + 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: 0 Data size: 23248 Basic stats: PARTIAL 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: 0 Data size: 23248 Basic stats: PARTIAL 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: 108 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: 0 Data size: 25572 Basic stats: PARTIAL Column stats: NONE + Select Operator + Statistics: Num rows: 0 Data size: 25572 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 + + 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: 0 Data size: 23248 Basic stats: PARTIAL Column stats: COMPLETE + Reduce Output Operator + key expressions: ds (type: string) + sort order: + + Map-reduce partition columns: ds (type: string) + Statistics: Num rows: 0 Data size: 23248 Basic stats: PARTIAL 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: 42 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 + Reducer 2 + Reduce Operator Tree: + Join Operator + condition map: + Inner Join 0 to 1 + condition expressions: + 0 + 1 + Statistics: Num rows: 0 Data size: 25572 Basic stats: PARTIAL Column stats: NONE + Select Operator + Statistics: Num rows: 0 Data size: 25572 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 + + 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: 0 Data size: 23248 Basic stats: PARTIAL Column stats: COMPLETE + Reduce Output Operator + key expressions: ds (type: string) + sort order: + + Map-reduce partition columns: ds (type: string) + Statistics: Num rows: 0 Data size: 23248 Basic stats: PARTIAL 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: 42 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 + Reducer 2 + Reduce Operator Tree: + Join Operator + condition map: + Inner Join 0 to 1 + condition expressions: + 0 + 1 + Statistics: Num rows: 0 Data size: 25572 Basic stats: PARTIAL Column stats: NONE + Select Operator + Statistics: Num rows: 0 Data size: 25572 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 + + 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: 0 Data size: 23248 Basic stats: PARTIAL Column stats: COMPLETE + Filter Operator + predicate: UDFToDouble(hr) is not null (type: boolean) + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE 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: 0 Data size: 0 Basic stats: NONE 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: 14 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: 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 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 + + 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: 0 Data size: 23248 Basic stats: PARTIAL Column stats: COMPLETE + Filter Operator + predicate: (hr * 2) is not null (type: boolean) + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE 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: 0 Data size: 0 Basic stats: NONE 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: 14 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: 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 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 + + 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: 0 Data size: 23248 Basic stats: PARTIAL Column stats: COMPLETE + Filter Operator + predicate: UDFToDouble(hr) is not null (type: boolean) + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE 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: 0 Data size: 0 Basic stats: NONE 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: 14 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: 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 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 + + 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: 0 Data size: 23248 Basic stats: PARTIAL Column stats: COMPLETE + Filter Operator + predicate: (hr * 2) is not null (type: boolean) + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE 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: 0 Data size: 0 Basic stats: NONE 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: 14 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: 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 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 + + 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: 0 Data size: 23248 Basic stats: PARTIAL Column stats: COMPLETE + Filter Operator + predicate: UDFToString((hr * 2)) is not null (type: boolean) + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE 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: 0 Data size: 0 Basic stats: NONE 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: 14 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: 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 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 + + 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: 0 Data size: 23248 Basic stats: PARTIAL Column stats: COMPLETE + Reduce Output Operator + key expressions: ds (type: string) + sort order: + + Map-reduce partition columns: ds (type: string) + Statistics: Num rows: 0 Data size: 23248 Basic stats: PARTIAL Column stats: COMPLETE + Select Operator + expressions: ds (type: string) + outputColumnNames: _col0 + Statistics: Num rows: 0 Data size: 23248 Basic stats: PARTIAL Column stats: COMPLETE + Group By Operator + keys: _col0 (type: string) + mode: hash + outputColumnNames: _col0 + Statistics: Num rows: 0 Data size: 23248 Basic stats: PARTIAL Column stats: COMPLETE + Dynamic Partitioning Event Operator + Target Input: srcpart + Partition key expr: ds + Statistics: Num rows: 0 Data size: 23248 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: 0 Data size: 11624 Basic stats: PARTIAL Column stats: COMPLETE + Select Operator + expressions: ds (type: string) + outputColumnNames: ds + Statistics: Num rows: 0 Data size: 11624 Basic stats: PARTIAL Column stats: COMPLETE + Group By Operator + keys: ds (type: string) + mode: hash + outputColumnNames: _col0 + Statistics: Num rows: 0 Data size: 11624 Basic stats: PARTIAL Column stats: COMPLETE + Reduce Output Operator + key expressions: _col0 (type: string) + sort order: + + Map-reduce partition columns: _col0 (type: string) + Statistics: Num rows: 0 Data size: 11624 Basic stats: PARTIAL Column stats: COMPLETE + Reducer 2 + Reduce Operator Tree: + Join Operator + condition map: + Inner Join 0 to 1 + condition expressions: + 0 + 1 + Statistics: Num rows: 0 Data size: 25572 Basic stats: PARTIAL Column stats: NONE + Select Operator + Statistics: Num rows: 0 Data size: 25572 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 + Reducer 5 + Reduce Operator Tree: + Group By Operator + keys: KEY._col0 (type: string) + mode: mergepartial + outputColumnNames: _col0 + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: COMPLETE + Select Operator + expressions: _col0 (type: string) + outputColumnNames: _col0 + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: COMPLETE + Reduce Output Operator + key expressions: _col0 (type: string) + sort order: + + Map-reduce partition columns: _col0 (type: string) + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: COMPLETE + + 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: 0 Data size: 23248 Basic stats: PARTIAL Column stats: COMPLETE + Reduce Output Operator + sort order: + Statistics: Num rows: 0 Data size: 23248 Basic stats: PARTIAL 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: 108 Basic stats: COMPLETE Column stats: NONE + Filter Operator + predicate: ((date = '2008-04-08') and (hour = 11)) (type: boolean) + Statistics: Num rows: 1 Data size: 27 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + sort order: + Statistics: Num rows: 1 Data size: 27 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: 1 Data size: 29 Basic stats: COMPLETE Column stats: NONE + Filter Operator + predicate: ((_col2 = _col7) or (_col3 = _col9)) (type: boolean) + 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 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 + + 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: 0 Data size: 23248 Basic stats: PARTIAL 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: 0 Data size: 23248 Basic stats: PARTIAL 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: 108 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: 0 Data size: 25572 Basic stats: PARTIAL Column stats: NONE + Filter Operator + predicate: ((_col2 = _col7) and (_col3 = _col9)) (type: boolean) + 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 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 + + 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: 0 Data size: 23248 Basic stats: PARTIAL Column stats: COMPLETE + Reduce Output Operator + key expressions: ds (type: string) + sort order: + + Map-reduce partition columns: ds (type: string) + Statistics: Num rows: 0 Data size: 23248 Basic stats: PARTIAL Column stats: COMPLETE + Map 4 + Map Operator Tree: + TableScan + alias: srcpart_date + Statistics: Num rows: 2 Data size: 42 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: 42 Basic stats: COMPLETE Column stats: NONE + value expressions: date (type: string) + 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: 2 Data size: 46 Basic stats: COMPLETE Column stats: NONE + Filter Operator + predicate: (_col8 = '2008-04-08') (type: boolean) + Statistics: Num rows: 1 Data size: 23 Basic stats: COMPLETE Column stats: NONE + Select Operator + Statistics: Num rows: 1 Data size: 23 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 + + 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: 0 Data size: 23248 Basic stats: PARTIAL Column stats: COMPLETE + Reduce Output Operator + key expressions: ds (type: string) + sort order: + + Map-reduce partition columns: ds (type: string) + Statistics: Num rows: 0 Data size: 23248 Basic stats: PARTIAL 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: 42 Basic stats: COMPLETE Column stats: NONE + Filter Operator + predicate: (date = '2008-04-08') (type: boolean) + Statistics: Num rows: 1 Data size: 21 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: 21 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: ds (type: string) + outputColumnNames: _col0 + Statistics: Num rows: 1 Data size: 21 Basic stats: COMPLETE Column stats: NONE + Group By Operator + keys: _col0 (type: string) + mode: hash + outputColumnNames: _col0 + Statistics: Num rows: 1 Data size: 21 Basic stats: COMPLETE Column stats: NONE + Dynamic Partitioning Event Operator + Target Input: srcpart + Partition key expr: ds + Statistics: Num rows: 1 Data size: 21 Basic stats: COMPLETE Column stats: NONE + Target column: ds + Target Vertex: Map 1 + Reducer 2 + Reduce Operator Tree: + Join Operator + condition map: + Left Outer Join0 to 1 + condition expressions: + 0 + 1 + Statistics: Num rows: 1 Data size: 23 Basic stats: COMPLETE Column stats: NONE + Select Operator + Statistics: Num rows: 1 Data size: 23 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 + + 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: 0 Data size: 23248 Basic stats: PARTIAL Column stats: COMPLETE + Reduce Output Operator + key expressions: ds (type: string) + sort order: + + Map-reduce partition columns: ds (type: string) + Statistics: Num rows: 0 Data size: 23248 Basic stats: PARTIAL Column stats: COMPLETE + Map 4 + Map Operator Tree: + TableScan + alias: srcpart_date + Statistics: Num rows: 2 Data size: 42 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: 42 Basic stats: COMPLETE Column stats: NONE + value expressions: date (type: string) + 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: 2 Data size: 46 Basic stats: COMPLETE Column stats: NONE + Filter Operator + predicate: (_col8 = '2008-04-08') (type: boolean) + Statistics: Num rows: 1 Data size: 23 Basic stats: COMPLETE Column stats: NONE + Select Operator + Statistics: Num rows: 1 Data size: 23 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 + + 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: 0 Data size: 11624 Basic stats: PARTIAL Column stats: COMPLETE + Reduce Output Operator + key expressions: ds (type: string) + sort order: + + Map-reduce partition columns: ds (type: string) + Statistics: Num rows: 0 Data size: 11624 Basic stats: PARTIAL 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: 10 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: 42 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 + 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: 0 Data size: 12786 Basic stats: PARTIAL Column stats: NONE + Reduce Output Operator + key expressions: _col3 (type: string) + sort order: + + Map-reduce partition columns: _col3 (type: string) + Statistics: Num rows: 0 Data size: 12786 Basic stats: PARTIAL 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: 14064 Basic stats: PARTIAL Column stats: NONE + Select Operator + Statistics: Num rows: 0 Data size: 14064 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 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 + + 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: 10 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: 42 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 + 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 + + 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: 0 Data size: 23248 Basic stats: PARTIAL Column stats: COMPLETE + Select Operator + expressions: ds (type: string) + outputColumnNames: ds + Statistics: Num rows: 0 Data size: 23248 Basic stats: PARTIAL 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: 0 Data size: 23248 Basic stats: PARTIAL Column stats: COMPLETE + Reduce Output Operator + key expressions: ds (type: string) + sort order: + + Map-reduce partition columns: ds (type: string) + Statistics: Num rows: 0 Data size: 23248 Basic stats: PARTIAL Column stats: COMPLETE + Map 7 + Map Operator Tree: + TableScan + alias: srcpart + Statistics: Num rows: 0 Data size: 23248 Basic stats: PARTIAL Column stats: COMPLETE + Select Operator + expressions: ds (type: string) + outputColumnNames: ds + Statistics: Num rows: 0 Data size: 23248 Basic stats: PARTIAL 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: 0 Data size: 25572 Basic stats: PARTIAL Column stats: NONE + Select Operator + Statistics: Num rows: 0 Data size: 25572 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 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: 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 + 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: 0 Data size: 23248 Basic stats: PARTIAL Column stats: COMPLETE + Select Operator + expressions: ds (type: string) + outputColumnNames: ds + Statistics: Num rows: 0 Data size: 23248 Basic stats: PARTIAL 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: 0 Data size: 23248 Basic stats: PARTIAL Column stats: COMPLETE + Reduce Output Operator + key expressions: ds (type: string) + sort order: + + Map-reduce partition columns: ds (type: string) + Statistics: Num rows: 0 Data size: 23248 Basic stats: PARTIAL Column stats: COMPLETE + Map 7 + Map Operator Tree: + TableScan + alias: srcpart + Statistics: Num rows: 0 Data size: 23248 Basic stats: PARTIAL Column stats: COMPLETE + Select Operator + expressions: ds (type: string) + outputColumnNames: ds + Statistics: Num rows: 0 Data size: 23248 Basic stats: PARTIAL 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: 0 Data size: 25572 Basic stats: PARTIAL Column stats: NONE + Select Operator + expressions: _col2 (type: string) + outputColumnNames: _col2 + Statistics: Num rows: 0 Data size: 25572 Basic stats: PARTIAL Column stats: NONE + Group By Operator + keys: _col2 (type: string) + mode: hash + outputColumnNames: _col0 + Statistics: Num rows: 0 Data size: 25572 Basic stats: PARTIAL Column stats: NONE + Reduce Output Operator + key expressions: _col0 (type: string) + sort order: + + Map-reduce partition columns: _col0 (type: string) + Statistics: Num rows: 0 Data size: 25572 Basic stats: PARTIAL Column stats: NONE + Reducer 5 + Reduce Operator Tree: + Group By Operator + keys: KEY._col0 (type: string) + mode: mergepartial + outputColumnNames: _col0 + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Select Operator + expressions: _col0 (type: string) + outputColumnNames: _col0 + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + File Output Operator + compressed: false + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE 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 + 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: 0 Data size: 23248 Basic stats: PARTIAL Column stats: COMPLETE + Select Operator + expressions: ds (type: string) + outputColumnNames: ds + Statistics: Num rows: 0 Data size: 23248 Basic stats: PARTIAL 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: 0 Data size: 23248 Basic stats: PARTIAL Column stats: COMPLETE + Select Operator + expressions: ds (type: string) + outputColumnNames: ds + Statistics: Num rows: 0 Data size: 23248 Basic stats: PARTIAL 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: 0 Data size: 23248 Basic stats: PARTIAL Column stats: COMPLETE + Select Operator + expressions: ds (type: string) + outputColumnNames: ds + Statistics: Num rows: 0 Data size: 23248 Basic stats: PARTIAL Column stats: COMPLETE + Group By Operator + keys: ds (type: string) + mode: hash + outputColumnNames: _col0 + Statistics: Num rows: 0 Data size: 23248 Basic stats: PARTIAL Column stats: COMPLETE + Reduce Output Operator + key expressions: _col0 (type: string) + sort order: + + Map-reduce partition columns: _col0 (type: string) + Statistics: Num rows: 0 Data size: 23248 Basic stats: PARTIAL Column stats: COMPLETE + Map 8 + Map Operator Tree: + TableScan + alias: srcpart + filterExpr: ds is not null (type: boolean) + Statistics: Num rows: 0 Data size: 23248 Basic stats: PARTIAL Column stats: COMPLETE + Select Operator + expressions: ds (type: string) + outputColumnNames: ds + Statistics: Num rows: 0 Data size: 23248 Basic stats: PARTIAL Column stats: COMPLETE + Group By Operator + keys: ds (type: string) + mode: hash + outputColumnNames: _col0 + Statistics: Num rows: 0 Data size: 23248 Basic stats: PARTIAL Column stats: COMPLETE + Reduce Output Operator + key expressions: _col0 (type: string) + sort order: + + Map-reduce partition columns: _col0 (type: string) + Statistics: Num rows: 0 Data size: 23248 Basic stats: PARTIAL 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: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Select Operator + expressions: _col0 (type: string) + outputColumnNames: _col0 + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + File Output Operator + compressed: false + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE 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 + 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) + 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) + 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: 0 Data size: 23248 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) + 1 ds (type: string) + Statistics: Num rows: 0 Data size: 25572 Basic stats: PARTIAL Column stats: NONE + Select Operator + Statistics: Num rows: 0 Data size: 25572 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 + filterExpr: (ds is not null and (date = '2008-04-08')) (type: boolean) + Statistics: Num rows: 2 Data size: 42 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 + 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 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: 0 Data size: 23248 Basic stats: PARTIAL 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: 0 Data size: 25572 Basic stats: PARTIAL 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: 0 Data size: 28129 Basic stats: PARTIAL Column stats: NONE + Select Operator + Statistics: Num rows: 0 Data size: 28129 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_hour + filterExpr: (hr is not null and (hour = 11)) (type: boolean) + Statistics: Num rows: 2 Data size: 10 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: 42 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 + 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 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: 0 Data size: 23248 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), hr (type: string) + 1 ds (type: string), hr (type: string) + Statistics: Num rows: 0 Data size: 25572 Basic stats: PARTIAL Column stats: NONE + Select Operator + Statistics: Num rows: 0 Data size: 25572 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 hr is not null) and (date = '2008-04-08')) and (hour = 11)) (type: boolean) + Statistics: Num rows: 4 Data size: 108 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 + + 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: 0 Data size: 23248 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) + 1 ds (type: string) + Statistics: Num rows: 0 Data size: 25572 Basic stats: PARTIAL Column stats: NONE + Select Operator + Statistics: Num rows: 0 Data size: 25572 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 + filterExpr: (ds is not null and (date = 'I DONT EXIST')) (type: boolean) + Statistics: Num rows: 2 Data size: 42 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 + 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: -- 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: 0 Data size: 23248 Basic stats: PARTIAL Column stats: COMPLETE + Filter Operator + predicate: UDFToDouble(hr) is not null (type: boolean) + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE 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: 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) + 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: 14 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 + + 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: 0 Data size: 23248 Basic stats: PARTIAL Column stats: COMPLETE + Filter Operator + predicate: (hr * 2) is not null (type: boolean) + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE 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: 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) + 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: 14 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 + + 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: + Map 1 <- Reducer 4 (BROADCAST_EDGE) + Reducer 2 <- Map 1 (SIMPLE_EDGE) + Reducer 4 <- Map 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: 0 Data size: 23248 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) + 1 _col0 (type: string) + Statistics: Num rows: 0 Data size: 25572 Basic stats: PARTIAL Column stats: NONE + Select Operator + Statistics: Num rows: 0 Data size: 25572 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 + filterExpr: (ds is not null and (ds = '2008-04-08')) (type: boolean) + Statistics: Num rows: 0 Data size: 11624 Basic stats: PARTIAL Column stats: COMPLETE + Select Operator + expressions: ds (type: string) + outputColumnNames: ds + Statistics: Num rows: 0 Data size: 11624 Basic stats: PARTIAL Column stats: COMPLETE + Group By Operator + keys: ds (type: string) + mode: hash + outputColumnNames: _col0 + Statistics: Num rows: 0 Data size: 11624 Basic stats: PARTIAL Column stats: COMPLETE + Reduce Output Operator + key expressions: _col0 (type: string) + sort order: + + Map-reduce partition columns: _col0 (type: string) + Statistics: Num rows: 0 Data size: 11624 Basic stats: PARTIAL Column stats: COMPLETE + 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 + Reducer 4 + Reduce Operator Tree: + Group By Operator + keys: KEY._col0 (type: string) + mode: mergepartial + outputColumnNames: _col0 + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: COMPLETE + Select Operator + expressions: _col0 (type: string) + outputColumnNames: _col0 + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: COMPLETE + Reduce Output Operator + key expressions: _col0 (type: string) + sort order: + + Map-reduce partition columns: _col0 (type: string) + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: COMPLETE + Select Operator + expressions: _col0 (type: string) + outputColumnNames: _col0 + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: COMPLETE + Group By Operator + keys: _col0 (type: string) + mode: hash + outputColumnNames: _col0 + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: COMPLETE + Dynamic Partitioning Event Operator + Target Input: srcpart + Partition key expr: ds + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: COMPLETE + Target column: ds + Target Vertex: Map 1 + + 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: 0 Data size: 23248 Basic stats: PARTIAL 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: 2 Data size: 46 Basic stats: COMPLETE Column stats: NONE + Filter Operator + predicate: (_col8 = '2008-04-08') (type: boolean) + Statistics: Num rows: 1 Data size: 23 Basic stats: COMPLETE Column stats: NONE + Select Operator + Statistics: Num rows: 1 Data size: 23 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: 42 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: 42 Basic stats: COMPLETE Column stats: NONE + value expressions: date (type: string) + 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: 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: 0 Data size: 23248 Basic stats: PARTIAL Column stats: COMPLETE + Reduce Output Operator + key expressions: ds (type: string) + sort order: + + Map-reduce partition columns: ds (type: string) + Statistics: Num rows: 0 Data size: 23248 Basic stats: PARTIAL 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: 42 Basic stats: COMPLETE Column stats: NONE + Filter Operator + predicate: (date = '2008-04-08') (type: boolean) + Statistics: Num rows: 1 Data size: 21 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: 1 Data size: 23 Basic stats: COMPLETE Column stats: NONE + Select Operator + Statistics: Num rows: 1 Data size: 23 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 + + 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: 0 Data size: 23248 Basic stats: PARTIAL Column stats: COMPLETE + Reduce Output Operator + key expressions: ds (type: string) + sort order: + + Map-reduce partition columns: ds (type: string) + Statistics: Num rows: 0 Data size: 23248 Basic stats: PARTIAL Column stats: COMPLETE + Map 4 + Map Operator Tree: + TableScan + alias: srcpart_date + Statistics: Num rows: 2 Data size: 42 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: 42 Basic stats: COMPLETE Column stats: NONE + value expressions: date (type: string) + 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: 2 Data size: 46 Basic stats: COMPLETE Column stats: NONE + Filter Operator + predicate: (_col8 = '2008-04-08') (type: boolean) + Statistics: Num rows: 1 Data size: 23 Basic stats: COMPLETE Column stats: NONE + Select Operator + Statistics: Num rows: 1 Data size: 23 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 + + 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: 0 Data size: 11624 Basic stats: PARTIAL 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: 0 Data size: 12786 Basic stats: PARTIAL 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: 0 Data size: 14064 Basic stats: PARTIAL Column stats: NONE + Select Operator + Statistics: Num rows: 0 Data size: 14064 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_hour + filterExpr: ((hr is not null and (hour = 11)) and (hr = 11)) (type: boolean) + Statistics: Num rows: 2 Data size: 10 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: 42 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 + 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 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: 10 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: 42 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 + 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: -- 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: 0 Data size: 23248 Basic stats: PARTIAL Column stats: COMPLETE + Select Operator + expressions: ds (type: string) + outputColumnNames: ds + Statistics: Num rows: 0 Data size: 23248 Basic stats: PARTIAL 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: 0 Data size: 23248 Basic stats: PARTIAL 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: 0 Data size: 25572 Basic stats: PARTIAL Column stats: NONE + Select Operator + expressions: _col2 (type: string) + outputColumnNames: _col2 + Statistics: Num rows: 0 Data size: 25572 Basic stats: PARTIAL Column stats: NONE + Group By Operator + keys: _col2 (type: string) + mode: hash + outputColumnNames: _col0 + Statistics: Num rows: 0 Data size: 25572 Basic stats: PARTIAL Column stats: NONE + Reduce Output Operator + key expressions: _col0 (type: string) + sort order: + + Map-reduce partition columns: _col0 (type: string) + Statistics: Num rows: 0 Data size: 25572 Basic stats: PARTIAL Column stats: NONE + Map 6 + Map Operator Tree: + TableScan + alias: srcpart + Statistics: Num rows: 0 Data size: 23248 Basic stats: PARTIAL Column stats: COMPLETE + Select Operator + expressions: ds (type: string) + outputColumnNames: ds + Statistics: Num rows: 0 Data size: 23248 Basic stats: PARTIAL 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: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Select Operator + expressions: _col0 (type: string) + outputColumnNames: _col0 + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + File Output Operator + compressed: false + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE 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 + 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: 108 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 diff --git service/src/gen/thrift/gen-py/TCLIService/TCLIService-remote service/src/gen/thrift/gen-py/TCLIService/TCLIService-remote old mode 100644 new mode 100755 diff --git service/src/gen/thrift/gen-py/hive_service/ThriftHive-remote service/src/gen/thrift/gen-py/hive_service/ThriftHive-remote old mode 100644 new mode 100755 diff --git shims/0.23/src/main/java/org/apache/hadoop/hive/shims/Hadoop23Shims.java shims/0.23/src/main/java/org/apache/hadoop/hive/shims/Hadoop23Shims.java index 6f4820e..6564c09 100644 --- shims/0.23/src/main/java/org/apache/hadoop/hive/shims/Hadoop23Shims.java +++ shims/0.23/src/main/java/org/apache/hadoop/hive/shims/Hadoop23Shims.java @@ -302,6 +302,7 @@ public MiniTezShim(Configuration conf, int numberOfTaskTrackers, mr = new MiniTezCluster("hive", numberOfTaskTrackers); conf.set("fs.defaultFS", nameNode); + conf.set("tez.am.log.level", "DEBUG"); conf.set(MRJobConfig.MR_AM_STAGING_DIR, "/apps_staging_dir"); mr.init(conf); mr.start();