You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hive.apache.org by gu...@apache.org on 2014/09/03 12:46:06 UTC

svn commit: r1622216 [1/4] - in /hive/branches/tez: common/src/java/org/apache/hadoop/hive/conf/ data/files/ itests/qtest/ ql/if/ ql/src/gen/thrift/gen-cpp/ ql/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/ql/plan/api/ ql/src/gen/thrift/gen-php/ q...

Author: gunther
Date: Wed Sep  3 10:46:04 2014
New Revision: 1622216

URL: http://svn.apache.org/r1622216
Log:
HIVE-7826: Dynamic partition pruning on Tez (Gunther Hagleitner, reviewed by Vikram Dixit K)

Added:
    hive/branches/tez/data/files/agg_01-p1.txt
    hive/branches/tez/data/files/agg_01-p2.txt
    hive/branches/tez/data/files/agg_01-p3.txt
    hive/branches/tez/data/files/dim_shops.txt
    hive/branches/tez/ql/src/java/org/apache/hadoop/hive/ql/exec/AppMasterEventOperator.java
    hive/branches/tez/ql/src/java/org/apache/hadoop/hive/ql/exec/tez/DynamicPartitionPruner.java
    hive/branches/tez/ql/src/java/org/apache/hadoop/hive/ql/optimizer/DynamicPartitionPruningOptimization.java
    hive/branches/tez/ql/src/java/org/apache/hadoop/hive/ql/optimizer/RemoveDynamicPruningBySize.java
    hive/branches/tez/ql/src/java/org/apache/hadoop/hive/ql/parse/AppMasterEventProcessor.java
    hive/branches/tez/ql/src/java/org/apache/hadoop/hive/ql/parse/UnionProcessor.java
    hive/branches/tez/ql/src/java/org/apache/hadoop/hive/ql/plan/AppMasterEventDesc.java
    hive/branches/tez/ql/src/java/org/apache/hadoop/hive/ql/plan/DynamicPruningEventDesc.java
    hive/branches/tez/ql/src/java/org/apache/hadoop/hive/ql/plan/ExprNodeDynamicListDesc.java
    hive/branches/tez/ql/src/java/org/apache/hadoop/hive/ql/ppd/SyntheticJoinPredicate.java
    hive/branches/tez/ql/src/test/queries/clientpositive/dynamic_partition_pruning.q
    hive/branches/tez/ql/src/test/queries/clientpositive/dynamic_partition_pruning_2.q
    hive/branches/tez/ql/src/test/results/clientpositive/tez/dynamic_partition_pruning.q.out
    hive/branches/tez/ql/src/test/results/clientpositive/tez/dynamic_partition_pruning_2.q.out
Modified:
    hive/branches/tez/common/src/java/org/apache/hadoop/hive/conf/HiveConf.java
    hive/branches/tez/itests/qtest/testconfiguration.properties
    hive/branches/tez/ql/if/queryplan.thrift
    hive/branches/tez/ql/src/gen/thrift/gen-cpp/queryplan_types.cpp
    hive/branches/tez/ql/src/gen/thrift/gen-cpp/queryplan_types.h
    hive/branches/tez/ql/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/ql/plan/api/OperatorType.java
    hive/branches/tez/ql/src/gen/thrift/gen-php/Types.php
    hive/branches/tez/ql/src/gen/thrift/gen-py/queryplan/ttypes.py
    hive/branches/tez/ql/src/gen/thrift/gen-rb/queryplan_types.rb
    hive/branches/tez/ql/src/java/org/apache/hadoop/hive/ql/exec/OperatorFactory.java
    hive/branches/tez/ql/src/java/org/apache/hadoop/hive/ql/exec/Utilities.java
    hive/branches/tez/ql/src/java/org/apache/hadoop/hive/ql/exec/tez/HiveSplitGenerator.java
    hive/branches/tez/ql/src/java/org/apache/hadoop/hive/ql/exec/tez/MapRecordProcessor.java
    hive/branches/tez/ql/src/java/org/apache/hadoop/hive/ql/exec/tez/ReduceRecordProcessor.java
    hive/branches/tez/ql/src/java/org/apache/hadoop/hive/ql/exec/tez/TezContext.java
    hive/branches/tez/ql/src/java/org/apache/hadoop/hive/ql/optimizer/ConstantPropagateProcFactory.java
    hive/branches/tez/ql/src/java/org/apache/hadoop/hive/ql/optimizer/ConvertJoinMapJoin.java
    hive/branches/tez/ql/src/java/org/apache/hadoop/hive/ql/optimizer/Optimizer.java
    hive/branches/tez/ql/src/java/org/apache/hadoop/hive/ql/optimizer/PrunerExpressionOperatorFactory.java
    hive/branches/tez/ql/src/java/org/apache/hadoop/hive/ql/optimizer/ReduceSinkMapJoinProc.java
    hive/branches/tez/ql/src/java/org/apache/hadoop/hive/ql/optimizer/pcr/PcrExprProcFactory.java
    hive/branches/tez/ql/src/java/org/apache/hadoop/hive/ql/optimizer/stats/annotation/StatsRulesProcFactory.java
    hive/branches/tez/ql/src/java/org/apache/hadoop/hive/ql/parse/FileSinkProcessor.java
    hive/branches/tez/ql/src/java/org/apache/hadoop/hive/ql/parse/GenTezProcContext.java
    hive/branches/tez/ql/src/java/org/apache/hadoop/hive/ql/parse/GenTezUtils.java
    hive/branches/tez/ql/src/java/org/apache/hadoop/hive/ql/parse/OptimizeTezProcContext.java
    hive/branches/tez/ql/src/java/org/apache/hadoop/hive/ql/parse/TezCompiler.java
    hive/branches/tez/ql/src/java/org/apache/hadoop/hive/ql/plan/MapWork.java
    hive/branches/tez/ql/src/java/org/apache/hadoop/hive/ql/plan/ReduceWork.java
    hive/branches/tez/ql/src/java/org/apache/hadoop/hive/ql/ppd/OpProcFactory.java
    hive/branches/tez/ql/src/test/results/clientpositive/tez/bucket2.q.out
    hive/branches/tez/ql/src/test/results/clientpositive/tez/bucket3.q.out
    hive/branches/tez/ql/src/test/results/clientpositive/tez/bucket4.q.out
    hive/branches/tez/ql/src/test/results/clientpositive/tez/cross_product_check_2.q.out
    hive/branches/tez/ql/src/test/results/clientpositive/tez/metadataonly1.q.out
    hive/branches/tez/ql/src/test/results/clientpositive/tez/optimize_nullscan.q.out
    hive/branches/tez/ql/src/test/results/clientpositive/tez/select_dummy_source.q.out
    hive/branches/tez/ql/src/test/results/clientpositive/tez/temp_table.q.out
    hive/branches/tez/ql/src/test/results/clientpositive/tez/vector_string_concat.q.out
    hive/branches/tez/ql/src/test/results/clientpositive/tez/vectorization_short_regress.q.out
    hive/branches/tez/shims/0.23/src/main/java/org/apache/hadoop/hive/shims/Hadoop23Shims.java

Modified: hive/branches/tez/common/src/java/org/apache/hadoop/hive/conf/HiveConf.java
URL: http://svn.apache.org/viewvc/hive/branches/tez/common/src/java/org/apache/hadoop/hive/conf/HiveConf.java?rev=1622216&r1=1622215&r2=1622216&view=diff
==============================================================================
--- hive/branches/tez/common/src/java/org/apache/hadoop/hive/conf/HiveConf.java (original)
+++ hive/branches/tez/common/src/java/org/apache/hadoop/hive/conf/HiveConf.java Wed Sep  3 10:46:04 2014
@@ -370,7 +370,6 @@ public class HiveConf extends Configurat
     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", "1000ms",
@@ -1774,7 +1773,15 @@ public class HiveConf extends Configurat
         "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;

Added: hive/branches/tez/data/files/agg_01-p1.txt
URL: http://svn.apache.org/viewvc/hive/branches/tez/data/files/agg_01-p1.txt?rev=1622216&view=auto
==============================================================================
--- hive/branches/tez/data/files/agg_01-p1.txt (added)
+++ hive/branches/tez/data/files/agg_01-p1.txt Wed Sep  3 10:46:04 2014
@@ -0,0 +1,3 @@
+1.0
+2.0
+3.0

Added: hive/branches/tez/data/files/agg_01-p2.txt
URL: http://svn.apache.org/viewvc/hive/branches/tez/data/files/agg_01-p2.txt?rev=1622216&view=auto
==============================================================================
--- hive/branches/tez/data/files/agg_01-p2.txt (added)
+++ hive/branches/tez/data/files/agg_01-p2.txt Wed Sep  3 10:46:04 2014
@@ -0,0 +1,3 @@
+4.0
+5.0
+6.0

Added: hive/branches/tez/data/files/agg_01-p3.txt
URL: http://svn.apache.org/viewvc/hive/branches/tez/data/files/agg_01-p3.txt?rev=1622216&view=auto
==============================================================================
--- hive/branches/tez/data/files/agg_01-p3.txt (added)
+++ hive/branches/tez/data/files/agg_01-p3.txt Wed Sep  3 10:46:04 2014
@@ -0,0 +1,3 @@
+7.0
+8.0
+9.0

Added: hive/branches/tez/data/files/dim_shops.txt
URL: http://svn.apache.org/viewvc/hive/branches/tez/data/files/dim_shops.txt?rev=1622216&view=auto
==============================================================================
--- hive/branches/tez/data/files/dim_shops.txt (added)
+++ hive/branches/tez/data/files/dim_shops.txt Wed Sep  3 10:46:04 2014
@@ -0,0 +1,3 @@
+1,foo
+2,bar
+3,baz

Modified: hive/branches/tez/itests/qtest/testconfiguration.properties
URL: http://svn.apache.org/viewvc/hive/branches/tez/itests/qtest/testconfiguration.properties?rev=1622216&r1=1622215&r2=1622216&view=diff
==============================================================================
--- hive/branches/tez/itests/qtest/testconfiguration.properties (original)
+++ hive/branches/tez/itests/qtest/testconfiguration.properties Wed Sep  3 10:46:04 2014
@@ -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_t
 able.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_overwr
 ite_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

Modified: hive/branches/tez/ql/if/queryplan.thrift
URL: http://svn.apache.org/viewvc/hive/branches/tez/ql/if/queryplan.thrift?rev=1622216&r1=1622215&r2=1622216&view=diff
==============================================================================
--- hive/branches/tez/ql/if/queryplan.thrift (original)
+++ hive/branches/tez/ql/if/queryplan.thrift Wed Sep  3 10:46:04 2014
@@ -56,6 +56,7 @@ enum OperatorType {
   PTF,
   MUX,
   DEMUX,
+  EVENT,
 }
 
 struct Operator {

Modified: hive/branches/tez/ql/src/gen/thrift/gen-cpp/queryplan_types.cpp
URL: http://svn.apache.org/viewvc/hive/branches/tez/ql/src/gen/thrift/gen-cpp/queryplan_types.cpp?rev=1622216&r1=1622215&r2=1622216&view=diff
==============================================================================
--- hive/branches/tez/ql/src/gen/thrift/gen-cpp/queryplan_types.cpp (original)
+++ hive/branches/tez/ql/src/gen/thrift/gen-cpp/queryplan_types.cpp Wed Sep  3 10:46:04 2014
@@ -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<int, const char*> _OperatorType_VALUES_TO_NAMES(::apache::thrift::TEnumIterator(21, _kOperatorTypeValues, _kOperatorTypeNames), ::apache::thrift::TEnumIterator(-1, NULL, NULL));
+const std::map<int, const char*> _OperatorType_VALUES_TO_NAMES(::apache::thrift::TEnumIterator(22, _kOperatorTypeValues, _kOperatorTypeNames), ::apache::thrift::TEnumIterator(-1, NULL, NULL));
 
 int _kTaskTypeValues[] = {
   TaskType::MAP,

Modified: hive/branches/tez/ql/src/gen/thrift/gen-cpp/queryplan_types.h
URL: http://svn.apache.org/viewvc/hive/branches/tez/ql/src/gen/thrift/gen-cpp/queryplan_types.h?rev=1622216&r1=1622215&r2=1622216&view=diff
==============================================================================
--- hive/branches/tez/ql/src/gen/thrift/gen-cpp/queryplan_types.h (original)
+++ hive/branches/tez/ql/src/gen/thrift/gen-cpp/queryplan_types.h Wed Sep  3 10:46:04 2014
@@ -56,7 +56,8 @@ struct OperatorType {
     HASHTABLEDUMMY = 17,
     PTF = 18,
     MUX = 19,
-    DEMUX = 20
+    DEMUX = 20,
+    EVENT = 21
   };
 };
 

Modified: hive/branches/tez/ql/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/ql/plan/api/OperatorType.java
URL: http://svn.apache.org/viewvc/hive/branches/tez/ql/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/ql/plan/api/OperatorType.java?rev=1622216&r1=1622215&r2=1622216&view=diff
==============================================================================
--- hive/branches/tez/ql/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/ql/plan/api/OperatorType.java (original)
+++ hive/branches/tez/ql/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/ql/plan/api/OperatorType.java Wed Sep  3 10:46:04 2014
@@ -32,7 +32,8 @@ public enum OperatorType implements org.
   HASHTABLEDUMMY(17),
   PTF(18),
   MUX(19),
-  DEMUX(20);
+  DEMUX(20),
+  EVENT(21);
 
   private final int value;
 
@@ -95,6 +96,8 @@ public enum OperatorType implements org.
         return MUX;
       case 20:
         return DEMUX;
+      case 21:
+        return EVENT;
       default:
         return null;
     }

Modified: hive/branches/tez/ql/src/gen/thrift/gen-php/Types.php
URL: http://svn.apache.org/viewvc/hive/branches/tez/ql/src/gen/thrift/gen-php/Types.php?rev=1622216&r1=1622215&r2=1622216&view=diff
==============================================================================
--- hive/branches/tez/ql/src/gen/thrift/gen-php/Types.php (original)
+++ hive/branches/tez/ql/src/gen/thrift/gen-php/Types.php Wed Sep  3 10:46:04 2014
@@ -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',
   );
 }
 

Modified: hive/branches/tez/ql/src/gen/thrift/gen-py/queryplan/ttypes.py
URL: http://svn.apache.org/viewvc/hive/branches/tez/ql/src/gen/thrift/gen-py/queryplan/ttypes.py?rev=1622216&r1=1622215&r2=1622216&view=diff
==============================================================================
--- hive/branches/tez/ql/src/gen/thrift/gen-py/queryplan/ttypes.py (original)
+++ hive/branches/tez/ql/src/gen/thrift/gen-py/queryplan/ttypes.py Wed Sep  3 10:46:04 2014
@@ -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:

Modified: hive/branches/tez/ql/src/gen/thrift/gen-rb/queryplan_types.rb
URL: http://svn.apache.org/viewvc/hive/branches/tez/ql/src/gen/thrift/gen-rb/queryplan_types.rb?rev=1622216&r1=1622215&r2=1622216&view=diff
==============================================================================
--- hive/branches/tez/ql/src/gen/thrift/gen-rb/queryplan_types.rb (original)
+++ hive/branches/tez/ql/src/gen/thrift/gen-rb/queryplan_types.rb Wed Sep  3 10:46:04 2014
@@ -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

Added: hive/branches/tez/ql/src/java/org/apache/hadoop/hive/ql/exec/AppMasterEventOperator.java
URL: http://svn.apache.org/viewvc/hive/branches/tez/ql/src/java/org/apache/hadoop/hive/ql/exec/AppMasterEventOperator.java?rev=1622216&view=auto
==============================================================================
--- hive/branches/tez/ql/src/java/org/apache/hadoop/hive/ql/exec/AppMasterEventOperator.java (added)
+++ hive/branches/tez/ql/src/java/org/apache/hadoop/hive/ql/exec/AppMasterEventOperator.java Wed Sep  3 10:46:04 2014
@@ -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<AppMasterEventDesc> {
+
+  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";
+  }
+}

Modified: hive/branches/tez/ql/src/java/org/apache/hadoop/hive/ql/exec/OperatorFactory.java
URL: http://svn.apache.org/viewvc/hive/branches/tez/ql/src/java/org/apache/hadoop/hive/ql/exec/OperatorFactory.java?rev=1622216&r1=1622215&r2=1622216&view=diff
==============================================================================
--- hive/branches/tez/ql/src/java/org/apache/hadoop/hive/ql/exec/OperatorFactory.java (original)
+++ hive/branches/tez/ql/src/java/org/apache/hadoop/hive/ql/exec/OperatorFactory.java Wed Sep  3 10:46:04 2014
@@ -29,13 +29,15 @@ import org.apache.hadoop.hive.ql.exec.ve
 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 @@ import org.apache.hadoop.hive.ql.plan.Un
  * OperatorFactory.
  *
  */
+@SuppressWarnings({ "rawtypes", "unchecked" })
 public final class OperatorFactory {
   private static final List<OpTuple> opvec;
   private static final List<OpTuple> vectorOpvec;
@@ -101,6 +104,10 @@ public final class OperatorFactory {
         DemuxOperator.class));
     opvec.add(new OpTuple<MuxDesc>(MuxDesc.class,
         MuxOperator.class));
+    opvec.add(new OpTuple<AppMasterEventDesc>(AppMasterEventDesc.class,
+        AppMasterEventOperator.class));
+    opvec.add(new OpTuple<DynamicPruningEventDesc>(DynamicPruningEventDesc.class,
+        AppMasterEventOperator.class));
   }
 
   static {
@@ -119,9 +126,9 @@ public final class OperatorFactory {
 
   private static final class OpTuple<T extends OperatorDesc> {
     private final Class<T> descClass;
-    private final Class<? extends Operator<T>> opClass;
+    private final Class<? extends Operator<?>> opClass;
 
-    public OpTuple(Class<T> descClass, Class<? extends Operator<T>> opClass) {
+    public OpTuple(Class<T> descClass, Class<? extends Operator<?>> opClass) {
       this.descClass = descClass;
       this.opClass = opClass;
     }

Modified: hive/branches/tez/ql/src/java/org/apache/hadoop/hive/ql/exec/Utilities.java
URL: http://svn.apache.org/viewvc/hive/branches/tez/ql/src/java/org/apache/hadoop/hive/ql/exec/Utilities.java?rev=1622216&r1=1622215&r2=1622216&view=diff
==============================================================================
--- hive/branches/tez/ql/src/java/org/apache/hadoop/hive/ql/exec/Utilities.java (original)
+++ hive/branches/tez/ql/src/java/org/apache/hadoop/hive/ql/exec/Utilities.java Wed Sep  3 10:46:04 2014
@@ -819,10 +819,12 @@ public final class Utilities {
     }
   }
 
-  public static Set<Operator<?>> cloneOperatorTree(Configuration conf, Set<Operator<?>> roots) {
+  public static List<Operator<?>> cloneOperatorTree(Configuration conf, List<Operator<?>> roots) {
     ByteArrayOutputStream baos = new ByteArrayOutputStream(4096);
     serializePlan(roots, baos, conf, true);
-    Set<Operator<?>> result = deserializePlan(new ByteArrayInputStream(baos.toByteArray()),
+    @SuppressWarnings("unchecked")
+    List<Operator<?>> result =
+        deserializePlan(new ByteArrayInputStream(baos.toByteArray()),
         roots.getClass(), conf, true);
     return result;
   }

Added: hive/branches/tez/ql/src/java/org/apache/hadoop/hive/ql/exec/tez/DynamicPartitionPruner.java
URL: http://svn.apache.org/viewvc/hive/branches/tez/ql/src/java/org/apache/hadoop/hive/ql/exec/tez/DynamicPartitionPruner.java?rev=1622216&view=auto
==============================================================================
--- hive/branches/tez/ql/src/java/org/apache/hadoop/hive/ql/exec/tez/DynamicPartitionPruner.java (added)
+++ hive/branches/tez/ql/src/java/org/apache/hadoop/hive/ql/exec/tez/DynamicPartitionPruner.java Wed Sep  3 10:46:04 2014
@@ -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<String, List<SourceInfo>> sourceInfoMap =
+      new HashMap<String, List<SourceInfo>>();
+
+  private final BytesWritable writable = new BytesWritable();
+
+  private final BlockingQueue<InputInitializerEvent> queue =
+      new LinkedBlockingQueue<InputInitializerEvent>();
+
+  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<InputInitializerEvent> getQueue() {
+    return queue;
+  }
+
+  private void clear() {
+    sourceInfoMap.clear();
+    sourceInfoCount = 0;
+  }
+
+  private void initialize(MapWork work, JobConf jobConf) throws SerDeException {
+    this.clear();
+    Map<String, SourceInfo> columnMap = new HashMap<String, SourceInfo>();
+
+    for (String s : work.getEventSourceTableDescMap().keySet()) {
+      List<TableDesc> tables = work.getEventSourceTableDescMap().get(s);
+      List<String> columnNames = work.getEventSourceColumnNameMap().get(s);
+      List<ExprNodeDesc> partKeyExprs = work.getEventSourcePartKeyExprMap().get(s);
+
+      Iterator<String> cit = columnNames.iterator();
+      Iterator<ExprNodeDesc> 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<SourceInfo>());
+        }
+        List<SourceInfo> 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<Object> 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<Object> values) throws HiveException {
+
+    Object[] row = new Object[1];
+
+    Iterator<String> it = work.getPathToPartitionInfo().keySet().iterator();
+    while (it.hasNext()) {
+      String p = it.next();
+      PartitionDesc desc = work.getPathToPartitionInfo().get(p);
+      Map<String, String> 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<Object> values = new HashSet<Object>();
+    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<? extends StructField> 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<SourceInfo> 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;
+    }
+  }
+
+}

Modified: hive/branches/tez/ql/src/java/org/apache/hadoop/hive/ql/exec/tez/HiveSplitGenerator.java
URL: http://svn.apache.org/viewvc/hive/branches/tez/ql/src/java/org/apache/hadoop/hive/ql/exec/tez/HiveSplitGenerator.java?rev=1622216&r1=1622215&r2=1622216&view=diff
==============================================================================
--- hive/branches/tez/ql/src/java/org/apache/hadoop/hive/ql/exec/tez/HiveSplitGenerator.java (original)
+++ hive/branches/tez/ql/src/java/org/apache/hadoop/hive/ql/exec/tez/HiveSplitGenerator.java Wed Sep  3 10:46:04 2014
@@ -46,12 +46,12 @@ import org.apache.tez.mapreduce.protos.M
 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 @@ import com.google.common.collect.Multima
  * 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 class HiveSplitGenerator extends 
   public List<Event> initialize() throws Exception {
     InputInitializerContext rootInputContext = getContext();
 
+    context = rootInputContext;
+
     MRInputUserPayloadProto userPayloadProto =
         MRInputHelpers.parseMRInputPayload(rootInputContext.getInputUserPayload());
 
@@ -90,6 +95,11 @@ public class HiveSplitGenerator extends 
     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 class HiveSplitGenerator extends 
     return createEventList(sendSerializedEvents, inputSplitInfo);
   }
 
+
   public static Multimap<Integer, InputSplit> generateGroupedSplits(JobConf jobConf,
       Configuration conf, InputSplit[] splits, float waves, int availableSlots)
       throws Exception {
@@ -189,10 +200,6 @@ public class HiveSplitGenerator extends 
     return groupedSplits;
   }
 
-  @Override
-  public void handleInputInitializerEvent(List<InputInitializerEvent> events) throws Exception {
-  }
-
   private List<Event> createEventList(boolean sendSerializedEvents, InputSplitInfoMem inputSplitInfo) {
 
     List<Event> events = Lists.newArrayListWithCapacity(inputSplitInfo.getNumTasks() + 1);
@@ -221,4 +228,11 @@ public class HiveSplitGenerator extends 
     }
     return events;
   }
+
+  @Override
+  public void handleInputInitializerEvent(List<InputInitializerEvent> events) throws Exception {
+    for (InputInitializerEvent e : events) {
+      pruner.getQueue().put(e);
+    }
+  }
 }

Modified: hive/branches/tez/ql/src/java/org/apache/hadoop/hive/ql/exec/tez/MapRecordProcessor.java
URL: http://svn.apache.org/viewvc/hive/branches/tez/ql/src/java/org/apache/hadoop/hive/ql/exec/tez/MapRecordProcessor.java?rev=1622216&r1=1622215&r2=1622216&view=diff
==============================================================================
--- hive/branches/tez/ql/src/java/org/apache/hadoop/hive/ql/exec/tez/MapRecordProcessor.java (original)
+++ hive/branches/tez/ql/src/java/org/apache/hadoop/hive/ql/exec/tez/MapRecordProcessor.java Wed Sep  3 10:46:04 2014
@@ -117,7 +117,8 @@ public class MapRecordProcessor extends 
       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);

Modified: hive/branches/tez/ql/src/java/org/apache/hadoop/hive/ql/exec/tez/ReduceRecordProcessor.java
URL: http://svn.apache.org/viewvc/hive/branches/tez/ql/src/java/org/apache/hadoop/hive/ql/exec/tez/ReduceRecordProcessor.java?rev=1622216&r1=1622215&r2=1622216&view=diff
==============================================================================
--- hive/branches/tez/ql/src/java/org/apache/hadoop/hive/ql/exec/tez/ReduceRecordProcessor.java (original)
+++ hive/branches/tez/ql/src/java/org/apache/hadoop/hive/ql/exec/tez/ReduceRecordProcessor.java Wed Sep  3 10:46:04 2014
@@ -138,7 +138,7 @@ public class ReduceRecordProcessor  exte
 
     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 @@ public class ReduceRecordProcessor  exte
         keyStructInspector = (StructObjectInspector)keyObjectInspector;
         batches = new VectorizedRowBatch[maxTags];
         valueStructInspectors = new StructObjectInspector[maxTags];
-        valueStringWriters = (List<VectorExpressionWriter>[])new List[maxTags];
+        valueStringWriters = new List[maxTags];
         keysColumnOffset = keyStructInspector.getAllStructFieldRefs().size();
         buffer = new DataOutputBuffer();
       }
@@ -213,7 +213,8 @@ public class ReduceRecordProcessor  exte
     }
 
     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 @@ public class ReduceRecordProcessor  exte
     Map<Integer, String> tag2input = redWork.getTagToInput();
     ArrayList<LogicalInput> shuffleInputs = new ArrayList<LogicalInput>();
     for(String inpStr : tag2input.values()){
-      shuffleInputs.add((LogicalInput)inputs.get(inpStr));
+      shuffleInputs.add(inputs.get(inpStr));
     }
     return shuffleInputs;
   }

Modified: hive/branches/tez/ql/src/java/org/apache/hadoop/hive/ql/exec/tez/TezContext.java
URL: http://svn.apache.org/viewvc/hive/branches/tez/ql/src/java/org/apache/hadoop/hive/ql/exec/tez/TezContext.java?rev=1622216&r1=1622215&r2=1622216&view=diff
==============================================================================
--- hive/branches/tez/ql/src/java/org/apache/hadoop/hive/ql/exec/tez/TezContext.java (original)
+++ hive/branches/tez/ql/src/java/org/apache/hadoop/hive/ql/exec/tez/TezContext.java Wed Sep  3 10:46:04 2014
@@ -23,6 +23,7 @@ import org.apache.hadoop.hive.ql.exec.Ma
 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 @@ public class TezContext extends MapredCo
 
   // all the inputs for the tez processor
   private Map<String, LogicalInput> inputs;
-  
+
   private Map<String, LogicalOutput> outputs;
 
+  private ProcessorContext processorContext;
+
   public TezContext(boolean isMap, JobConf jobConf) {
     super(isMap, jobConf);
   }
@@ -41,7 +44,7 @@ public class TezContext extends MapredCo
   public void setInputs(Map<String, LogicalInput> inputs) {
     this.inputs = inputs;
   }
-  
+
   public void setOutputs(Map<String, LogicalOutput> outputs) {
     this.outputs = outputs;
   }
@@ -52,11 +55,19 @@ public class TezContext extends MapredCo
     }
     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;
+  }
 }

Modified: hive/branches/tez/ql/src/java/org/apache/hadoop/hive/ql/optimizer/ConstantPropagateProcFactory.java
URL: http://svn.apache.org/viewvc/hive/branches/tez/ql/src/java/org/apache/hadoop/hive/ql/optimizer/ConstantPropagateProcFactory.java?rev=1622216&r1=1622215&r2=1622216&view=diff
==============================================================================
--- hive/branches/tez/ql/src/java/org/apache/hadoop/hive/ql/optimizer/ConstantPropagateProcFactory.java (original)
+++ hive/branches/tez/ql/src/java/org/apache/hadoop/hive/ql/optimizer/ConstantPropagateProcFactory.java Wed Sep  3 10:46:04 2014
@@ -941,12 +941,12 @@ public final class ConstantPropagateProc
         return null;
       }
 
-      List<ExprNodeDesc> newChildren = new ArrayList<ExprNodeDesc>();
-      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;
     }
   }

Modified: hive/branches/tez/ql/src/java/org/apache/hadoop/hive/ql/optimizer/ConvertJoinMapJoin.java
URL: http://svn.apache.org/viewvc/hive/branches/tez/ql/src/java/org/apache/hadoop/hive/ql/optimizer/ConvertJoinMapJoin.java?rev=1622216&r1=1622215&r2=1622216&view=diff
==============================================================================
--- hive/branches/tez/ql/src/java/org/apache/hadoop/hive/ql/optimizer/ConvertJoinMapJoin.java (original)
+++ hive/branches/tez/ql/src/java/org/apache/hadoop/hive/ql/optimizer/ConvertJoinMapJoin.java Wed Sep  3 10:46:04 2014
@@ -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 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.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.lib.Nod
 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 class ConvertJoinMapJoin implemen
     Operator<? extends OperatorDesc> 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<Operator<?>> dynamicPartitionOperators = new HashSet<Operator<?>>();
+        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 class ConvertJoinMapJoin implemen
 
     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;
+  }
 }

Added: hive/branches/tez/ql/src/java/org/apache/hadoop/hive/ql/optimizer/DynamicPartitionPruningOptimization.java
URL: http://svn.apache.org/viewvc/hive/branches/tez/ql/src/java/org/apache/hadoop/hive/ql/optimizer/DynamicPartitionPruningOptimization.java?rev=1622216&view=auto
==============================================================================
--- hive/branches/tez/ql/src/java/org/apache/hadoop/hive/ql/optimizer/DynamicPartitionPruningOptimization.java (added)
+++ hive/branches/tez/ql/src/java/org/apache/hadoop/hive/ql/optimizer/DynamicPartitionPruningOptimization.java Wed Sep  3 10:46:04 2014
@@ -0,0 +1,347 @@
+/**
+ * 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<Node> 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<DynamicListContext> {
+    public List<DynamicListContext> dynLists = new ArrayList<DynamicListContext>();
+
+    public void addDynamicList(ExprNodeDynamicListDesc desc, ExprNodeDesc parent,
+        ExprNodeDesc grandParent, ReduceSinkOperator generator) {
+      dynLists.add(new DynamicListContext(desc, parent, grandParent, generator));
+    }
+
+    @Override
+    public Iterator<DynamicListContext> 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<Node> 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<? extends OperatorDesc> 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<ExprNodeDesc> keyExprs = new ArrayList<ExprNodeDesc>();
+    keyExprs.add(key);
+
+    // group by requires "ArrayList", don't ask.
+    ArrayList<String> outputNames = new ArrayList<String>();
+    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<ExprNodeDesc> groupByExprs = new ArrayList<ExprNodeDesc>();
+    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<AggregationDesc>(), false, groupByMemoryUsage, memoryThreshold,
+            null, false, 0, true);
+
+    GroupByOperator groupByOp =
+        (GroupByOperator) OperatorFactory.getAndMakeChild(groupBy, selectOp);
+
+    Map<String, ExprNodeDesc> colMap = new HashMap<String, ExprNodeDesc>();
+    colMap.put(outputNames.get(0), groupByExpr);
+    groupByOp.setColumnExprMap(colMap);
+
+    // 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<Node, Object> 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<Rule, NodeProcessor> exprRules = new LinkedHashMap<Rule, NodeProcessor>();
+    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<Node> startNodes = new ArrayList<Node>();
+    startNodes.add(pred);
+
+    HashMap<Node, Object> outputMap = new HashMap<Node, Object>();
+    egw.startWalking(startNodes, outputMap);
+    return outputMap;
+  }
+
+}

Modified: hive/branches/tez/ql/src/java/org/apache/hadoop/hive/ql/optimizer/Optimizer.java
URL: http://svn.apache.org/viewvc/hive/branches/tez/ql/src/java/org/apache/hadoop/hive/ql/optimizer/Optimizer.java?rev=1622216&r1=1622215&r2=1622216&view=diff
==============================================================================
--- hive/branches/tez/ql/src/java/org/apache/hadoop/hive/ql/optimizer/Optimizer.java (original)
+++ hive/branches/tez/ql/src/java/org/apache/hadoop/hive/ql/optimizer/Optimizer.java Wed Sep  3 10:46:04 2014
@@ -36,6 +36,7 @@ import org.apache.hadoop.hive.ql.parse.P
 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 class Optimizer {
     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 class Optimizer {
     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());
     }

Modified: hive/branches/tez/ql/src/java/org/apache/hadoop/hive/ql/optimizer/PrunerExpressionOperatorFactory.java
URL: http://svn.apache.org/viewvc/hive/branches/tez/ql/src/java/org/apache/hadoop/hive/ql/optimizer/PrunerExpressionOperatorFactory.java?rev=1622216&r1=1622215&r2=1622216&view=diff
==============================================================================
--- hive/branches/tez/ql/src/java/org/apache/hadoop/hive/ql/optimizer/PrunerExpressionOperatorFactory.java (original)
+++ hive/branches/tez/ql/src/java/org/apache/hadoop/hive/ql/optimizer/PrunerExpressionOperatorFactory.java Wed Sep  3 10:46:04 2014
@@ -186,8 +186,7 @@ public abstract class PrunerExpressionOp
         return ((ExprNodeNullDesc) nd).clone();
       }
 
-      assert (false);
-      return null;
+      return new ExprNodeConstantDesc(((ExprNodeDesc)nd).getTypeInfo(), null);
     }
   }
 

Modified: hive/branches/tez/ql/src/java/org/apache/hadoop/hive/ql/optimizer/ReduceSinkMapJoinProc.java
URL: http://svn.apache.org/viewvc/hive/branches/tez/ql/src/java/org/apache/hadoop/hive/ql/optimizer/ReduceSinkMapJoinProc.java?rev=1622216&r1=1622215&r2=1622216&view=diff
==============================================================================
--- hive/branches/tez/ql/src/java/org/apache/hadoop/hive/ql/optimizer/ReduceSinkMapJoinProc.java (original)
+++ hive/branches/tez/ql/src/java/org/apache/hadoop/hive/ql/optimizer/ReduceSinkMapJoinProc.java Wed Sep  3 10:46:04 2014
@@ -26,8 +26,6 @@ import java.util.Stack;
 
 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.G
 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 class ReduceSinkMapJoinProc imple
         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;

Added: hive/branches/tez/ql/src/java/org/apache/hadoop/hive/ql/optimizer/RemoveDynamicPruningBySize.java
URL: http://svn.apache.org/viewvc/hive/branches/tez/ql/src/java/org/apache/hadoop/hive/ql/optimizer/RemoveDynamicPruningBySize.java?rev=1622216&view=auto
==============================================================================
--- hive/branches/tez/ql/src/java/org/apache/hadoop/hive/ql/optimizer/RemoveDynamicPruningBySize.java (added)
+++ hive/branches/tez/ql/src/java/org/apache/hadoop/hive/ql/optimizer/RemoveDynamicPruningBySize.java Wed Sep  3 10:46:04 2014
@@ -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<Node> 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;
+  }
+}

Modified: hive/branches/tez/ql/src/java/org/apache/hadoop/hive/ql/optimizer/pcr/PcrExprProcFactory.java
URL: http://svn.apache.org/viewvc/hive/branches/tez/ql/src/java/org/apache/hadoop/hive/ql/optimizer/pcr/PcrExprProcFactory.java?rev=1622216&r1=1622215&r2=1622216&view=diff
==============================================================================
--- hive/branches/tez/ql/src/java/org/apache/hadoop/hive/ql/optimizer/pcr/PcrExprProcFactory.java (original)
+++ hive/branches/tez/ql/src/java/org/apache/hadoop/hive/ql/optimizer/pcr/PcrExprProcFactory.java Wed Sep  3 10:46:04 2014
@@ -396,8 +396,7 @@ public final class PcrExprProcFactory {
         return new NodeInfoWrapper(WalkState.CONSTANT, null,
             (ExprNodeDesc) nd);
       }
-      assert (false);
-      return null;
+      return new NodeInfoWrapper(WalkState.UNKNOWN, null, (ExprNodeDesc)nd);
     }
   }
 

Modified: hive/branches/tez/ql/src/java/org/apache/hadoop/hive/ql/optimizer/stats/annotation/StatsRulesProcFactory.java
URL: http://svn.apache.org/viewvc/hive/branches/tez/ql/src/java/org/apache/hadoop/hive/ql/optimizer/stats/annotation/StatsRulesProcFactory.java?rev=1622216&r1=1622215&r2=1622216&view=diff
==============================================================================
--- hive/branches/tez/ql/src/java/org/apache/hadoop/hive/ql/optimizer/stats/annotation/StatsRulesProcFactory.java (original)
+++ hive/branches/tez/ql/src/java/org/apache/hadoop/hive/ql/optimizer/stats/annotation/StatsRulesProcFactory.java Wed Sep  3 10:46:04 2014
@@ -18,13 +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;
@@ -67,12 +72,8 @@ import org.apache.hadoop.hive.ql.udf.gen
 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 {
 
@@ -657,7 +658,8 @@ public class StatsRulesProcFactory {
           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();

Added: hive/branches/tez/ql/src/java/org/apache/hadoop/hive/ql/parse/AppMasterEventProcessor.java
URL: http://svn.apache.org/viewvc/hive/branches/tez/ql/src/java/org/apache/hadoop/hive/ql/parse/AppMasterEventProcessor.java?rev=1622216&view=auto
==============================================================================
--- hive/branches/tez/ql/src/java/org/apache/hadoop/hive/ql/parse/AppMasterEventProcessor.java (added)
+++ hive/branches/tez/ql/src/java/org/apache/hadoop/hive/ql/parse/AppMasterEventProcessor.java Wed Sep  3 10:46:04 2014
@@ -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<Node> 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<AppMasterEventOperator> events;
+    if (context.tsToEventMap.containsKey(desc.getTableScan())) {
+      events = context.tsToEventMap.get(desc.getTableScan());
+    } else {
+      events = new ArrayList<AppMasterEventOperator>();
+    }
+    context.tsToEventMap.put(desc.getTableScan(), events);
+    return true;
+  }
+}