You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hive.apache.org by na...@apache.org on 2009/06/15 07:43:29 UTC

svn commit: r784656 [1/30] - in /hadoop/hive/trunk: ./ common/src/java/org/apache/hadoop/hive/conf/ ql/src/java/org/apache/hadoop/hive/ql/exec/ ql/src/java/org/apache/hadoop/hive/ql/optimizer/ ql/src/java/org/apache/hadoop/hive/ql/parse/ ql/src/java/or...

Author: namit
Date: Mon Jun 15 05:43:22 2009
New Revision: 784656

URL: http://svn.apache.org/viewvc?rev=784656&view=rev
Log:
HIVE-460. Improve Column Pruning. (He Yongqiang via namit)


Modified:
    hadoop/hive/trunk/CHANGES.txt
    hadoop/hive/trunk/common/src/java/org/apache/hadoop/hive/conf/HiveConf.java
    hadoop/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/exec/CommonJoinOperator.java
    hadoop/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/exec/GroupByOperator.java
    hadoop/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/exec/JoinOperator.java
    hadoop/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/exec/MapJoinOperator.java
    hadoop/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/exec/Operator.java
    hadoop/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/exec/ReduceSinkOperator.java
    hadoop/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/exec/SelectOperator.java
    hadoop/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/optimizer/ColumnPruner.java
    hadoop/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/optimizer/ColumnPrunerProcCtx.java
    hadoop/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/optimizer/ColumnPrunerProcFactory.java
    hadoop/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/optimizer/GenMRProcContext.java
    hadoop/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/optimizer/MapJoinFactory.java
    hadoop/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/optimizer/MapJoinProcessor.java
    hadoop/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/parse/SemanticAnalyzer.java
    hadoop/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/plan/PlanUtils.java
    hadoop/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/plan/groupByDesc.java
    hadoop/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/plan/joinDesc.java
    hadoop/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/plan/mapJoinDesc.java
    hadoop/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/plan/reduceSinkDesc.java
    hadoop/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/plan/selectDesc.java
    hadoop/hive/trunk/ql/src/test/org/apache/hadoop/hive/ql/exec/TestExecDriver.java
    hadoop/hive/trunk/ql/src/test/org/apache/hadoop/hive/ql/exec/TestOperators.java
    hadoop/hive/trunk/ql/src/test/results/clientpositive/binarysortable_1.q.out
    hadoop/hive/trunk/ql/src/test/results/clientpositive/case_sensitivity.q.out
    hadoop/hive/trunk/ql/src/test/results/clientpositive/cast1.q.out
    hadoop/hive/trunk/ql/src/test/results/clientpositive/cluster.q.out
    hadoop/hive/trunk/ql/src/test/results/clientpositive/columnarserde_create_shortcut.q.out
    hadoop/hive/trunk/ql/src/test/results/clientpositive/groupby1.q.out
    hadoop/hive/trunk/ql/src/test/results/clientpositive/groupby1_limit.q.out
    hadoop/hive/trunk/ql/src/test/results/clientpositive/groupby1_map.q.out
    hadoop/hive/trunk/ql/src/test/results/clientpositive/groupby1_map_nomap.q.out
    hadoop/hive/trunk/ql/src/test/results/clientpositive/groupby1_map_skew.q.out
    hadoop/hive/trunk/ql/src/test/results/clientpositive/groupby1_noskew.q.out
    hadoop/hive/trunk/ql/src/test/results/clientpositive/groupby2.q.out
    hadoop/hive/trunk/ql/src/test/results/clientpositive/groupby2_limit.q.out
    hadoop/hive/trunk/ql/src/test/results/clientpositive/groupby2_map.q.out
    hadoop/hive/trunk/ql/src/test/results/clientpositive/groupby2_map_skew.q.out
    hadoop/hive/trunk/ql/src/test/results/clientpositive/groupby2_noskew.q.out
    hadoop/hive/trunk/ql/src/test/results/clientpositive/groupby3.q.out
    hadoop/hive/trunk/ql/src/test/results/clientpositive/groupby3_map.q.out
    hadoop/hive/trunk/ql/src/test/results/clientpositive/groupby3_map_skew.q.out
    hadoop/hive/trunk/ql/src/test/results/clientpositive/groupby3_noskew.q.out
    hadoop/hive/trunk/ql/src/test/results/clientpositive/groupby4.q.out
    hadoop/hive/trunk/ql/src/test/results/clientpositive/groupby4_map.q.out
    hadoop/hive/trunk/ql/src/test/results/clientpositive/groupby4_map_skew.q.out
    hadoop/hive/trunk/ql/src/test/results/clientpositive/groupby4_noskew.q.out
    hadoop/hive/trunk/ql/src/test/results/clientpositive/groupby5.q.out
    hadoop/hive/trunk/ql/src/test/results/clientpositive/groupby5_map.q.out
    hadoop/hive/trunk/ql/src/test/results/clientpositive/groupby5_map_skew.q.out
    hadoop/hive/trunk/ql/src/test/results/clientpositive/groupby5_noskew.q.out
    hadoop/hive/trunk/ql/src/test/results/clientpositive/groupby6.q.out
    hadoop/hive/trunk/ql/src/test/results/clientpositive/groupby6_map.q.out
    hadoop/hive/trunk/ql/src/test/results/clientpositive/groupby6_map_skew.q.out
    hadoop/hive/trunk/ql/src/test/results/clientpositive/groupby6_noskew.q.out
    hadoop/hive/trunk/ql/src/test/results/clientpositive/groupby7_map.q.out
    hadoop/hive/trunk/ql/src/test/results/clientpositive/groupby7_map_skew.q.out
    hadoop/hive/trunk/ql/src/test/results/clientpositive/groupby7_noskew.q.out
    hadoop/hive/trunk/ql/src/test/results/clientpositive/groupby8.q.out
    hadoop/hive/trunk/ql/src/test/results/clientpositive/groupby8_map.q.out
    hadoop/hive/trunk/ql/src/test/results/clientpositive/groupby8_map_skew.q.out
    hadoop/hive/trunk/ql/src/test/results/clientpositive/groupby8_noskew.q.out
    hadoop/hive/trunk/ql/src/test/results/clientpositive/groupby9.q.out
    hadoop/hive/trunk/ql/src/test/results/clientpositive/input11.q.out
    hadoop/hive/trunk/ql/src/test/results/clientpositive/input11_limit.q.out
    hadoop/hive/trunk/ql/src/test/results/clientpositive/input12.q.out
    hadoop/hive/trunk/ql/src/test/results/clientpositive/input13.q.out
    hadoop/hive/trunk/ql/src/test/results/clientpositive/input14.q.out
    hadoop/hive/trunk/ql/src/test/results/clientpositive/input14_limit.q.out
    hadoop/hive/trunk/ql/src/test/results/clientpositive/input17.q.out
    hadoop/hive/trunk/ql/src/test/results/clientpositive/input18.q.out
    hadoop/hive/trunk/ql/src/test/results/clientpositive/input1_limit.q.out
    hadoop/hive/trunk/ql/src/test/results/clientpositive/input20.q.out
    hadoop/hive/trunk/ql/src/test/results/clientpositive/input21.q.out
    hadoop/hive/trunk/ql/src/test/results/clientpositive/input22.q.out
    hadoop/hive/trunk/ql/src/test/results/clientpositive/input23.q.out
    hadoop/hive/trunk/ql/src/test/results/clientpositive/input24.q.out
    hadoop/hive/trunk/ql/src/test/results/clientpositive/input25.q.out
    hadoop/hive/trunk/ql/src/test/results/clientpositive/input26.q.out
    hadoop/hive/trunk/ql/src/test/results/clientpositive/input30.q.out
    hadoop/hive/trunk/ql/src/test/results/clientpositive/input31.q.out
    hadoop/hive/trunk/ql/src/test/results/clientpositive/input32.q.out
    hadoop/hive/trunk/ql/src/test/results/clientpositive/input3_limit.q.out
    hadoop/hive/trunk/ql/src/test/results/clientpositive/input4_limit.q.out
    hadoop/hive/trunk/ql/src/test/results/clientpositive/input5.q.out
    hadoop/hive/trunk/ql/src/test/results/clientpositive/input7.q.out
    hadoop/hive/trunk/ql/src/test/results/clientpositive/input8.q.out
    hadoop/hive/trunk/ql/src/test/results/clientpositive/input9.q.out
    hadoop/hive/trunk/ql/src/test/results/clientpositive/input_columnarserde.q.out
    hadoop/hive/trunk/ql/src/test/results/clientpositive/input_dynamicserde.q.out
    hadoop/hive/trunk/ql/src/test/results/clientpositive/input_lazyserde.q.out
    hadoop/hive/trunk/ql/src/test/results/clientpositive/input_part1.q.out
    hadoop/hive/trunk/ql/src/test/results/clientpositive/input_part2.q.out
    hadoop/hive/trunk/ql/src/test/results/clientpositive/input_part7.q.out
    hadoop/hive/trunk/ql/src/test/results/clientpositive/input_testsequencefile.q.out
    hadoop/hive/trunk/ql/src/test/results/clientpositive/input_testxpath.q.out
    hadoop/hive/trunk/ql/src/test/results/clientpositive/input_testxpath2.q.out
    hadoop/hive/trunk/ql/src/test/results/clientpositive/input_testxpath3.q.out
    hadoop/hive/trunk/ql/src/test/results/clientpositive/input_testxpath4.q.out
    hadoop/hive/trunk/ql/src/test/results/clientpositive/join0.q.out
    hadoop/hive/trunk/ql/src/test/results/clientpositive/join1.q.out
    hadoop/hive/trunk/ql/src/test/results/clientpositive/join10.q.out
    hadoop/hive/trunk/ql/src/test/results/clientpositive/join11.q.out
    hadoop/hive/trunk/ql/src/test/results/clientpositive/join12.q.out
    hadoop/hive/trunk/ql/src/test/results/clientpositive/join13.q.out
    hadoop/hive/trunk/ql/src/test/results/clientpositive/join14.q.out
    hadoop/hive/trunk/ql/src/test/results/clientpositive/join15.q.out
    hadoop/hive/trunk/ql/src/test/results/clientpositive/join16.q.out
    hadoop/hive/trunk/ql/src/test/results/clientpositive/join17.q.out
    hadoop/hive/trunk/ql/src/test/results/clientpositive/join18.q.out
    hadoop/hive/trunk/ql/src/test/results/clientpositive/join19.q.out
    hadoop/hive/trunk/ql/src/test/results/clientpositive/join2.q.out
    hadoop/hive/trunk/ql/src/test/results/clientpositive/join20.q.out
    hadoop/hive/trunk/ql/src/test/results/clientpositive/join21.q.out
    hadoop/hive/trunk/ql/src/test/results/clientpositive/join22.q.out
    hadoop/hive/trunk/ql/src/test/results/clientpositive/join23.q.out
    hadoop/hive/trunk/ql/src/test/results/clientpositive/join25.q.out
    hadoop/hive/trunk/ql/src/test/results/clientpositive/join26.q.out
    hadoop/hive/trunk/ql/src/test/results/clientpositive/join27.q.out
    hadoop/hive/trunk/ql/src/test/results/clientpositive/join28.q.out
    hadoop/hive/trunk/ql/src/test/results/clientpositive/join29.q.out
    hadoop/hive/trunk/ql/src/test/results/clientpositive/join3.q.out
    hadoop/hive/trunk/ql/src/test/results/clientpositive/join30.q.out
    hadoop/hive/trunk/ql/src/test/results/clientpositive/join31.q.out
    hadoop/hive/trunk/ql/src/test/results/clientpositive/join32.q.out
    hadoop/hive/trunk/ql/src/test/results/clientpositive/join33.q.out
    hadoop/hive/trunk/ql/src/test/results/clientpositive/join34.q.out
    hadoop/hive/trunk/ql/src/test/results/clientpositive/join35.q.out
    hadoop/hive/trunk/ql/src/test/results/clientpositive/join36.q.out
    hadoop/hive/trunk/ql/src/test/results/clientpositive/join4.q.out
    hadoop/hive/trunk/ql/src/test/results/clientpositive/join5.q.out
    hadoop/hive/trunk/ql/src/test/results/clientpositive/join6.q.out
    hadoop/hive/trunk/ql/src/test/results/clientpositive/join7.q.out
    hadoop/hive/trunk/ql/src/test/results/clientpositive/join8.q.out
    hadoop/hive/trunk/ql/src/test/results/clientpositive/join9.q.out
    hadoop/hive/trunk/ql/src/test/results/clientpositive/join_thrift.q.out
    hadoop/hive/trunk/ql/src/test/results/clientpositive/mapreduce1.q.out
    hadoop/hive/trunk/ql/src/test/results/clientpositive/mapreduce2.q.out
    hadoop/hive/trunk/ql/src/test/results/clientpositive/mapreduce3.q.out
    hadoop/hive/trunk/ql/src/test/results/clientpositive/mapreduce4.q.out
    hadoop/hive/trunk/ql/src/test/results/clientpositive/mapreduce5.q.out
    hadoop/hive/trunk/ql/src/test/results/clientpositive/mapreduce6.q.out
    hadoop/hive/trunk/ql/src/test/results/clientpositive/mapreduce7.q.out
    hadoop/hive/trunk/ql/src/test/results/clientpositive/mapreduce8.q.out
    hadoop/hive/trunk/ql/src/test/results/clientpositive/no_hooks.q.out
    hadoop/hive/trunk/ql/src/test/results/clientpositive/noalias_subq1.q.out
    hadoop/hive/trunk/ql/src/test/results/clientpositive/notable_alias1.q.out
    hadoop/hive/trunk/ql/src/test/results/clientpositive/notable_alias2.q.out
    hadoop/hive/trunk/ql/src/test/results/clientpositive/nullgroup.q.out
    hadoop/hive/trunk/ql/src/test/results/clientpositive/nullgroup2.q.out
    hadoop/hive/trunk/ql/src/test/results/clientpositive/nullgroup3.q.out
    hadoop/hive/trunk/ql/src/test/results/clientpositive/nullgroup4.q.out
    hadoop/hive/trunk/ql/src/test/results/clientpositive/nullscript.q.out
    hadoop/hive/trunk/ql/src/test/results/clientpositive/order.q.out
    hadoop/hive/trunk/ql/src/test/results/clientpositive/order2.q.out
    hadoop/hive/trunk/ql/src/test/results/clientpositive/ppd1.q.out
    hadoop/hive/trunk/ql/src/test/results/clientpositive/ppd_clusterby.q.out
    hadoop/hive/trunk/ql/src/test/results/clientpositive/ppd_constant_expr.q.out
    hadoop/hive/trunk/ql/src/test/results/clientpositive/ppd_gby.q.out
    hadoop/hive/trunk/ql/src/test/results/clientpositive/ppd_gby2.q.out
    hadoop/hive/trunk/ql/src/test/results/clientpositive/ppd_gby_join.q.out
    hadoop/hive/trunk/ql/src/test/results/clientpositive/ppd_join.q.out
    hadoop/hive/trunk/ql/src/test/results/clientpositive/ppd_join2.q.out
    hadoop/hive/trunk/ql/src/test/results/clientpositive/ppd_join3.q.out
    hadoop/hive/trunk/ql/src/test/results/clientpositive/ppd_multi_insert.q.out
    hadoop/hive/trunk/ql/src/test/results/clientpositive/ppd_outer_join1.q.out
    hadoop/hive/trunk/ql/src/test/results/clientpositive/ppd_outer_join2.q.out
    hadoop/hive/trunk/ql/src/test/results/clientpositive/ppd_outer_join3.q.out
    hadoop/hive/trunk/ql/src/test/results/clientpositive/ppd_outer_join4.q.out
    hadoop/hive/trunk/ql/src/test/results/clientpositive/ppd_random.q.out
    hadoop/hive/trunk/ql/src/test/results/clientpositive/ppd_transform.q.out
    hadoop/hive/trunk/ql/src/test/results/clientpositive/ppd_udf_case.q.out
    hadoop/hive/trunk/ql/src/test/results/clientpositive/ppd_union.q.out
    hadoop/hive/trunk/ql/src/test/results/clientpositive/quote1.q.out
    hadoop/hive/trunk/ql/src/test/results/clientpositive/regex_col.q.out
    hadoop/hive/trunk/ql/src/test/results/clientpositive/regexp_extract.q.out
    hadoop/hive/trunk/ql/src/test/results/clientpositive/sample1.q.out
    hadoop/hive/trunk/ql/src/test/results/clientpositive/sample2.q.out
    hadoop/hive/trunk/ql/src/test/results/clientpositive/sample3.q.out
    hadoop/hive/trunk/ql/src/test/results/clientpositive/sample4.q.out
    hadoop/hive/trunk/ql/src/test/results/clientpositive/sample5.q.out
    hadoop/hive/trunk/ql/src/test/results/clientpositive/sample6.q.out
    hadoop/hive/trunk/ql/src/test/results/clientpositive/sample7.q.out
    hadoop/hive/trunk/ql/src/test/results/clientpositive/sample8.q.out
    hadoop/hive/trunk/ql/src/test/results/clientpositive/sort.q.out
    hadoop/hive/trunk/ql/src/test/results/clientpositive/subq.q.out
    hadoop/hive/trunk/ql/src/test/results/clientpositive/subq2.q.out
    hadoop/hive/trunk/ql/src/test/results/clientpositive/udf1.q.out
    hadoop/hive/trunk/ql/src/test/results/clientpositive/udf3.q.out
    hadoop/hive/trunk/ql/src/test/results/clientpositive/udf4.q.out
    hadoop/hive/trunk/ql/src/test/results/clientpositive/udf5.q.out
    hadoop/hive/trunk/ql/src/test/results/clientpositive/udf6.q.out
    hadoop/hive/trunk/ql/src/test/results/clientpositive/udf7.q.out
    hadoop/hive/trunk/ql/src/test/results/clientpositive/udf8.q.out
    hadoop/hive/trunk/ql/src/test/results/clientpositive/udf9.q.out
    hadoop/hive/trunk/ql/src/test/results/clientpositive/udf_10_trims.q.out
    hadoop/hive/trunk/ql/src/test/results/clientpositive/udf_case.q.out
    hadoop/hive/trunk/ql/src/test/results/clientpositive/udf_case_column_pruning.q.out
    hadoop/hive/trunk/ql/src/test/results/clientpositive/udf_case_thrift.q.out
    hadoop/hive/trunk/ql/src/test/results/clientpositive/udf_coalesce.q.out
    hadoop/hive/trunk/ql/src/test/results/clientpositive/udf_hash.q.out
    hadoop/hive/trunk/ql/src/test/results/clientpositive/udf_length.q.out
    hadoop/hive/trunk/ql/src/test/results/clientpositive/udf_lower.q.out
    hadoop/hive/trunk/ql/src/test/results/clientpositive/udf_when.q.out
    hadoop/hive/trunk/ql/src/test/results/clientpositive/union.q.out
    hadoop/hive/trunk/ql/src/test/results/clientpositive/union10.q.out
    hadoop/hive/trunk/ql/src/test/results/clientpositive/union11.q.out
    hadoop/hive/trunk/ql/src/test/results/clientpositive/union12.q.out
    hadoop/hive/trunk/ql/src/test/results/clientpositive/union13.q.out
    hadoop/hive/trunk/ql/src/test/results/clientpositive/union14.q.out
    hadoop/hive/trunk/ql/src/test/results/clientpositive/union15.q.out
    hadoop/hive/trunk/ql/src/test/results/clientpositive/union16.q.out
    hadoop/hive/trunk/ql/src/test/results/clientpositive/union17.q.out
    hadoop/hive/trunk/ql/src/test/results/clientpositive/union18.q.out
    hadoop/hive/trunk/ql/src/test/results/clientpositive/union19.q.out
    hadoop/hive/trunk/ql/src/test/results/clientpositive/union2.q.out
    hadoop/hive/trunk/ql/src/test/results/clientpositive/union20.q.out
    hadoop/hive/trunk/ql/src/test/results/clientpositive/union3.q.out
    hadoop/hive/trunk/ql/src/test/results/clientpositive/union4.q.out
    hadoop/hive/trunk/ql/src/test/results/clientpositive/union5.q.out
    hadoop/hive/trunk/ql/src/test/results/clientpositive/union6.q.out
    hadoop/hive/trunk/ql/src/test/results/clientpositive/union7.q.out
    hadoop/hive/trunk/ql/src/test/results/clientpositive/union8.q.out
    hadoop/hive/trunk/ql/src/test/results/clientpositive/union9.q.out
    hadoop/hive/trunk/ql/src/test/results/compiler/plan/case_sensitivity.q.xml
    hadoop/hive/trunk/ql/src/test/results/compiler/plan/cast1.q.xml
    hadoop/hive/trunk/ql/src/test/results/compiler/plan/groupby1.q.xml
    hadoop/hive/trunk/ql/src/test/results/compiler/plan/groupby2.q.xml
    hadoop/hive/trunk/ql/src/test/results/compiler/plan/groupby3.q.xml
    hadoop/hive/trunk/ql/src/test/results/compiler/plan/groupby4.q.xml
    hadoop/hive/trunk/ql/src/test/results/compiler/plan/groupby5.q.xml
    hadoop/hive/trunk/ql/src/test/results/compiler/plan/groupby6.q.xml
    hadoop/hive/trunk/ql/src/test/results/compiler/plan/input1.q.xml
    hadoop/hive/trunk/ql/src/test/results/compiler/plan/input2.q.xml
    hadoop/hive/trunk/ql/src/test/results/compiler/plan/input20.q.xml
    hadoop/hive/trunk/ql/src/test/results/compiler/plan/input3.q.xml
    hadoop/hive/trunk/ql/src/test/results/compiler/plan/input4.q.xml
    hadoop/hive/trunk/ql/src/test/results/compiler/plan/input5.q.xml
    hadoop/hive/trunk/ql/src/test/results/compiler/plan/input6.q.xml
    hadoop/hive/trunk/ql/src/test/results/compiler/plan/input7.q.xml
    hadoop/hive/trunk/ql/src/test/results/compiler/plan/input8.q.xml
    hadoop/hive/trunk/ql/src/test/results/compiler/plan/input9.q.xml
    hadoop/hive/trunk/ql/src/test/results/compiler/plan/input_part1.q.xml
    hadoop/hive/trunk/ql/src/test/results/compiler/plan/input_testsequencefile.q.xml
    hadoop/hive/trunk/ql/src/test/results/compiler/plan/input_testxpath.q.xml
    hadoop/hive/trunk/ql/src/test/results/compiler/plan/input_testxpath2.q.xml
    hadoop/hive/trunk/ql/src/test/results/compiler/plan/join1.q.xml
    hadoop/hive/trunk/ql/src/test/results/compiler/plan/join2.q.xml
    hadoop/hive/trunk/ql/src/test/results/compiler/plan/join3.q.xml
    hadoop/hive/trunk/ql/src/test/results/compiler/plan/join4.q.xml
    hadoop/hive/trunk/ql/src/test/results/compiler/plan/join5.q.xml
    hadoop/hive/trunk/ql/src/test/results/compiler/plan/join6.q.xml
    hadoop/hive/trunk/ql/src/test/results/compiler/plan/join7.q.xml
    hadoop/hive/trunk/ql/src/test/results/compiler/plan/join8.q.xml
    hadoop/hive/trunk/ql/src/test/results/compiler/plan/sample1.q.xml
    hadoop/hive/trunk/ql/src/test/results/compiler/plan/sample2.q.xml
    hadoop/hive/trunk/ql/src/test/results/compiler/plan/sample3.q.xml
    hadoop/hive/trunk/ql/src/test/results/compiler/plan/sample4.q.xml
    hadoop/hive/trunk/ql/src/test/results/compiler/plan/sample5.q.xml
    hadoop/hive/trunk/ql/src/test/results/compiler/plan/sample6.q.xml
    hadoop/hive/trunk/ql/src/test/results/compiler/plan/sample7.q.xml
    hadoop/hive/trunk/ql/src/test/results/compiler/plan/subq.q.xml
    hadoop/hive/trunk/ql/src/test/results/compiler/plan/udf1.q.xml
    hadoop/hive/trunk/ql/src/test/results/compiler/plan/udf4.q.xml
    hadoop/hive/trunk/ql/src/test/results/compiler/plan/udf6.q.xml
    hadoop/hive/trunk/ql/src/test/results/compiler/plan/udf_case.q.xml
    hadoop/hive/trunk/ql/src/test/results/compiler/plan/udf_when.q.xml
    hadoop/hive/trunk/ql/src/test/results/compiler/plan/union.q.xml
    hadoop/hive/trunk/serde/src/java/org/apache/hadoop/hive/serde2/objectinspector/MetadataListStructObjectInspector.java
    hadoop/hive/trunk/serde/src/java/org/apache/hadoop/hive/serde2/objectinspector/ObjectInspectorUtils.java
    hadoop/hive/trunk/serde/src/java/org/apache/hadoop/hive/serde2/objectinspector/StandardStructObjectInspector.java

Modified: hadoop/hive/trunk/CHANGES.txt
URL: http://svn.apache.org/viewvc/hadoop/hive/trunk/CHANGES.txt?rev=784656&r1=784655&r2=784656&view=diff
==============================================================================
--- hadoop/hive/trunk/CHANGES.txt (original)
+++ hadoop/hive/trunk/CHANGES.txt Mon Jun 15 05:43:22 2009
@@ -43,6 +43,8 @@
 
     HIVE-470. Add COALESCE. (Zheng Shao via namit)
 
+    HIVE-460. Improve Column Pruning. (He Yongqiang via namit)
+
   IMPROVEMENTS
     HIVE-389. Option to build without ivy (jssarma)
 

Modified: hadoop/hive/trunk/common/src/java/org/apache/hadoop/hive/conf/HiveConf.java
URL: http://svn.apache.org/viewvc/hadoop/hive/trunk/common/src/java/org/apache/hadoop/hive/conf/HiveConf.java?rev=784656&r1=784655&r2=784656&view=diff
==============================================================================
--- hadoop/hive/trunk/common/src/java/org/apache/hadoop/hive/conf/HiveConf.java (original)
+++ hadoop/hive/trunk/common/src/java/org/apache/hadoop/hive/conf/HiveConf.java Mon Jun 15 05:43:22 2009
@@ -395,6 +395,8 @@
       throw (IOException)new IOException().initCause(e);
     }
   }
-
-
+  
+  public static String getColumnInternalName(int pos){
+    return "_col"+pos;
+  }
 }

Modified: hadoop/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/exec/CommonJoinOperator.java
URL: http://svn.apache.org/viewvc/hadoop/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/exec/CommonJoinOperator.java?rev=784656&r1=784655&r2=784656&view=diff
==============================================================================
--- hadoop/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/exec/CommonJoinOperator.java (original)
+++ hadoop/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/exec/CommonJoinOperator.java Mon Jun 15 05:43:22 2009
@@ -37,6 +37,7 @@
 import org.apache.hadoop.hive.ql.plan.joinDesc;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspector;
+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;
@@ -155,7 +156,6 @@
 
       for (int j = 0; j < sz; j++)
         nr.add(null);
-
       dummyObj[pos] = nr;
       Vector<ArrayList<Object>> values = new Vector<ArrayList<Object>>();
       values.add((ArrayList<Object>) dummyObj[pos]);

Modified: hadoop/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/exec/GroupByOperator.java
URL: http://svn.apache.org/viewvc/hadoop/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/exec/GroupByOperator.java?rev=784656&r1=784655&r2=784656&view=diff
==============================================================================
--- hadoop/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/exec/GroupByOperator.java (original)
+++ hadoop/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/exec/GroupByOperator.java Mon Jun 15 05:43:22 2009
@@ -241,10 +241,7 @@
           aggregationsEvaluateMethods[i].getReturnType()).primitiveCategory));
     }
 
-    fieldNames = new ArrayList<String>(objectInspectors.size());
-    for(int i=0; i<objectInspectors.size(); i++) {
-      fieldNames.add(Integer.valueOf(i).toString());
-    }
+    fieldNames = conf.getOutputColumnNames();
 
     for (int i = 0; i < keyFields.length; i++) {
       if (keyObjectInspectors[i] == null) {
@@ -255,7 +252,7 @@
     
     outputObjectInspector = 
       ObjectInspectorFactory.getStandardStructObjectInspector(fieldNames, objectInspectors);
-    
+	
     firstRow = true;
     // estimate the number of hash table entries based on the size of each entry. Since the size of a entry
     // is not known, estimate that based on the number of entries

Modified: hadoop/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/exec/JoinOperator.java
URL: http://svn.apache.org/viewvc/hadoop/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/exec/JoinOperator.java?rev=784656&r1=784655&r2=784656&view=diff
==============================================================================
--- hadoop/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/exec/JoinOperator.java (original)
+++ hadoop/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/exec/JoinOperator.java Mon Jun 15 05:43:22 2009
@@ -35,6 +35,7 @@
 import org.apache.hadoop.io.Text;
 import org.apache.hadoop.mapred.Reporter;
 
+
 /**
  * Join operator implementation.
  */
@@ -57,10 +58,9 @@
                 ObjectInspectorCopyOption.KEEP));
       }
     }
-
+    
     joinOutputObjectInspector = ObjectInspectorFactory
-        .getStandardStructObjectInspector(ObjectInspectorUtils
-            .getIntegerArray(totalSz), structFieldObjectInspectors);
+    .getStandardStructObjectInspector(conf.getOutputColumnNames(), structFieldObjectInspectors);
     LOG.info("JOIN " + ((StructObjectInspector)joinOutputObjectInspector).getTypeName() + " totalsz = " + totalSz);
 
     initializeChildren(hconf, reporter, new ObjectInspector[]{joinOutputObjectInspector});

Modified: hadoop/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/exec/MapJoinOperator.java
URL: http://svn.apache.org/viewvc/hadoop/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/exec/MapJoinOperator.java?rev=784656&r1=784655&r2=784656&view=diff
==============================================================================
--- hadoop/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/exec/MapJoinOperator.java (original)
+++ hadoop/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/exec/MapJoinOperator.java Mon Jun 15 05:43:22 2009
@@ -32,6 +32,7 @@
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.hive.conf.HiveConf;
 import org.apache.hadoop.hive.ql.metadata.HiveException;
+import org.apache.hadoop.hive.ql.parse.SemanticAnalyzer;
 import org.apache.hadoop.hive.ql.plan.exprNodeDesc;
 import org.apache.hadoop.hive.ql.plan.mapJoinDesc;
 import org.apache.hadoop.hive.ql.plan.tableDesc;
@@ -192,11 +193,10 @@
           structFieldObjectInspectors.add(fld.getFieldObjectInspector());
         }
       }
-
-      joinOutputObjectInspector = ObjectInspectorFactory
-          .getStandardStructObjectInspector(ObjectInspectorUtils
-              .getIntegerArray(totalSz), structFieldObjectInspectors);
       
+      joinOutputObjectInspector = ObjectInspectorFactory
+      .getStandardStructObjectInspector(conf.getOutputColumnNames(), structFieldObjectInspectors);
+
       initializeChildren(hconf, reporter, new ObjectInspector[]{joinOutputObjectInspector});
     } catch (Exception e) {
       e.printStackTrace();
@@ -244,8 +244,12 @@
           for (exprNodeDesc e: conf.getKeys().get(new Byte((byte)tag))) {
             keyEval[i++] = ExprNodeEvaluatorFactory.get(e);
           }
-
-          ObjectInspector keyObjectInspector = initEvaluatorsAndReturnStruct(keyEval, rowInspector);
+          
+          List<String> keyOutputCols = new ArrayList<String>();
+          for (int k = 0; k < keyEval.length; k++) {
+            keyOutputCols.add(HiveConf.getColumnInternalName(k));
+          }
+          ObjectInspector keyObjectInspector = initEvaluatorsAndReturnStruct(keyEval, keyOutputCols, rowInspector);
 
           Deserializer deserializer = (Deserializer)ReflectionUtils.newInstance(keyTableDesc.getDeserializerClass(), null);
           deserializer.initialize(null, keyTableDesc.getProperties());
@@ -281,8 +285,13 @@
           for (exprNodeDesc e: conf.getExprs().get(new Byte((byte)tag))) {
             valueEval[i++] = ExprNodeEvaluatorFactory.get(e);
           }
-
-          ObjectInspector valueObjectInspector = initEvaluatorsAndReturnStruct(valueEval, rowInspector);
+          List<String> tagOutputCols = new ArrayList<String>(); 
+          int start = 0;
+          for (int k = 0; k < tag; k++)
+            start+=conf.getExprs().get(new Byte((byte)k)).size();
+          for (int k=0;k<conf.getExprs().get(new Byte((byte)tag)).size();k++)
+            tagOutputCols.add(HiveConf.getColumnInternalName(k));
+          ObjectInspector valueObjectInspector = initEvaluatorsAndReturnStruct(valueEval, tagOutputCols, rowInspector);
  
           Deserializer deserializer = (Deserializer)ReflectionUtils.newInstance(valueTableDesc.getDeserializerClass(), null);
           deserializer.initialize(null, valueTableDesc.getProperties());

Modified: hadoop/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/exec/Operator.java
URL: http://svn.apache.org/viewvc/hadoop/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/exec/Operator.java?rev=784656&r1=784655&r2=784656&view=diff
==============================================================================
--- hadoop/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/exec/Operator.java (original)
+++ hadoop/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/exec/Operator.java Mon Jun 15 05:43:22 2009
@@ -587,11 +587,11 @@
    * StructObjectInspector with integer field names.
    */  
   protected static StructObjectInspector initEvaluatorsAndReturnStruct(
-      ExprNodeEvaluator[] evals, ObjectInspector rowInspector) 
+      ExprNodeEvaluator[] evals, List<String> outputColName, ObjectInspector rowInspector) 
       throws HiveException {
     ObjectInspector[] fieldObjectInspectors = initEvaluators(evals, rowInspector);
     return ObjectInspectorFactory.getStandardStructObjectInspector(
-        ObjectInspectorUtils.getIntegerArray(fieldObjectInspectors.length),
+        outputColName,
         Arrays.asList(fieldObjectInspectors));
   }
   

Modified: hadoop/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/exec/ReduceSinkOperator.java
URL: http://svn.apache.org/viewvc/hadoop/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/exec/ReduceSinkOperator.java?rev=784656&r1=784655&r2=784656&view=diff
==============================================================================
--- hadoop/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/exec/ReduceSinkOperator.java (original)
+++ hadoop/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/exec/ReduceSinkOperator.java Mon Jun 15 05:43:22 2009
@@ -134,8 +134,8 @@
     try {
       if (firstRow) {
         firstRow = false;
-        keyObjectInspector = initEvaluatorsAndReturnStruct(keyEval, rowInspector);
-        valueObjectInspector = initEvaluatorsAndReturnStruct(valueEval, rowInspector);
+        keyObjectInspector = initEvaluatorsAndReturnStruct(keyEval, conf.getOutputKeyColumnNames(), rowInspector);
+        valueObjectInspector = initEvaluatorsAndReturnStruct(valueEval, conf.getOutputValueColumnNames(), rowInspector);
         partitionObjectInspectors = initEvaluators(partitionEval, rowInspector);
 
         cachedKeys = new Object[keyEval.length];

Modified: hadoop/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/exec/SelectOperator.java
URL: http://svn.apache.org/viewvc/hadoop/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/exec/SelectOperator.java?rev=784656&r1=784655&r2=784656&view=diff
==============================================================================
--- hadoop/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/exec/SelectOperator.java (original)
+++ hadoop/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/exec/SelectOperator.java Mon Jun 15 05:43:22 2009
@@ -40,7 +40,7 @@
   transient Object[] output;
   transient ObjectInspector outputObjectInspector;
   
-  public void initializeOp(Configuration hconf, Reporter reporter, ObjectInspector[] inputObjInspector) throws HiveException {    
+  public void initializeOp(Configuration hconf, Reporter reporter, ObjectInspector[] inputObjInspector) throws HiveException {
     // Just forward the row as is
     if (conf.isSelStarNoCompute()) {
       initializeChildren(hconf, reporter, inputObjInspector);
@@ -57,7 +57,8 @@
     assert inputObjInspector.length == 1;
     output = new Object[eval.length];
     LOG.info("SELECT " + ((StructObjectInspector)inputObjInspector[0]).getTypeName());
-    outputObjectInspector = initEvaluatorsAndReturnStruct(eval, inputObjInspector[0]); 
+    outputObjectInspector = initEvaluatorsAndReturnStruct(eval, conf
+          .getOutputColumnNames(), inputObjInspector[0]);
     initializeChildren(hconf, reporter, new ObjectInspector[]{outputObjectInspector});
   }
 

Modified: hadoop/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/optimizer/ColumnPruner.java
URL: http://svn.apache.org/viewvc/hadoop/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/optimizer/ColumnPruner.java?rev=784656&r1=784655&r2=784656&view=diff
==============================================================================
--- hadoop/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/optimizer/ColumnPruner.java (original)
+++ hadoop/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/optimizer/ColumnPruner.java Mon Jun 15 05:43:22 2009
@@ -21,15 +21,10 @@
 import java.io.Serializable;
 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 org.apache.hadoop.hive.ql.exec.ColumnInfo;
 import org.apache.hadoop.hive.ql.exec.FileSinkOperator;
 import org.apache.hadoop.hive.ql.exec.Operator;
-import org.apache.hadoop.hive.ql.exec.OperatorFactory;
-import org.apache.hadoop.hive.ql.exec.RowSchema;
 import org.apache.hadoop.hive.ql.exec.ScriptOperator;
 import org.apache.hadoop.hive.ql.exec.SelectOperator;
 import org.apache.hadoop.hive.ql.lib.DefaultGraphWalker;
@@ -42,14 +37,8 @@
 import org.apache.hadoop.hive.ql.lib.RuleRegExp;
 import org.apache.hadoop.hive.ql.parse.OpParseContext;
 import org.apache.hadoop.hive.ql.parse.ParseContext;
-import org.apache.hadoop.hive.ql.parse.QB;
 import org.apache.hadoop.hive.ql.parse.RowResolver;
-import org.apache.hadoop.hive.ql.parse.SemanticAnalyzer;
-import org.apache.hadoop.hive.ql.parse.SemanticAnalyzerFactory;
 import org.apache.hadoop.hive.ql.parse.SemanticException;
-import org.apache.hadoop.hive.ql.plan.exprNodeColumnDesc;
-import org.apache.hadoop.hive.ql.plan.exprNodeDesc;
-import org.apache.hadoop.hive.ql.plan.selectDesc;
 
 /**
  * Implementation of one of the rule-based optimization steps. ColumnPruner gets the current operator tree. The \
@@ -66,19 +55,8 @@
   /**
    * empty constructor
    */
-	public ColumnPruner() {
+  public ColumnPruner() {
     pGraphContext = null;
-	}
-
-	/**
-	 * Whether some column pruning needs to be done
-	 * @param op Operator for the base table
-	 * @param colNames columns needed by the query
-	 * @return boolean
-	 */
-  private boolean pushSelect(Operator<? extends Serializable> op, List<String> colNames) {
-    if (pGraphContext.getOpParseCtx().get(op).getRR().getColumnInfos().size() == colNames.size()) return false;
-    return true;
   }
 
   /**
@@ -93,73 +71,16 @@
     pGraphContext.getOpParseCtx().put(op, ctx);
     return op;
   }
-
-  /**
-   * insert a select to include only columns needed by the query
-   * @param input operator for the base table
-   * @param colNames columns needed
-   * @return
-   * @throws SemanticException
-   */
-  @SuppressWarnings("nls")
-  private Operator genSelectPlan(Operator input, List<String> colNames) 
-    throws SemanticException {
-
-    RowResolver inputRR  = pGraphContext.getOpParseCtx().get(input).getRR();
-    RowResolver outputRR = new RowResolver();
-    ArrayList<exprNodeDesc> col_list = new ArrayList<exprNodeDesc>();
-    Map<String, exprNodeDesc> colExprMap = new HashMap<String, exprNodeDesc>();
-    
-    // Iterate over the selects
-    for (int pos = 0; pos < colNames.size(); pos++) {
-      String   internalName = colNames.get(pos);
-      String[] colName      = inputRR.reverseLookup(internalName);
-      ColumnInfo in = inputRR.get(colName[0], colName[1]);
-      outputRR.put(colName[0], colName[1], 
-                   new ColumnInfo((Integer.valueOf(pos)).toString(), in.getType()));
-      col_list.add(new exprNodeColumnDesc(in.getType(), internalName));
-      colExprMap.put(Integer.toString(pos), col_list.get(pos));
-    }
-
-    Operator output = putOpInsertMap(OperatorFactory.getAndMakeChild(
-      new selectDesc(col_list), new RowSchema(outputRR.getColumnInfos()), input), outputRR);
-
-    output.setColumnExprMap(colExprMap);
-    return output;
-  }
-
-  /**
-   * reset parse context
-   * @param pctx parse context
-   */
-  private void resetParseContext(ParseContext pctx) {
-    pctx.getAliasToPruner().clear();
-    pctx.getAliasToSamplePruner().clear();
-    pctx.getLoadTableWork().clear();
-    pctx.getLoadFileWork().clear();
-    pctx.getJoinContext().clear();
-
-    Iterator<Operator<? extends Serializable>> iter = pctx.getOpParseCtx().keySet().iterator();
-    while (iter.hasNext()) {
-      Operator<? extends Serializable> op = iter.next();
-      if ((!pctx.getTopOps().containsValue(op)) && (!pctx.getTopSelOps().containsValue(op)))
-        iter.remove();
-    }
-    pctx.setDestTableId(1);
-    pctx.getIdToTableNameMap().clear();
-  }
-	
+  
   /**
    * Transform the query tree. For each table under consideration, check if all columns are needed. If not, 
    * only select the operators needed at the beginning and proceed 
    * @param pactx the current parse context
    */
-	public ParseContext transform(ParseContext pactx) throws SemanticException {
+  public ParseContext transform(ParseContext pactx) throws SemanticException {
     this.pGraphContext = pactx;
     this.opToParseCtxMap = pGraphContext.getOpParseCtx();
 
-    boolean done = true;    
-
     // generate pruned column list for all relevant operators
     ColumnPrunerProcCtx cppCtx = new ColumnPrunerProcCtx(opToParseCtxMap);
     
@@ -179,81 +100,49 @@
     ArrayList<Node> topNodes = new ArrayList<Node>();
     topNodes.addAll(pGraphContext.getTopOps().values());
     ogw.startWalking(topNodes, null);
-
-    // create a new select operator if any of input tables' columns can be pruned
-    for (String alias_id : pGraphContext.getTopOps().keySet()) {
-      Operator<? extends Serializable> topOp = pGraphContext.getTopOps().get(alias_id);
-
-      List<String> colNames = cppCtx.getPrunedColList(topOp);
-      
-      // do we need to push a SELECT - all the columns of the table are not used
-      if (pushSelect(topOp, colNames)) {
-        topOp.setChildOperators(null);
-        // Generate a select and make it a child of the table scan
-        Operator select = genSelectPlan(topOp, colNames);
-        pGraphContext.getTopSelOps().put(alias_id, select);
-        done = false;
-      }
-    }
-
-    // a select was pushed on top of the table. The old plan is no longer valid. Generate the plan again.
-    // The current tables and the select pushed above (after column pruning) are maintained in the parse context.
-    if (!done) {
-      SemanticAnalyzer sem = (SemanticAnalyzer)SemanticAnalyzerFactory.get(pGraphContext.getConf(), pGraphContext.getParseTree());
-
-      resetParseContext(pGraphContext);
-      QB qb = new QB(null, null, false);
-      pGraphContext.setQB(qb);
-      sem.init(pGraphContext);
-
-      sem.doPhase1(pGraphContext.getParseTree(), qb, sem.initPhase1Ctx());
-      sem.getMetaData(qb);
-      sem.genPlan(qb);
-      pGraphContext = sem.getParseContext();
-    }	
     return pGraphContext;
-	}
-	
-	/**
-	 * Walks the op tree in post order fashion (skips selects with file sink or script op children)
-	 */
-	public static class ColumnPrunerWalker extends DefaultGraphWalker {
+  }
+  
+  /**
+   * Walks the op tree in post order fashion (skips selects with file sink or script op children)
+   */
+  public static class ColumnPrunerWalker extends DefaultGraphWalker {
 
-	  public ColumnPrunerWalker(Dispatcher disp) {
+    public ColumnPrunerWalker(Dispatcher disp) {
       super(disp);
     }
 
     /**
-	   * Walk the given operator
-	   */
-	  @Override
-	  public void walk(Node nd) throws SemanticException {
-	    boolean walkChildren = true;
-	    opStack.push(nd);
-
-	    // no need to go further down for a select op with a file sink or script child
-	    // since all cols are needed for these ops
-	    if(nd instanceof SelectOperator) {
-	      for(Node child: nd.getChildren()) {
-	        if ((child instanceof FileSinkOperator) || (child instanceof ScriptOperator))
-	          walkChildren = false;
-	      }
-	    }
-
-	    if((nd.getChildren() == null) 
-	        || getDispatchedList().containsAll(nd.getChildren()) 
-	        || !walkChildren) {
-	      // all children are done or no need to walk the children
-	      dispatch(nd, opStack);
-	      opStack.pop();
-	      return;
-	    }
-	    // move all the children to the front of queue
-	    getToWalk().removeAll(nd.getChildren());
-	    getToWalk().addAll(0, nd.getChildren());
-	    // add self to the end of the queue
-	    getToWalk().add(nd);
-	    opStack.pop();
-	  }
-	}
+     * Walk the given operator
+     */
+    @Override
+    public void walk(Node nd) throws SemanticException {
+      boolean walkChildren = true;
+      opStack.push(nd);
+
+      // no need to go further down for a select op with a file sink or script child
+      // since all cols are needed for these ops
+      if(nd instanceof SelectOperator) {
+        for(Node child: nd.getChildren()) {
+          if ((child instanceof FileSinkOperator) || (child instanceof ScriptOperator))
+            walkChildren = false;
+        }
+      }
+
+      if((nd.getChildren() == null) 
+          || getDispatchedList().containsAll(nd.getChildren()) 
+          || !walkChildren) {
+        // all children are done or no need to walk the children
+        dispatch(nd, opStack);
+        opStack.pop();
+        return;
+      }
+      // move all the children to the front of queue
+      getToWalk().removeAll(nd.getChildren());
+      getToWalk().addAll(0, nd.getChildren());
+      // add self to the end of the queue
+      getToWalk().add(nd);
+      opStack.pop();
+    }
+  }
 }

Modified: hadoop/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/optimizer/ColumnPrunerProcCtx.java
URL: http://svn.apache.org/viewvc/hadoop/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/optimizer/ColumnPrunerProcCtx.java?rev=784656&r1=784655&r2=784656&view=diff
==============================================================================
--- hadoop/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/optimizer/ColumnPrunerProcCtx.java (original)
+++ hadoop/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/optimizer/ColumnPrunerProcCtx.java Mon Jun 15 05:43:22 2009
@@ -108,12 +108,17 @@
     List<String> cols = new ArrayList<String>();
     selectDesc conf = op.getConf();
     ArrayList<exprNodeDesc> selectExprs = conf.getColList();
-
-    for (String col : colList) {
-      // col is the internal name i.e. position within the expression list
-      exprNodeDesc expr = selectExprs.get(Integer.parseInt(col));
-      cols = Utilities.mergeUniqElems(cols, expr.getCols());
+    
+    // The colList is the output columns used by child operators, they are different
+    // from input columns of the current operator. we need to find out which input columns are used.   
+    ArrayList<String> outputColumnNames = conf.getOutputColumnNames();
+    for(int i=0;i<outputColumnNames.size();i++){
+      if(colList.contains(outputColumnNames.get(i))){
+        exprNodeDesc expr = selectExprs.get(i);
+        cols = Utilities.mergeUniqElems(cols, expr.getCols());
+      }
     }
+   
     return cols;
   }
 }

Modified: hadoop/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/optimizer/ColumnPrunerProcFactory.java
URL: http://svn.apache.org/viewvc/hadoop/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/optimizer/ColumnPrunerProcFactory.java?rev=784656&r1=784655&r2=784656&view=diff
==============================================================================
--- hadoop/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/optimizer/ColumnPrunerProcFactory.java (original)
+++ hadoop/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/optimizer/ColumnPrunerProcFactory.java Mon Jun 15 05:43:22 2009
@@ -23,20 +23,26 @@
 import java.util.HashMap;
 import java.util.List;
 import java.util.Stack;
+import java.util.Vector;
 
 import org.apache.hadoop.hive.ql.exec.ColumnInfo;
+import org.apache.hadoop.hive.ql.exec.ExprNodeEvaluator;
 import org.apache.hadoop.hive.ql.exec.FileSinkOperator;
 import org.apache.hadoop.hive.ql.exec.FilterOperator;
 import org.apache.hadoop.hive.ql.exec.GroupByOperator;
 import org.apache.hadoop.hive.ql.exec.JoinOperator;
+import org.apache.hadoop.hive.ql.exec.LimitOperator;
 import org.apache.hadoop.hive.ql.exec.Operator;
 import org.apache.hadoop.hive.ql.exec.ReduceSinkOperator;
+import org.apache.hadoop.hive.ql.exec.RowSchema;
 import org.apache.hadoop.hive.ql.exec.ScriptOperator;
 import org.apache.hadoop.hive.ql.exec.SelectOperator;
+import org.apache.hadoop.hive.ql.exec.UnionOperator;
 import org.apache.hadoop.hive.ql.exec.Utilities;
 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.unionproc.UnionProcessor;
 import org.apache.hadoop.hive.ql.parse.OpParseContext;
 import org.apache.hadoop.hive.ql.parse.RowResolver;
 import org.apache.hadoop.hive.ql.parse.SemanticException;
@@ -202,7 +208,9 @@
           // If one of my children is a FileSink or Script, return all columns.
           // Without this break, a bug in ReduceSink to Extract edge column pruning will manifest
           // which should be fixed before remove this
-          if ((child instanceof FileSinkOperator) || (child instanceof ScriptOperator)) {
+          if ((child instanceof FileSinkOperator)
+              || (child instanceof ScriptOperator)
+              || (child instanceof LimitOperator) || (child instanceof UnionOperator)) {
             cppCtx.getPrunedColLists().put(op, cppCtx.getColsFromSelectExpr(op));
             return null;
           }
@@ -211,15 +219,97 @@
       }
 
       selectDesc conf = op.getConf();
-      if (conf.isSelectStar() && !cols.isEmpty()) {
-        // The input to the select does not matter. Go over the expressions 
-        // and return the ones which have a marked column
-        cppCtx.getPrunedColLists().put(op, cppCtx.getSelectColsFromChildren(op, cols));
-        return null;
+      // The input to the select does not matter. Go over the expressions 
+      // and return the ones which have a marked column
+      cppCtx.getPrunedColLists().put(op, cppCtx.getSelectColsFromChildren(op, cols));
+      
+      // do we need to prune the select operator?
+      List<exprNodeDesc> originalColList = op.getConf().getColList();
+      List<String> columns = new ArrayList<String>();
+      for (exprNodeDesc expr : originalColList)
+        Utilities.mergeUniqElems(columns, expr.getCols());
+      // by now, 'prunedCols' are columns used by child operators, and 'columns'
+      // are columns used by this select operator.
+      ArrayList<String> originalOutputColumnNames = conf.getOutputColumnNames();
+      if (cols.size() < originalOutputColumnNames.size()) {
+        ArrayList<exprNodeDesc> newColList = new ArrayList<exprNodeDesc>();
+        ArrayList<String> newOutputColumnNames = new ArrayList<String>();
+        Vector<ColumnInfo> rs_oldsignature = op.getSchema().getSignature();
+        Vector<ColumnInfo> rs_newsignature = new Vector<ColumnInfo>();
+        RowResolver old_rr = cppCtx.getOpToParseCtxMap().get(op).getRR();
+        RowResolver new_rr = new RowResolver();
+        for(String col : cols){
+          int index = originalOutputColumnNames.indexOf(col);
+          newOutputColumnNames.add(col);
+          newColList.add(originalColList.get(index));
+          rs_newsignature.add(rs_oldsignature.get(index));
+          String[] tabcol = old_rr.reverseLookup(col);
+          ColumnInfo columnInfo = old_rr.get(tabcol[0], tabcol[1]);
+          new_rr.put(tabcol[0], tabcol[1], columnInfo);
+        }
+        cppCtx.getOpToParseCtxMap().get(op).setRR(new_rr);
+        op.getSchema().setSignature(rs_newsignature);
+        conf.setColList(newColList);
+        conf.setOutputColumnNames(newOutputColumnNames);
+        handleChildren(op, cols);
       }
-      cppCtx.getPrunedColLists().put(op, cppCtx.getColsFromSelectExpr(op));
       return null;
     }
+
+    /**
+     * since we pruned the select operator, we should let its children operator
+     * know that. ReduceSinkOperator may send out every output columns of its
+     * parent select. When the select operator is pruned, its child reduce
+     * sink(direct child) operator should also be pruned.
+     * 
+     * @param op
+     * @param retainedSelOutputCols
+     */
+    private void handleChildren(SelectOperator op,
+        List<String> retainedSelOutputCols) {
+      for(Operator<? extends Serializable> child: op.getChildOperators()) {
+        if (child instanceof ReduceSinkOperator) {
+          pruneReduceSinkOperator(retainedSelOutputCols, (ReduceSinkOperator)child);
+        }else if (child instanceof FilterOperator){
+          //filter operator has the same output columns as its parent
+          for(Operator<? extends Serializable> filterChild: child.getChildOperators()){
+            if (filterChild instanceof ReduceSinkOperator)
+              pruneReduceSinkOperator(retainedSelOutputCols, (ReduceSinkOperator)filterChild);
+          }
+        }
+      }
+    }
+
+    private void pruneReduceSinkOperator(List<String> retainedSelOpOutputCols,
+        ReduceSinkOperator child) {
+      ReduceSinkOperator reduce = (ReduceSinkOperator) child;
+      reduceSinkDesc reduceConf = reduce.getConf();
+      ArrayList<String> originalValueOutputColNames = reduceConf
+          .getOutputValueColumnNames();
+      java.util.ArrayList<exprNodeDesc> originalValueEval = reduceConf
+          .getValueCols();
+      ArrayList<String> newOutputColNames = new ArrayList<String>();
+      java.util.ArrayList<exprNodeDesc> newValueEval = new ArrayList<exprNodeDesc>();
+      for (int i = 0; i < originalValueEval.size(); i++) {
+        boolean retain = false;
+        List<String> current = originalValueEval.get(i).getCols();
+        if (current != null) {
+          for (int j = 0; j < current.size(); j++) {
+            if (retainedSelOpOutputCols.contains(current.get(j))) {
+              retain = true;
+              break;
+            }
+          }
+        }
+        if (retain) {
+          newOutputColNames.add(originalValueOutputColNames.get(i));
+          newValueEval.add(originalValueEval.get(i));
+        }
+      }
+      reduceConf.setOutputValueColumnNames(newOutputColNames);
+      reduceConf.setValueCols(newValueEval);
+    }
+
   }
 
   /**

Modified: hadoop/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/optimizer/GenMRProcContext.java
URL: http://svn.apache.org/viewvc/hadoop/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/optimizer/GenMRProcContext.java?rev=784656&r1=784655&r2=784656&view=diff
==============================================================================
--- hadoop/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/optimizer/GenMRProcContext.java (original)
+++ hadoop/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/optimizer/GenMRProcContext.java Mon Jun 15 05:43:22 2009
@@ -53,8 +53,6 @@
      * @param currTask    the current task
      * @param currTopOp   the current top operator being traversed
      * @param currAliasId the current alias for the to operator
-     * @param inputs      the list of read entities
-     * @param outputs     the list of write entities
      */
     public GenMapRedCtx (Task<? extends Serializable>         currTask,
                          Operator<? extends Serializable>     currTopOp,

Modified: hadoop/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/optimizer/MapJoinFactory.java
URL: http://svn.apache.org/viewvc/hadoop/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/optimizer/MapJoinFactory.java?rev=784656&r1=784655&r2=784656&view=diff
==============================================================================
--- hadoop/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/optimizer/MapJoinFactory.java (original)
+++ hadoop/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/optimizer/MapJoinFactory.java Mon Jun 15 05:43:22 2009
@@ -187,7 +187,7 @@
       Task<? extends Serializable> mjTask = TaskFactory.get(mjPlan, parseCtx.getConf());
       
       tableDesc tt_desc = 
-        PlanUtils.getLazySimpleSerDeTableDesc(PlanUtils.getFieldSchemasFromRowSchema(mapJoin.getSchema(), "temporarycol")); 
+        PlanUtils.getLazySimpleSerDeTableDesc(PlanUtils.sortFieldSchemas(PlanUtils.getFieldSchemasFromRowSchema(mapJoin.getSchema(), "temporarycol"))); 
       
       // generate the temporary file
       Context baseCtx = parseCtx.getContext();

Modified: hadoop/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/optimizer/MapJoinProcessor.java
URL: http://svn.apache.org/viewvc/hadoop/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/optimizer/MapJoinProcessor.java?rev=784656&r1=784655&r2=784656&view=diff
==============================================================================
--- hadoop/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/optimizer/MapJoinProcessor.java (original)
+++ hadoop/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/optimizer/MapJoinProcessor.java Mon Jun 15 05:43:22 2009
@@ -34,6 +34,7 @@
 import org.apache.hadoop.hive.ql.exec.Operator;
 import org.apache.hadoop.hive.ql.exec.OperatorFactory;
 import org.apache.hadoop.hive.ql.exec.FunctionRegistry;
+import org.apache.hadoop.hive.ql.exec.ReduceSinkOperator;
 import org.apache.hadoop.hive.ql.exec.RowSchema;
 import org.apache.hadoop.hive.ql.exec.SelectOperator;
 import org.apache.hadoop.hive.ql.parse.ASTNode;
@@ -50,6 +51,7 @@
 import org.apache.hadoop.hive.ql.plan.exprNodeColumnDesc;
 import org.apache.hadoop.hive.ql.plan.exprNodeDesc;
 import org.apache.hadoop.hive.ql.plan.mapJoinDesc;
+import org.apache.hadoop.hive.ql.plan.reduceSinkDesc;
 import org.apache.hadoop.hive.ql.plan.selectDesc;
 import org.apache.hadoop.hive.ql.plan.tableDesc;
 import org.apache.hadoop.hive.ql.plan.joinDesc;
@@ -77,6 +79,7 @@
     return op;
   }
   
+  
   /**
    * convert a regular join to a a map-side join. 
    * @param op join operator
@@ -95,8 +98,10 @@
       if ((condn.getType() == joinDesc.RIGHT_OUTER_JOIN) && (condn.getRight() != mapJoinPos))
         throw new SemanticException(ErrorMsg.NO_OUTER_MAPJOIN.getMsg());
     }
-
+    
+    RowResolver oldOutputRS = pctx.getOpParseCtx().get(op).getRR();
     RowResolver outputRS = new RowResolver();
+    ArrayList<String> outputColumnNames = new ArrayList<String>();
     Map<Byte, List<exprNodeDesc>> keyExprMap   = new HashMap<Byte, List<exprNodeDesc>>();
     Map<Byte, List<exprNodeDesc>> valueExprMap = new HashMap<Byte, List<exprNodeDesc>>();
 
@@ -106,6 +111,7 @@
 
     List<Operator<? extends Serializable>> parentOps = op.getParentOperators();
     List<Operator<? extends Serializable>> newParentOps = new ArrayList<Operator<? extends Serializable>>();
+    List<Operator<? extends Serializable>> oldReduceSinkParentOps = new ArrayList<Operator<? extends Serializable>>();
     
     // found a source which is not to be stored in memory
     if (leftSrc != null) {
@@ -113,7 +119,7 @@
       Operator<? extends Serializable> parentOp = parentOps.get(0);
       assert parentOp.getParentOperators().size() == 1;
       Operator<? extends Serializable> grandParentOp = parentOp.getParentOperators().get(0);
-      
+      oldReduceSinkParentOps.add(parentOp);
       grandParentOp.removeChild(parentOp);
       newParentOps.add(grandParentOp);
     }
@@ -127,36 +133,29 @@
         Operator<? extends Serializable> grandParentOp = parentOp.getParentOperators().get(0);
         
         grandParentOp.removeChild(parentOp);
-
+        oldReduceSinkParentOps.add(parentOp);
         newParentOps.add(grandParentOp);
       }
       pos++;
     }
 
     int keyLength = 0;
-    int outputPos = 0;
-
+    
+    //get the join keys from old parent ReduceSink operators
+    for (pos = 0; pos < newParentOps.size(); pos++) {
+      ReduceSinkOperator oldPar = (ReduceSinkOperator)oldReduceSinkParentOps.get(pos);
+      reduceSinkDesc rsconf = oldPar.getConf();
+      Byte tag = (byte)rsconf.getTag();
+      List<exprNodeDesc> keys = rsconf.getKeyCols();
+      keyExprMap.put(tag, keys);
+    }
+    
     // create the map-join operator
     for (pos = 0; pos < newParentOps.size(); pos++) {
       RowResolver inputRS = pGraphContext.getOpParseCtx().get(newParentOps.get(pos)).getRR();
     
-      List<exprNodeDesc> keys   = new ArrayList<exprNodeDesc>();
       List<exprNodeDesc> values = new ArrayList<exprNodeDesc>();
 
-      // Compute join keys and store in reduceKeys
-      Vector<ASTNode> exprs = joinTree.getExpressions().get(pos);
-      for (int i = 0; i < exprs.size(); i++) {
-        ASTNode expr = exprs.get(i);
-        keys.add(SemanticAnalyzer.genExprNodeDesc(expr, inputRS));
-      }
-
-      if (pos == 0)
-        keyLength = keys.size();
-      else
-        assert (keyLength == keys.size());
-    
-      keyExprMap.put(new Byte((byte)pos), keys);
-
       Iterator<String> keysIter = inputRS.getTableNames().iterator();
       while (keysIter.hasNext())
       {
@@ -168,9 +167,15 @@
           String field = fNamesIter.next();
           ColumnInfo valueInfo = inputRS.get(key, field);
           values.add(new exprNodeColumnDesc(valueInfo.getType(), valueInfo.getInternalName()));
-          if (outputRS.get(key, field) == null)
-            outputRS.put(key, field, new ColumnInfo((Integer.valueOf(outputPos++)).toString(), 
-                                                    valueInfo.getType()));
+          ColumnInfo oldValueInfo = oldOutputRS.get(key, field);
+          String col = field;
+          if(oldValueInfo != null)
+            col = oldValueInfo.getInternalName();
+          if (outputRS.get(key, col) == null) {
+            outputColumnNames.add(col);
+            outputRS.put(key, col, new ColumnInfo(col, 
+                valueInfo.getType()));
+          }
         }
       }
       
@@ -198,11 +203,7 @@
       }
     }
     
-    org.apache.hadoop.hive.ql.plan.joinCond[] joinCondns = new org.apache.hadoop.hive.ql.plan.joinCond[joinTree.getJoinCond().length];
-    for (int i = 0; i < joinTree.getJoinCond().length; i++) {
-      joinCond condn = joinTree.getJoinCond()[i];
-      joinCondns[i] = new org.apache.hadoop.hive.ql.plan.joinCond(condn);
-    }
+    org.apache.hadoop.hive.ql.plan.joinCond[] joinCondns = op.getConf().getConds();
 
     Operator[] newPar = new Operator[newParentOps.size()];
     pos = 0;
@@ -234,7 +235,7 @@
     }
       
     MapJoinOperator mapJoinOp = (MapJoinOperator)putOpInsertMap(OperatorFactory.getAndMakeChild(
-      new mapJoinDesc(keyExprMap, keyTableDesc, valueExprMap, valueTableDescs, mapJoinPos, joinCondns),
+      new mapJoinDesc(keyExprMap, keyTableDesc, valueExprMap, valueTableDescs, outputColumnNames, mapJoinPos, joinCondns),
       new RowSchema(outputRS.getColumnInfos()), newPar), outputRS);
     
     // change the children of the original join operator to point to the map join operator

Modified: hadoop/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/parse/SemanticAnalyzer.java
URL: http://svn.apache.org/viewvc/hadoop/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/parse/SemanticAnalyzer.java?rev=784656&r1=784655&r2=784656&view=diff
==============================================================================
--- hadoop/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/parse/SemanticAnalyzer.java (original)
+++ hadoop/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/parse/SemanticAnalyzer.java Mon Jun 15 05:43:22 2009
@@ -1043,7 +1043,7 @@
       
       exprNodeColumnDesc expr = new exprNodeColumnDesc(colInfo.getType(), name);
       col_list.add(expr);
-      output.put(tmp[0], tmp[1], new ColumnInfo(pos.toString(), colInfo.getType()));
+      output.put(tmp[0], tmp[1], new ColumnInfo(getColumnInternalName(pos), colInfo.getType()));
       pos = Integer.valueOf(pos.intValue() + 1);
       matched ++;
     }
@@ -1053,6 +1053,11 @@
     return pos;
   }
   
+  private String getColumnInternalName(int pos) {
+    return HiveConf.getColumnInternalName(pos);
+  }
+
+
   /**
    * If the user script command needs any modifications - do it here
    */
@@ -1307,24 +1312,27 @@
           throw new SemanticException(ErrorMsg.AMBIGUOUS_COLUMN.getMsg(expr.getChild(1)));
         }
         out_rwsch.put(tabAlias, colAlias,
-                      new ColumnInfo((Integer.valueOf(pos)).toString(),
+                      new ColumnInfo(getColumnInternalName(pos),
                                      exp.getTypeInfo()));
         pos = Integer.valueOf(pos.intValue() + 1);
       }
     }
     selectStar = selectStar && exprList.getChildCount() == posn + 1;
-    
+
+    ArrayList<String> columnNames = new ArrayList<String>();
     Map<String, exprNodeDesc> colExprMap = new HashMap<String, exprNodeDesc>();
     for (int i=0; i<col_list.size(); i++) {
       // Replace NULL with CAST(NULL AS STRING)
       if (col_list.get(i) instanceof exprNodeNullDesc) {
         col_list.set(i, new exprNodeConstantDesc(TypeInfoFactory.stringTypeInfo, null));
       }
-      colExprMap.put(Integer.toString(i), col_list.get(i));
+      String outputCol = getColumnInternalName(i);
+      colExprMap.put(outputCol, col_list.get(i));
+      columnNames.add(outputCol);
     }
     
     Operator output = putOpInsertMap(OperatorFactory.getAndMakeChild(
-        new selectDesc(col_list, selectStar), new RowSchema(out_rwsch.getColumnInfos()),
+        new selectDesc(col_list, columnNames, selectStar), new RowSchema(out_rwsch.getColumnInfos()),
         input), out_rwsch);
 
     output.setColumnExprMap(colExprMap);
@@ -1425,6 +1433,7 @@
     ArrayList<aggregationDesc> aggregations = new ArrayList<aggregationDesc>();
     ArrayList<String> evalMethods = new ArrayList<String>();
     ArrayList<String> aggMethods = new ArrayList<String>();
+    ArrayList<String> outputColumnNames = new ArrayList<String>();
     Map<String, exprNodeDesc> colExprMap = new HashMap<String, exprNodeDesc>();
     List<ASTNode> grpByExprs = getGroupByForClause(parseInfo, dest);
     for (int i = 0; i < grpByExprs.size(); ++i) {
@@ -1437,7 +1446,8 @@
       }
 
       groupByKeys.add(new exprNodeColumnDesc(exprInfo.getType(), exprInfo.getInternalName()));
-      String field = (Integer.valueOf(i)).toString();
+      String field = getColumnInternalName(i);
+      outputColumnNames.add(field);
       groupByOutputRowResolver.put("",grpbyExpr.toStringTree(),
                                    new ColumnInfo(field, exprInfo.getType()));
       colExprMap.put(field, groupByKeys.get(groupByKeys.size() - 1));
@@ -1472,13 +1482,15 @@
           value.getToken().getType() == HiveParser.TOK_FUNCTIONDI));
       evalMethods.add(udaf.evalMethod.getName());
       aggMethods.add(udaf.aggMethod.getName());
+      String field = getColumnInternalName(groupByKeys.size() + aggregations.size() -1);
+      outputColumnNames.add(field);
       groupByOutputRowResolver.put("",value.toStringTree(),
-                                   new ColumnInfo(Integer.valueOf(groupByKeys.size() + aggregations.size() -1).toString(),
+                                   new ColumnInfo(field,
                                        udaf.retType));
     }
 
     Operator op =  
-      putOpInsertMap(OperatorFactory.getAndMakeChild(new groupByDesc(mode, groupByKeys, aggregations, evalMethods, aggMethods),
+      putOpInsertMap(OperatorFactory.getAndMakeChild(new groupByDesc(mode, outputColumnNames, groupByKeys, aggregations, evalMethods, aggMethods),
                                                      new RowSchema(groupByOutputRowResolver.getColumnInfos()),
                                                      reduceSinkOperatorInfo),
         groupByOutputRowResolver
@@ -1499,6 +1511,7 @@
         QBParseInfo parseInfo, String dest, Operator reduceSinkOperatorInfo,
         groupByDesc.Mode mode)
     throws SemanticException {
+    ArrayList<String> outputColumnNames = new ArrayList<String>();
     RowResolver groupByInputRowResolver = opParseCtx.get(reduceSinkOperatorInfo).getRR();
     RowResolver groupByOutputRowResolver = new RowResolver();
     groupByOutputRowResolver.setIsExprResolver(true);
@@ -1518,7 +1531,8 @@
       }
 
       groupByKeys.add(new exprNodeColumnDesc(exprInfo.getType(), exprInfo.getInternalName()));
-      String field = (Integer.valueOf(i)).toString();
+      String field = getColumnInternalName(i);
+      outputColumnNames.add(field);
       groupByOutputRowResolver.put("",grpbyExpr.toStringTree(),
                                    new ColumnInfo(field, exprInfo.getType()));
       colExprMap.put(field, groupByKeys.get(groupByKeys.size() - 1));
@@ -1565,13 +1579,15 @@
           ((mode == groupByDesc.Mode.FINAL) ? false : (value.getToken().getType() == HiveParser.TOK_FUNCTIONDI))));
       evalMethods.add(udaf.evalMethod.getName());
       aggMethods.add(udaf.aggMethod.getName());
+      String field = getColumnInternalName(groupByKeys.size() + aggregations.size() - 1);
+      outputColumnNames.add(field);
       groupByOutputRowResolver.put("", value.toStringTree(),
-                                    new ColumnInfo(Integer.valueOf(groupByKeys.size() + aggregations.size() - 1).toString(),
+                                    new ColumnInfo(field,
                                         udaf.retType));
     }
 
     Operator op = putOpInsertMap(
-        OperatorFactory.getAndMakeChild(new groupByDesc(mode, groupByKeys, aggregations, evalMethods, aggMethods),
+        OperatorFactory.getAndMakeChild(new groupByDesc(mode, outputColumnNames, groupByKeys, aggregations, evalMethods, aggMethods),
                                         new RowSchema(groupByOutputRowResolver.getColumnInfos()),
                                         reduceSinkOperatorInfo),
         groupByOutputRowResolver);
@@ -1595,6 +1611,7 @@
     RowResolver groupByOutputRowResolver = new RowResolver();
     groupByOutputRowResolver.setIsExprResolver(true);
     ArrayList<exprNodeDesc> groupByKeys = new ArrayList<exprNodeDesc>();
+    ArrayList<String> outputColumnNames = new ArrayList<String>();
     ArrayList<aggregationDesc> aggregations = new ArrayList<aggregationDesc>();
     ArrayList<String> evalMethods = new ArrayList<String>();
     ArrayList<String> aggMethods = new ArrayList<String>();
@@ -1605,7 +1622,8 @@
       exprNodeDesc grpByExprNode = genExprNodeDesc(grpbyExpr, groupByInputRowResolver);
 
       groupByKeys.add(grpByExprNode);
-      String field = (Integer.valueOf(i)).toString();
+      String field = getColumnInternalName(i);
+      outputColumnNames.add(field);
       groupByOutputRowResolver.put("",grpbyExpr.toStringTree(),
                                    new ColumnInfo(field, grpByExprNode.getTypeInfo()));
       colExprMap.put(field, groupByKeys.get(groupByKeys.size() - 1));
@@ -1623,7 +1641,8 @@
           exprNodeDesc distExprNode = genExprNodeDesc(parameter, groupByInputRowResolver);
           groupByKeys.add(distExprNode);
           numDistn++;
-          String field = (Integer.valueOf(grpByExprs.size() + numDistn -1)).toString();
+          String field = getColumnInternalName(grpByExprs.size() + numDistn -1);
+          outputColumnNames.add(field);
           groupByOutputRowResolver.put("", text, new ColumnInfo(field, distExprNode.getTypeInfo()));
           colExprMap.put(field, groupByKeys.get(groupByKeys.size() - 1));
         }
@@ -1656,13 +1675,15 @@
                                            value.getToken().getType() == HiveParser.TOK_FUNCTIONDI));
       evalMethods.add(udaf.evalMethod.getName());
       aggMethods.add(udaf.aggMethod.getName());
+      String field = getColumnInternalName(groupByKeys.size() + aggregations.size() -1);
+      outputColumnNames.add(field);
       groupByOutputRowResolver.put("",value.toStringTree(),
-                                   new ColumnInfo(Integer.valueOf(groupByKeys.size() + aggregations.size() -1).toString(),
+                                   new ColumnInfo(field,
                                        udaf.retType));
     }
 
     Operator op = putOpInsertMap(
-      OperatorFactory.getAndMakeChild(new groupByDesc(mode, groupByKeys, aggregations, evalMethods, aggMethods),
+      OperatorFactory.getAndMakeChild(new groupByDesc(mode, outputColumnNames, groupByKeys, aggregations, evalMethods, aggMethods),
                                       new RowSchema(groupByOutputRowResolver.getColumnInfos()),
                                       inputOperatorInfo),
       groupByOutputRowResolver);
@@ -1695,6 +1716,7 @@
     ArrayList<exprNodeDesc> reduceKeys = new ArrayList<exprNodeDesc>();
     // Pre-compute group-by keys and store in reduceKeys
 
+    List<String> outputColumnNames = new ArrayList<String>();
     List<ASTNode> grpByExprs = getGroupByForClause(parseInfo, dest);
     for (int i = 0; i < grpByExprs.size(); ++i) {
       ASTNode grpbyExpr = grpByExprs.get(i);
@@ -1702,7 +1724,9 @@
       reduceKeys.add(inputExpr);
       String text = grpbyExpr.toStringTree();
       if (reduceSinkOutputRowResolver.get("", text) == null) {
-        ColumnInfo colInfo = new ColumnInfo(Utilities.ReduceField.KEY.toString() + "." + Integer.valueOf(reduceKeys.size() - 1).toString(),
+        outputColumnNames.add(getColumnInternalName(reduceKeys.size() - 1));
+        String field = Utilities.ReduceField.KEY.toString() + "." + getColumnInternalName(reduceKeys.size() - 1);
+        ColumnInfo colInfo = new ColumnInfo(field,
             reduceKeys.get(reduceKeys.size()-1).getTypeInfo());
         reduceSinkOutputRowResolver.put("", text, colInfo);
         colExprMap.put(colInfo.getInternalName(), inputExpr);
@@ -1720,7 +1744,9 @@
         String text = parameter.toStringTree();
         if (reduceSinkOutputRowResolver.get("",text) == null) {
           reduceKeys.add(genExprNodeDesc(parameter, reduceSinkInputRowResolver));
-          ColumnInfo colInfo = new ColumnInfo(Utilities.ReduceField.KEY.toString() + "." + Integer.valueOf(reduceKeys.size() - 1).toString(),
+          outputColumnNames.add(getColumnInternalName(reduceKeys.size() - 1));
+          String field = Utilities.ReduceField.KEY.toString() + "." + getColumnInternalName(reduceKeys.size() - 1);
+          ColumnInfo colInfo = new ColumnInfo(field,
               reduceKeys.get(reduceKeys.size()-1).getTypeInfo());
           reduceSinkOutputRowResolver.put("", text, colInfo);
           colExprMap.put(colInfo.getInternalName(), reduceKeys.get(reduceKeys.size()-1));
@@ -1741,8 +1767,10 @@
           String text = parameter.toStringTree();
           if (reduceSinkOutputRowResolver.get("",text) == null) {
             reduceValues.add(genExprNodeDesc(parameter, reduceSinkInputRowResolver));
+            outputColumnNames.add(getColumnInternalName(reduceValues.size() - 1));
+            String field = Utilities.ReduceField.VALUE.toString() + "." + getColumnInternalName(reduceValues.size() - 1);
             reduceSinkOutputRowResolver.put("", text,
-                                            new ColumnInfo(Utilities.ReduceField.VALUE.toString() + "." + Integer.valueOf(reduceValues.size() - 1).toString(),
+                                            new ColumnInfo(field,
                                                            reduceValues.get(reduceValues.size()-1).getTypeInfo()));
           }
         }
@@ -1756,16 +1784,18 @@
       for (Map.Entry<String, ASTNode> entry : aggregationTrees.entrySet()) {
         
         TypeInfo type = reduceSinkInputRowResolver.getColumnInfos().get(inputField).getType(); 
-        reduceValues.add(new exprNodeColumnDesc(type, (Integer.valueOf(inputField)).toString()));
+        reduceValues.add(new exprNodeColumnDesc(type, getColumnInternalName(inputField)));
         inputField++;
+        outputColumnNames.add(getColumnInternalName(reduceValues.size() - 1));
+        String field = Utilities.ReduceField.VALUE.toString() + "." + getColumnInternalName(reduceValues.size() - 1);
         reduceSinkOutputRowResolver.put("", ((ASTNode)entry.getValue()).toStringTree(),
-                                        new ColumnInfo(Utilities.ReduceField.VALUE.toString() + "." + (Integer.valueOf(reduceValues.size()-1)).toString(),
+                                        new ColumnInfo(field,
                                                        type));
       }
     }
 
     ReduceSinkOperator rsOp = (ReduceSinkOperator)  putOpInsertMap(
-      OperatorFactory.getAndMakeChild(PlanUtils.getReduceSinkDesc(reduceKeys, reduceValues, -1, numPartitionFields,
+      OperatorFactory.getAndMakeChild(PlanUtils.getReduceSinkDesc(reduceKeys, reduceValues, outputColumnNames, true, -1, numPartitionFields,
                                                                   numReducers),
                                         new RowSchema(reduceSinkOutputRowResolver.getColumnInfos()),
                                         inputOperatorInfo),
@@ -1797,11 +1827,13 @@
     reduceSinkOutputRowResolver2.setIsExprResolver(true);
     Map<String, exprNodeDesc> colExprMap = new HashMap<String, exprNodeDesc>();
     ArrayList<exprNodeDesc> reduceKeys = new ArrayList<exprNodeDesc>();
+    ArrayList<String> outputColumnNames = new ArrayList<String>();
     // Get group-by keys and store in reduceKeys
     List<ASTNode> grpByExprs = getGroupByForClause(parseInfo, dest);
     for (int i = 0; i < grpByExprs.size(); ++i) {
       ASTNode grpbyExpr = grpByExprs.get(i);
-      String field = (Integer.valueOf(i)).toString();
+      String field = getColumnInternalName(i);
+      outputColumnNames.add(field);
       TypeInfo typeInfo = reduceSinkInputRowResolver2.get("", grpbyExpr.toStringTree()).getType();
       exprNodeColumnDesc inputExpr = new exprNodeColumnDesc(typeInfo, field);
       reduceKeys.add(inputExpr);
@@ -1817,18 +1849,20 @@
     HashMap<String, ASTNode> aggregationTrees = parseInfo
         .getAggregationExprsForClause(dest);
     for (Map.Entry<String, ASTNode> entry : aggregationTrees.entrySet()) {
-      String field = (Integer.valueOf(inputField)).toString();
+      String field = getColumnInternalName(inputField);
       ASTNode t = entry.getValue();
       TypeInfo typeInfo = reduceSinkInputRowResolver2.get("", t.toStringTree()).getType();
       reduceValues.add(new exprNodeColumnDesc(typeInfo, field));
       inputField++;
+      String col = getColumnInternalName(reduceValues.size()-1);
+      outputColumnNames.add(col);
       reduceSinkOutputRowResolver2.put("", t.toStringTree(),
-                                       new ColumnInfo(Utilities.ReduceField.VALUE.toString() + "." + (Integer.valueOf(reduceValues.size()-1)).toString(),
+                                       new ColumnInfo(Utilities.ReduceField.VALUE.toString() + "." + col,
                                            typeInfo));
     }
 
     ReduceSinkOperator rsOp = (ReduceSinkOperator) putOpInsertMap(
-      OperatorFactory.getAndMakeChild(PlanUtils.getReduceSinkDesc(reduceKeys, reduceValues, -1, 
+      OperatorFactory.getAndMakeChild(PlanUtils.getReduceSinkDesc(reduceKeys, reduceValues, outputColumnNames, true, -1, 
                                                                   numPartitionFields, numReducers),
                                         new RowSchema(reduceSinkOutputRowResolver2.getColumnInfos()),
                                         groupByOperatorInfo),
@@ -1861,6 +1895,7 @@
     ArrayList<String> aggMethods = new ArrayList<String>();
     Map<String, exprNodeDesc> colExprMap = new HashMap<String, exprNodeDesc>();
     List<ASTNode> grpByExprs = getGroupByForClause(parseInfo, dest);
+    ArrayList<String> outputColumnNames = new ArrayList<String>(); 
     for (int i = 0; i < grpByExprs.size(); ++i) {
       ASTNode grpbyExpr = grpByExprs.get(i);
       String text = grpbyExpr.toStringTree();
@@ -1871,7 +1906,8 @@
 
       String expression = exprInfo.getInternalName();
       groupByKeys.add(new exprNodeColumnDesc(exprInfo.getType(), expression));
-      String field = (Integer.valueOf(i)).toString();
+      String field = getColumnInternalName(i);
+      outputColumnNames.add(field);
       groupByOutputRowResolver2.put("",grpbyExpr.toStringTree(),
                                     new ColumnInfo(field, exprInfo.getType()));
       colExprMap.put(field, groupByKeys.get(groupByKeys.size() - 1));
@@ -1899,13 +1935,15 @@
                                            ((mode == groupByDesc.Mode.FINAL) ? false : (value.getToken().getType() == HiveParser.TOK_FUNCTIONDI))));
       evalMethods.add(udaf.evalMethod.getName());
       aggMethods.add(udaf.aggMethod.getName());
+      String field = getColumnInternalName(groupByKeys.size() + aggregations.size() - 1);
+      outputColumnNames.add(field);
       groupByOutputRowResolver2.put("", value.toStringTree(),
-                                    new ColumnInfo(Integer.valueOf(groupByKeys.size() + aggregations.size() - 1).toString(),
+                                    new ColumnInfo(field,
                                         udaf.retType));
     }
 
     Operator op = putOpInsertMap(
-      OperatorFactory.getAndMakeChild(new groupByDesc(mode, groupByKeys, aggregations, evalMethods, aggMethods),
+      OperatorFactory.getAndMakeChild(new groupByDesc(mode, outputColumnNames, groupByKeys, aggregations, evalMethods, aggMethods),
                                       new RowSchema(groupByOutputRowResolver2.getColumnInfos()),
                                       reduceSinkOperatorInfo2),
         groupByOutputRowResolver2
@@ -2383,7 +2421,7 @@
         ObjectInspector tableFieldOI = tableFields.get(i).getFieldObjectInspector();
         TypeInfo tableFieldTypeInfo = TypeInfoUtils.getTypeInfoFromObjectInspector(tableFieldOI);
         TypeInfo rowFieldTypeInfo = rowFields.get(i).getType();
-        exprNodeDesc column = new exprNodeColumnDesc(rowFieldTypeInfo, Integer.valueOf(i).toString());
+        exprNodeDesc column = new exprNodeColumnDesc(rowFieldTypeInfo, rowFields.get(i).getInternalName());
         // LazySimpleSerDe can convert any types to String type using JSON-format.
         if (!tableFieldTypeInfo.equals(rowFieldTypeInfo)
             && !(isLazySimpleSerDe && tableFieldTypeInfo.getCategory().equals(Category.PRIMITIVE)
@@ -2410,12 +2448,14 @@
     if (converted) {
       // add the select operator
       RowResolver rowResolver = new RowResolver();
+      ArrayList<String> colName = new ArrayList<String>();
       for (int i=0; i<expressions.size(); i++) {
-        String name = Integer.valueOf(i).toString();
+        String name = getColumnInternalName(i);
         rowResolver.put("", name, new ColumnInfo(name, expressions.get(i).getTypeInfo()));
+        colName.add(name);
       }
       Operator output = putOpInsertMap(OperatorFactory.getAndMakeChild(
-        new selectDesc(expressions), new RowSchema(rowResolver.getColumnInfos()), input), rowResolver);
+        new selectDesc(expressions, colName), new RowSchema(rowResolver.getColumnInfos()), input), rowResolver);
 
       return output;
     } else {
@@ -2531,10 +2571,13 @@
       valueCols.add(new exprNodeColumnDesc(colInfo.getType(), colInfo.getInternalName()));
       colExprMap.put(colInfo.getInternalName(), valueCols.get(valueCols.size() - 1));
     }
-
+    
+    ArrayList<String> outputColumns = new ArrayList<String>();
+    for (int i = 0; i < valueCols.size(); i++)
+      outputColumns.add(getColumnInternalName(i));
     Operator interim = putOpInsertMap(
       OperatorFactory.getAndMakeChild(
-        PlanUtils.getReduceSinkDesc(sortCols, valueCols, -1, partitionCols, order.toString(),
+        PlanUtils.getReduceSinkDesc(sortCols, valueCols, outputColumns, false, -1, partitionCols, order.toString(),
             numReducers),
         new RowSchema(inputRR.getColumnInfos()),
         input), inputRR);
@@ -2547,7 +2590,7 @@
     for(ColumnInfo colInfo: interim_rwsch.getColumnInfos()) {
       String [] info = interim_rwsch.reverseLookup(colInfo.getInternalName());
       out_rwsch.put(info[0], info[1],
-                    new ColumnInfo(pos.toString(), colInfo.getType()));
+                    new ColumnInfo(getColumnInternalName(pos), colInfo.getType()));
       pos = Integer.valueOf(pos.intValue() + 1);
     }
 
@@ -2565,6 +2608,7 @@
   private Operator genJoinOperatorChildren(QBJoinTree join, Operator left, Operator[] right) 
     throws SemanticException {
     RowResolver outputRS = new RowResolver();
+    ArrayList<String> outputColumnNames = new ArrayList<String>();
     // all children are base classes
     Operator<?>[] rightOps = new Operator[right.length];
     int pos = 0;
@@ -2599,7 +2643,9 @@
           ColumnInfo valueInfo = inputRS.get(key, field);
           keyDesc.add(new exprNodeColumnDesc(valueInfo.getType(), valueInfo.getInternalName()));
           if (outputRS.get(key, field) == null) {
-            String colName = (Integer.valueOf(outputPos++)).toString();
+            String colName = getColumnInternalName(outputPos);
+            outputPos++;
+            outputColumnNames.add(colName);
             colExprMap.put(colName, keyDesc.get(keyDesc.size() - 1));
             outputRS.put(key, field, new ColumnInfo(colName, 
                                                     valueInfo.getType()));
@@ -2617,7 +2663,7 @@
       joinCondns[i] = new org.apache.hadoop.hive.ql.plan.joinCond(condn);
     }
 
-    JoinOperator joinOp = (JoinOperator) OperatorFactory.getAndMakeChild(new joinDesc(exprMap, joinCondns),
+    JoinOperator joinOp = (JoinOperator) OperatorFactory.getAndMakeChild(new joinDesc(exprMap, outputColumnNames, joinCondns),
                                     new RowSchema(outputRS.getColumnInfos()), rightOps);
     joinOp.setColumnExprMap(colExprMap);
     joinOp.setPosToAliasMap(posToAliasMap);
@@ -2629,6 +2675,7 @@
       Operator child, String srcName, int pos) throws SemanticException {
     RowResolver inputRS = opParseCtx.get(child).getRR();
     RowResolver outputRS = new RowResolver();
+    ArrayList<String> outputColumns = new ArrayList<String>();
     ArrayList<exprNodeDesc> reduceKeys = new ArrayList<exprNodeDesc>();
 
     // Compute join keys and store in reduceKeys
@@ -2652,8 +2699,10 @@
         exprNodeColumnDesc inputExpr = new exprNodeColumnDesc(valueInfo.getType(), valueInfo.getInternalName());
         reduceValues.add(inputExpr);
         if (outputRS.get(src, field) == null) {
+          String col = getColumnInternalName(reduceValues.size() - 1);
+          outputColumns.add(col);
           ColumnInfo newColInfo = new ColumnInfo(Utilities.ReduceField.VALUE.toString() + "." +
-                          Integer.valueOf(reduceValues.size() - 1).toString(),
+                          col,
                           valueInfo.getType());
           colExprMap.put(newColInfo.getInternalName(), inputExpr);
           outputRS.put(src, field, newColInfo);
@@ -2674,7 +2723,7 @@
 
     ReduceSinkOperator rsOp = (ReduceSinkOperator)putOpInsertMap(
       OperatorFactory.getAndMakeChild(
-        PlanUtils.getReduceSinkDesc(reduceKeys, reduceValues, joinTree.getNextTag(), reduceKeys.size(), numReds), 
+        PlanUtils.getReduceSinkDesc(reduceKeys, reduceValues, outputColumns, false, joinTree.getNextTag(), reduceKeys.size(), numReds), 
         new RowSchema(outputRS.getColumnInfos()),
         child), outputRS);
     rsOp.setColumnExprMap(colExprMap);
@@ -3077,6 +3126,24 @@
       }
     }
   }
+  
+  private Operator insertSelectAllPlanForGroupBy(String dest, Operator input)
+      throws SemanticException {
+    OpParseContext inputCtx = opParseCtx.get(input);
+    RowResolver inputRR = inputCtx.getRR();
+    Vector<ColumnInfo> columns = inputRR.getColumnInfos();
+    ArrayList<exprNodeDesc> colList = new ArrayList<exprNodeDesc>();
+    ArrayList<String> columnNames = new ArrayList<String>();
+    for (int i = 0; i < columns.size(); i++) {
+      ColumnInfo col = columns.get(i);
+      colList.add(new exprNodeColumnDesc(col.getType(), col.getInternalName()));
+      columnNames.add(col.getInternalName());
+    }
+    Operator output = putOpInsertMap(OperatorFactory.getAndMakeChild(
+        new selectDesc(colList, columnNames, true), new RowSchema(inputRR.getColumnInfos()), input), inputRR);
+    output.setColumnExprMap(input.getColumnExprMap());
+    return output;
+  }
 
   @SuppressWarnings("nls")
   private Operator genBodyPlan(QB qb, Operator input)
@@ -3099,6 +3166,8 @@
       if (qbp.getAggregationExprsForClause(dest).size() != 0
           || getGroupByForClause(qbp, dest).size() > 0)
       {
+        // insert a select operator here used by the ColumnPruner to reduce the data to shuffle
+        curr = insertSelectAllPlanForGroupBy(dest, curr);
         if (conf.getVar(HiveConf.ConfVars.HIVEMAPSIDEAGGREGATE).equalsIgnoreCase("true")) {
           if (conf.getVar(HiveConf.ConfVars.HIVEGROUPBYSKEW).equalsIgnoreCase("false"))
             curr = genGroupByPlanMapAggr1MR(dest, qb, curr);