You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hive.apache.org by sp...@apache.org on 2016/05/26 15:40:39 UTC

[17/66] [abbrv] hive git commit: HIVE-13068: Disable Hive ConstantPropagate optimizer when CBO has optimized the plan II (Jesus Camacho Rodriguez, reviewed by Ashutosh Chauhan)

HIVE-13068: Disable Hive ConstantPropagate optimizer when CBO has optimized the plan II (Jesus Camacho Rodriguez, reviewed by Ashutosh Chauhan)


Project: http://git-wip-us.apache.org/repos/asf/hive/repo
Commit: http://git-wip-us.apache.org/repos/asf/hive/commit/10423f51
Tree: http://git-wip-us.apache.org/repos/asf/hive/tree/10423f51
Diff: http://git-wip-us.apache.org/repos/asf/hive/diff/10423f51

Branch: refs/heads/java8
Commit: 10423f51c86fd3266b8756225d8dfdf3c523e9c3
Parents: ec4b936
Author: Jesus Camacho Rodriguez <jc...@apache.org>
Authored: Sat May 14 11:39:38 2016 +0100
Committer: Jesus Camacho Rodriguez <jc...@apache.org>
Committed: Tue May 24 11:31:16 2016 +0100

----------------------------------------------------------------------
 .../results/positive/hbase_ppd_key_range.q.out  |   1 +
 .../test/results/positive/hbase_queries.q.out   |   5 +
 .../ql/optimizer/ConstantPropagateProcCtx.java  |   4 +
 .../ql/optimizer/NonBlockingOpDeDupProc.java    |   5 +-
 .../hadoop/hive/ql/optimizer/Optimizer.java     |  12 +-
 .../optimizer/SortedDynPartitionOptimizer.java  |  34 +-
 .../hive/ql/optimizer/StatsOptimizer.java       |  18 +-
 .../ql/optimizer/calcite/HiveRelOptUtil.java    |  38 +-
 .../hive/ql/optimizer/calcite/HiveRexUtil.java  | 332 +++++++--
 .../rules/HiveAggregatePullUpConstantsRule.java |  47 ++
 .../HiveProjectFilterPullUpConstantsRule.java   | 177 +++++
 .../rules/HiveReduceExpressionsRule.java        | 186 +++--
 .../rules/HiveSortLimitPullUpConstantsRule.java |  20 +-
 .../rules/HiveUnionPullUpConstantsRule.java     |  27 +-
 .../calcite/stats/HiveRelMdPredicates.java      | 532 --------------
 .../calcite/translator/ASTConverter.java        |   5 +
 .../calcite/translator/ExprNodeConverter.java   |  28 +-
 .../calcite/translator/HiveGBOpConvUtil.java    |   3 +-
 .../calcite/translator/HiveOpConverter.java     |   7 +-
 .../translator/HiveOpConverterPostProc.java     |   2 +-
 .../calcite/translator/RexNodeConverter.java    |  15 +-
 .../translator/SqlFunctionConverter.java        |   9 +-
 .../calcite/translator/TypeConverter.java       |  20 +-
 .../ql/optimizer/pcr/PcrExprProcFactory.java    |  16 +-
 .../hadoop/hive/ql/parse/CalcitePlanner.java    |  22 +-
 .../hadoop/hive/ql/parse/SemanticAnalyzer.java  |  22 +-
 .../hive/ql/parse/TypeCheckProcFactory.java     |  13 +
 .../hadoop/hive/ql/plan/ExprNodeDescUtils.java  |  74 +-
 ql/src/test/queries/clientpositive/join_view.q  |   4 +-
 .../clientpositive/annotate_stats_filter.q.out  |   8 +-
 .../archive_excludeHadoop20.q.out               |   1 +
 .../results/clientpositive/archive_multi.q.out  |   1 +
 .../authorization_explain.q.java1.7.out         |   4 +-
 .../results/clientpositive/auto_join33.q.out    |   8 +-
 .../results/clientpositive/auto_join8.q.out     |   2 +-
 .../clientpositive/auto_join_filters.q.out      |   4 +-
 .../clientpositive/auto_join_nulls.q.out        |   4 +-
 .../clientpositive/auto_sortmerge_join_12.q.out |   4 +-
 .../results/clientpositive/bucket_groupby.q.out |  64 +-
 .../bucketizedhiveinputformat.q.out             |   2 +-
 ql/src/test/results/clientpositive/cast1.q.out  |   4 +-
 .../test/results/clientpositive/cbo_const.q.out |   5 +-
 .../cbo_rp_annotate_stats_groupby.q.out         |  70 +-
 .../results/clientpositive/cbo_rp_join1.q.out   |   2 +-
 .../clientpositive/cbo_rp_lineage2.q.out        |   2 +-
 .../clientpositive/colstats_all_nulls.q.out     |   6 +-
 .../constantPropagateForSubQuery.q.out          |   4 +-
 .../clientpositive/constant_prop_3.q.out        |   2 +-
 .../results/clientpositive/constprog2.q.out     |  20 +-
 .../results/clientpositive/constprog3.q.out     |  16 +-
 .../clientpositive/constprog_partitioner.q.out  |  14 +-
 .../clientpositive/constprog_semijoin.q.out     |  46 +-
 ql/src/test/results/clientpositive/cp_sel.q.out |  16 +-
 .../clientpositive/create_genericudf.q.out      |   6 +-
 .../results/clientpositive/create_view.q.out    |   2 +-
 .../results/clientpositive/cross_join.q.out     |   8 +-
 .../clientpositive/cross_join_merge.q.out       |   6 +-
 .../clientpositive/cross_product_check_1.q.out  |  12 +-
 .../clientpositive/cross_product_check_2.q.out  |  16 +-
 ql/src/test/results/clientpositive/cte_5.q.out  |   2 +-
 .../test/results/clientpositive/cte_mat_1.q.out |  13 +-
 .../test/results/clientpositive/cte_mat_2.q.out |  13 +-
 .../results/clientpositive/decimal_stats.q.out  |   2 +-
 .../clientpositive/dynamic_rdd_cache.q.out      |  78 +--
 .../dynpart_sort_optimization.q.out             | 104 +--
 .../dynpart_sort_optimization2.q.out            |  44 +-
 .../clientpositive/explain_logical.q.out        |  48 +-
 .../clientpositive/filter_cond_pushdown.q.out   |   2 +-
 .../test/results/clientpositive/fold_case.q.out |  60 +-
 .../clientpositive/fold_eq_with_case_when.q.out |   4 +-
 .../test/results/clientpositive/fold_when.q.out |   4 +-
 .../clientpositive/folder_predicate.q.out       |  50 +-
 .../results/clientpositive/groupby_ppd.q.out    |  46 +-
 .../clientpositive/groupby_sort_1_23.q.out      |  86 +--
 .../clientpositive/groupby_sort_skew_1_23.q.out | 110 +--
 .../clientpositive/index_auto_unused.q.out      |  38 +-
 .../clientpositive/infer_join_preds.q.out       |  66 +-
 .../test/results/clientpositive/input23.q.out   |   4 +-
 .../test/results/clientpositive/input26.q.out   |  40 +-
 ql/src/test/results/clientpositive/input6.q.out |   2 +-
 ql/src/test/results/clientpositive/input8.q.out |   4 +-
 .../results/clientpositive/input_part10.q.out   |   5 +-
 .../results/clientpositive/insert_into5.q.out   |   5 +-
 .../insert_nonacid_from_acid.q.out              |   2 +-
 ql/src/test/results/clientpositive/join38.q.out |  22 +-
 ql/src/test/results/clientpositive/join42.q.out |   4 +-
 ql/src/test/results/clientpositive/join8.q.out  |   2 +-
 .../clientpositive/join_alt_syntax.q.out        |   4 +-
 .../clientpositive/join_cond_pushdown_1.q.out   |   4 +-
 .../clientpositive/join_cond_pushdown_3.q.out   |   4 +-
 .../join_cond_pushdown_unqual1.q.out            |   4 +-
 .../join_cond_pushdown_unqual3.q.out            |   4 +-
 .../results/clientpositive/join_filters.q.out   |   4 +-
 .../results/clientpositive/join_nulls.q.out     |   4 +-
 .../results/clientpositive/join_reorder.q.out   |   6 +-
 .../test/results/clientpositive/join_view.q.out |  13 +-
 .../test/results/clientpositive/lineage2.q.out  |   2 +-
 .../test/results/clientpositive/lineage3.q.out  |  14 +-
 .../list_bucket_query_oneskew_1.q.out           |   2 +-
 .../list_bucket_query_oneskew_2.q.out           |   8 +-
 .../results/clientpositive/llap/cte_mat_1.q.out |  13 +-
 .../results/clientpositive/llap/cte_mat_2.q.out |  13 +-
 .../llap/dynamic_partition_pruning.q.out        | 231 +++----
 .../llap/dynamic_partition_pruning_2.q.out      |   4 +-
 .../llap/hybridgrace_hashjoin_1.q.out           |   8 +-
 .../llap/tez_dynpart_hashjoin_1.q.out           |  12 +-
 .../clientpositive/llap/tez_self_join.q.out     |  32 +-
 .../llap/tez_union_dynamic_partition.q.out      |  44 +-
 .../llap/tez_vector_dynpart_hashjoin_1.q.out    |  12 +-
 .../vectorized_dynamic_partition_pruning.q.out  | 231 +++----
 .../test/results/clientpositive/masking_2.q.out |   2 +-
 .../test/results/clientpositive/mergejoin.q.out |   6 +-
 .../nonblock_op_deduplicate.q.out               |   8 +-
 .../test/results/clientpositive/orc_llap.q.out  |   8 +-
 .../clientpositive/partition_multilevels.q.out  |  40 +-
 ql/src/test/results/clientpositive/pcr.q.out    | 102 +--
 .../results/clientpositive/perf/query18.q.out   |   6 +-
 .../results/clientpositive/perf/query26.q.out   |   6 +-
 .../results/clientpositive/perf/query27.q.out   |   6 +-
 .../results/clientpositive/perf/query28.q.out   |  34 +-
 .../results/clientpositive/perf/query31.q.out   | 556 ++++++++-------
 .../results/clientpositive/perf/query39.q.out   |  60 +-
 .../results/clientpositive/perf/query42.q.out   |  34 +-
 .../results/clientpositive/perf/query48.q.out   |   6 +-
 .../results/clientpositive/perf/query52.q.out   |  16 +-
 .../results/clientpositive/perf/query64.q.out   |  42 +-
 .../results/clientpositive/perf/query66.q.out   | 304 ++++----
 .../results/clientpositive/perf/query7.q.out    |   6 +-
 .../results/clientpositive/perf/query72.q.out   |   6 +-
 .../results/clientpositive/perf/query75.q.out   | 686 +++++++++----------
 .../results/clientpositive/perf/query88.q.out   | 136 ++--
 .../results/clientpositive/perf/query90.q.out   |  46 +-
 .../results/clientpositive/pointlookup2.q.out   | 100 +--
 .../results/clientpositive/pointlookup3.q.out   | 100 +--
 .../clientpositive/ppd_constant_expr.q.out      |   8 +-
 .../test/results/clientpositive/ppd_join5.q.out |   6 +-
 .../clientpositive/ppd_outer_join4.q.out        |  36 +-
 .../clientpositive/ppd_outer_join5.q.out        |   6 +-
 .../clientpositive/ppd_repeated_alias.q.out     |   2 +-
 .../results/clientpositive/ppd_udf_case.q.out   |  64 +-
 .../results/clientpositive/ppd_union_view.q.out |  48 +-
 .../results/clientpositive/quotedid_basic.q.out | 100 ++-
 .../clientpositive/quotedid_partition.q.out     |  18 +-
 .../clientpositive/rand_partitionpruner3.q.out  |  12 +-
 .../results/clientpositive/recursive_dir.q.out  |   2 +-
 .../test/results/clientpositive/semijoin4.q.out |  68 +-
 .../test/results/clientpositive/semijoin5.q.out |  28 +-
 .../test/results/clientpositive/skewjoin.q.out  |   6 +-
 .../results/clientpositive/smb_mapjoin_25.q.out |  59 +-
 .../clientpositive/spark/auto_join8.q.out       |   2 +-
 .../spark/auto_join_filters.q.out               |   4 +-
 .../clientpositive/spark/auto_join_nulls.q.out  |   4 +-
 .../spark/auto_sortmerge_join_12.q.out          |   4 +-
 .../spark/bucketizedhiveinputformat.q.out       |   2 +-
 .../spark/constprog_semijoin.q.out              |  46 +-
 .../clientpositive/spark/cross_join.q.out       |   8 +-
 .../spark/cross_product_check_1.q.out           |  12 +-
 .../spark/cross_product_check_2.q.out           |  12 +-
 .../spark/dynamic_rdd_cache.q.out               |  78 +--
 .../spark/groupby_sort_1_23.q.out               |  78 +--
 .../spark/groupby_sort_skew_1_23.q.out          |  90 +--
 .../results/clientpositive/spark/join38.q.out   |  22 +-
 .../results/clientpositive/spark/join8.q.out    |   2 +-
 .../clientpositive/spark/join_alt_syntax.q.out  |   4 +-
 .../spark/join_cond_pushdown_1.q.out            |   4 +-
 .../spark/join_cond_pushdown_3.q.out            |   4 +-
 .../spark/join_cond_pushdown_unqual1.q.out      |   4 +-
 .../spark/join_cond_pushdown_unqual3.q.out      |   4 +-
 .../clientpositive/spark/join_reorder.q.out     |   6 +-
 .../clientpositive/spark/join_view.q.out        |  15 +-
 .../test/results/clientpositive/spark/pcr.q.out |  46 +-
 .../clientpositive/spark/ppd_join5.q.out        |   6 +-
 .../clientpositive/spark/ppd_outer_join4.q.out  |  36 +-
 .../clientpositive/spark/ppd_outer_join5.q.out  |   6 +-
 .../results/clientpositive/spark/skewjoin.q.out |   6 +-
 .../clientpositive/spark/smb_mapjoin_25.q.out   |  37 +-
 .../spark/table_access_keys_stats.q.out         |  17 +-
 .../clientpositive/spark/union_view.q.out       |  20 +-
 .../spark/vector_mapjoin_reduce.q.out           |  10 +-
 .../spark/vector_outer_join1.q.out              |   8 +-
 .../spark/vector_outer_join2.q.out              |   8 +-
 .../spark/vector_outer_join3.q.out              |   8 +-
 .../spark/vector_outer_join4.q.out              |   8 +-
 .../spark/vector_outer_join5.q.out              |   8 +-
 .../spark/vectorization_short_regress.q.out     |   4 +-
 .../results/clientpositive/subquery_notin.q.out |  50 +-
 .../subquery_notin_having.q.java1.7.out         |  38 +-
 .../subquery_unqualcolumnrefs.q.out             |   2 +-
 .../results/clientpositive/subquery_views.q.out |   8 +-
 .../table_access_keys_stats.q.out               |  17 +-
 .../clientpositive/tez/auto_join_filters.q.out  |   4 +-
 .../clientpositive/tez/auto_join_nulls.q.out    |   4 +-
 .../tez/auto_sortmerge_join_12.q.out            |   4 +-
 .../clientpositive/tez/constprog_semijoin.q.out |  54 +-
 .../results/clientpositive/tez/cross_join.q.out |   8 +-
 .../tez/cross_product_check_1.q.out             |  12 +-
 .../tez/cross_product_check_2.q.out             |  12 +-
 .../results/clientpositive/tez/cte_mat_1.q.out  |  11 +-
 .../results/clientpositive/tez/cte_mat_2.q.out  |  11 +-
 .../tez/dynamic_partition_pruning.q.out         | 230 +++----
 .../tez/dynamic_partition_pruning_2.q.out       |   4 +-
 .../tez/dynpart_sort_optimization.q.out         | 104 +--
 .../tez/dynpart_sort_optimization2.q.out        |  44 +-
 .../clientpositive/tez/explainuser_1.q.out      | 148 ++--
 .../clientpositive/tez/explainuser_4.q.out      |  12 +-
 .../tez/hybridgrace_hashjoin_1.q.out            |   8 +-
 .../results/clientpositive/tez/mergejoin.q.out  |   6 +-
 ...names_with_leading_and_trailing_spaces.q.out |   2 +-
 .../results/clientpositive/tez/skewjoin.q.out   |   6 +-
 .../tez/tez_dynpart_hashjoin_1.q.out            |  12 +-
 .../clientpositive/tez/tez_self_join.q.out      |  32 +-
 .../tez/tez_union_dynamic_partition.q.out       |  44 +-
 .../tez/tez_vector_dynpart_hashjoin_1.q.out     |  12 +-
 .../clientpositive/tez/union_fast_stats.q.out   |  16 +-
 .../tez/vector_between_columns.q.out            |   4 +-
 .../tez/vector_binary_join_groupby.q.out        |   2 +-
 .../clientpositive/tez/vector_coalesce.q.out    |  10 +-
 .../clientpositive/tez/vector_date_1.q.out      |   2 +-
 .../clientpositive/tez/vector_decimal_2.q.out   |  52 +-
 .../tez/vector_decimal_round_2.q.out            |   8 +-
 .../tez/vector_groupby_mapjoin.q.out            |  28 +-
 .../clientpositive/tez/vector_interval_1.q.out  |   8 +-
 .../tez/vector_interval_arithmetic.q.out        |   5 +-
 .../tez/vector_join_filters.q.out               |   2 +-
 .../clientpositive/tez/vector_join_nulls.q.out  |   4 +-
 .../tez/vector_mapjoin_reduce.q.out             |  12 +-
 .../tez/vector_null_projection.q.out            |  10 +-
 .../clientpositive/tez/vector_outer_join1.q.out |   8 +-
 .../clientpositive/tez/vector_outer_join2.q.out |   8 +-
 .../clientpositive/tez/vector_outer_join3.q.out |   8 +-
 .../clientpositive/tez/vector_outer_join4.q.out |   8 +-
 .../clientpositive/tez/vector_outer_join5.q.out |  12 +-
 .../tez/vectorization_short_regress.q.out       |   4 +-
 .../vectorized_dynamic_partition_pruning.q.out  | 232 +++----
 .../clientpositive/udf_folder_constants.q.out   |   8 +-
 .../clientpositive/udf_unix_timestamp.q.out     |   2 +-
 .../clientpositive/union_fast_stats.q.out       |  16 +-
 .../results/clientpositive/union_offcbo.q.out   |  40 +-
 .../clientpositive/union_remove_12.q.out        |   2 +-
 .../clientpositive/union_remove_14.q.out        |   2 +-
 .../results/clientpositive/union_view.q.out     |  20 +-
 .../clientpositive/unionall_unbalancedppd.q.out |  36 +-
 .../clientpositive/vector_between_columns.q.out |   4 +-
 .../vector_binary_join_groupby.q.out            |   2 +-
 .../clientpositive/vector_coalesce.q.out        |  10 +-
 .../results/clientpositive/vector_date_1.q.out  |   2 +-
 .../clientpositive/vector_decimal_2.q.out       |  52 +-
 .../clientpositive/vector_decimal_round_2.q.out |   8 +-
 .../clientpositive/vector_groupby_mapjoin.q.out |   4 +-
 .../clientpositive/vector_interval_1.q.out      |   8 +-
 .../vector_interval_arithmetic.q.out            |   5 +-
 .../clientpositive/vector_join_filters.q.out    |   2 +-
 .../clientpositive/vector_join_nulls.q.out      |   4 +-
 .../clientpositive/vector_mapjoin_reduce.q.out  |  10 +-
 .../clientpositive/vector_null_projection.q.out |  10 +-
 .../vector_number_compare_projection.q.out      |  10 +-
 .../clientpositive/vector_outer_join1.q.out     |   8 +-
 .../clientpositive/vector_outer_join2.q.out     |   8 +-
 .../clientpositive/vector_outer_join3.q.out     |   8 +-
 .../clientpositive/vector_outer_join4.q.out     |   8 +-
 .../clientpositive/vector_outer_join5.q.out     |   8 +-
 .../vectorization_short_regress.q.out           |   4 +-
 262 files changed, 4230 insertions(+), 4395 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hive/blob/10423f51/hbase-handler/src/test/results/positive/hbase_ppd_key_range.q.out
----------------------------------------------------------------------
diff --git a/hbase-handler/src/test/results/positive/hbase_ppd_key_range.q.out b/hbase-handler/src/test/results/positive/hbase_ppd_key_range.q.out
index 27446b4..0ef0efd 100644
--- a/hbase-handler/src/test/results/positive/hbase_ppd_key_range.q.out
+++ b/hbase-handler/src/test/results/positive/hbase_ppd_key_range.q.out
@@ -438,6 +438,7 @@ STAGE PLANS:
       Map Operator Tree:
           TableScan
             alias: hbase_pushdown
+            filterExpr: (key >= '90') (type: boolean)
             Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
             Filter Operator
               predicate: (key >= '90') (type: boolean)

http://git-wip-us.apache.org/repos/asf/hive/blob/10423f51/hbase-handler/src/test/results/positive/hbase_queries.q.out
----------------------------------------------------------------------
diff --git a/hbase-handler/src/test/results/positive/hbase_queries.q.out b/hbase-handler/src/test/results/positive/hbase_queries.q.out
index a99f561..8aa5f84 100644
--- a/hbase-handler/src/test/results/positive/hbase_queries.q.out
+++ b/hbase-handler/src/test/results/positive/hbase_queries.q.out
@@ -917,7 +917,12 @@ WITH SERDEPROPERTIES (
   'hbase.columns.mapping'='cf:string', 
   'serialization.format'='1')
 TBLPROPERTIES (
+  'COLUMN_STATS_ACCURATE'='{\"BASIC_STATS\":\"true\"}', 
   'hbase.table.name'='hbase_table_0', 
+  'numFiles'='0', 
+  'numRows'='0', 
+  'rawDataSize'='0', 
+  'totalSize'='0', 
 #### A masked pattern was here ####
 PREHOOK: query: DROP TABLE IF EXISTS hbase_table_9
 PREHOOK: type: DROPTABLE

http://git-wip-us.apache.org/repos/asf/hive/blob/10423f51/ql/src/java/org/apache/hadoop/hive/ql/optimizer/ConstantPropagateProcCtx.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/ConstantPropagateProcCtx.java b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/ConstantPropagateProcCtx.java
index bc52f7b..89de234 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/ConstantPropagateProcCtx.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/ConstantPropagateProcCtx.java
@@ -216,6 +216,10 @@ public class ConstantPropagateProcCtx implements NodeProcessorCtx {
         Operator<?> parent = op.getParentOperators().get(0);
         if (op.getColumnExprMap() != null && op.getColumnExprMap().entrySet() != null) {
           for (Entry<String, ExprNodeDesc> entry : op.getColumnExprMap().entrySet()) {
+            if (op.getSchema().getPosition(entry.getKey()) == -1) {
+              // Not present
+              continue;
+            }
             ExprNodeDesc expr = entry.getValue();
             if (expr instanceof ExprNodeColumnDesc) {
               String parentColName = ((ExprNodeColumnDesc) expr).getColumn();

http://git-wip-us.apache.org/repos/asf/hive/blob/10423f51/ql/src/java/org/apache/hadoop/hive/ql/optimizer/NonBlockingOpDeDupProc.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/NonBlockingOpDeDupProc.java b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/NonBlockingOpDeDupProc.java
index 37dbe32..de4d0e4 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/NonBlockingOpDeDupProc.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/NonBlockingOpDeDupProc.java
@@ -28,7 +28,6 @@ import java.util.Map;
 import java.util.Set;
 import java.util.Stack;
 
-import org.apache.hadoop.hive.conf.HiveConf;
 import org.apache.hadoop.hive.ql.exec.FilterOperator;
 import org.apache.hadoop.hive.ql.exec.JoinOperator;
 import org.apache.hadoop.hive.ql.exec.MapJoinOperator;
@@ -109,7 +108,7 @@ public class NonBlockingOpDeDupProc extends Transform {
         if (cSEL.getColumnExprMap() == null) {
           // If the child SelectOperator does not have the ColumnExprMap,
           // we do not need to update the ColumnExprMap in the parent SelectOperator.
-          pSEL.getConf().setColList(ExprNodeDescUtils.backtrack(cSELColList, cSEL, pSEL));
+          pSEL.getConf().setColList(ExprNodeDescUtils.backtrack(cSELColList, cSEL, pSEL, true));
           pSEL.getConf().setOutputColumnNames(cSELOutputColumnNames);
         } else {
           // If the child SelectOperator has the ColumnExprMap,
@@ -121,7 +120,7 @@ public class NonBlockingOpDeDupProc extends Transform {
             String outputColumnName = cSELOutputColumnNames.get(i);
             ExprNodeDesc cSELExprNodeDesc = cSELColList.get(i);
             ExprNodeDesc newPSELExprNodeDesc =
-                ExprNodeDescUtils.backtrack(cSELExprNodeDesc, cSEL, pSEL);
+                ExprNodeDescUtils.backtrack(cSELExprNodeDesc, cSEL, pSEL, true);
             newPSELColList.add(newPSELExprNodeDesc);
             newPSELOutputColumnNames.add(outputColumnName);
             colExprMap.put(outputColumnName, newPSELExprNodeDesc);

http://git-wip-us.apache.org/repos/asf/hive/blob/10423f51/ql/src/java/org/apache/hadoop/hive/ql/optimizer/Optimizer.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/Optimizer.java b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/Optimizer.java
index bf9a0a3..5ee54b9 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/Optimizer.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/Optimizer.java
@@ -129,13 +129,11 @@ public class Optimizer {
         /* Add list bucketing pruner. */
         transformations.add(new ListBucketingPruner());
       }
-    }
-    if ((HiveConf.getBoolVar(hiveConf, HiveConf.ConfVars.HIVEOPTPPD)
-            && HiveConf.getBoolVar(hiveConf, HiveConf.ConfVars.HIVEOPTCONSTANTPROPAGATION)) ||
-            (HiveConf.getBoolVar(hiveConf, HiveConf.ConfVars.HIVEOPTCONSTANTPROPAGATION)
-                    && pctx.getContext().isCboSucceeded())) {
-      // PartitionPruner may create more folding opportunities, run ConstantPropagate again.
-      transformations.add(new ConstantPropagate());
+      if (HiveConf.getBoolVar(hiveConf, HiveConf.ConfVars.HIVEOPTCONSTANTPROPAGATION) &&
+              !pctx.getContext().isCboSucceeded()) {
+        // PartitionPruner may create more folding opportunities, run ConstantPropagate again.
+        transformations.add(new ConstantPropagate());
+      }
     }
 
     if (HiveConf.getBoolVar(hiveConf, HiveConf.ConfVars.HIVEOPTGROUPBY) ||

http://git-wip-us.apache.org/repos/asf/hive/blob/10423f51/ql/src/java/org/apache/hadoop/hive/ql/optimizer/SortedDynPartitionOptimizer.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/SortedDynPartitionOptimizer.java b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/SortedDynPartitionOptimizer.java
index 4adf7b2..36b7036 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/SortedDynPartitionOptimizer.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/SortedDynPartitionOptimizer.java
@@ -56,6 +56,7 @@ import org.apache.hadoop.hive.ql.plan.DynamicPartitionCtx;
 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.ExprNodeDescUtils;
 import org.apache.hadoop.hive.ql.plan.ExprNodeGenericFuncDesc;
 import org.apache.hadoop.hive.ql.plan.FileSinkDesc;
 import org.apache.hadoop.hive.ql.plan.ListBucketingCtx;
@@ -287,7 +288,7 @@ public class SortedDynPartitionOptimizer extends Transform {
       }
       if (op.getColumnExprMap() != null) {
         for(String dpCol : dpCols) {
-          ExprNodeDesc end = findConstantExprOrigin(dpCol, op);
+          ExprNodeDesc end = ExprNodeDescUtils.findConstantExprOrigin(dpCol, op);
           if (!(end instanceof ExprNodeConstantDesc)) {
             return false;
           }
@@ -298,37 +299,6 @@ public class SortedDynPartitionOptimizer extends Transform {
       return true;
     }
 
-    // Find the constant origin of a certain column if it is originated from a constant
-    // Otherwise, it returns the expression that originated the column
-    private ExprNodeDesc findConstantExprOrigin(String dpCol, Operator<? extends OperatorDesc> op) {
-      ExprNodeDesc expr = op.getColumnExprMap().get(dpCol);
-      ExprNodeDesc foldedExpr;
-      // If it is a function, we try to fold it
-      if (expr instanceof ExprNodeGenericFuncDesc) {
-        foldedExpr = ConstantPropagateProcFactory.foldExpr((ExprNodeGenericFuncDesc)expr);
-        if (foldedExpr == null) {
-          foldedExpr = expr;
-        }
-      } else {
-        foldedExpr = expr;
-      }
-      // If it is a column reference, we will try to resolve it
-      if (foldedExpr instanceof ExprNodeColumnDesc) {
-        Operator<? extends OperatorDesc> originOp = null;
-        for(Operator<? extends OperatorDesc> parentOp : op.getParentOperators()) {
-          if (parentOp.getColumnExprMap() != null) {
-            originOp = parentOp;
-            break;
-          }
-        }
-        if (originOp != null) {
-          return findConstantExprOrigin(((ExprNodeColumnDesc)foldedExpr).getColumn(), originOp);
-        }
-      }
-      // Otherwise, we return the expression
-      return foldedExpr;
-    }
-
     // Remove RS and SEL introduced by enforce bucketing/sorting config
     // Convert PARENT -> RS -> SEL -> FS to PARENT -> FS
     private boolean removeRSInsertedByEnforceBucketing(FileSinkOperator fsOp) {

http://git-wip-us.apache.org/repos/asf/hive/blob/10423f51/ql/src/java/org/apache/hadoop/hive/ql/optimizer/StatsOptimizer.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/StatsOptimizer.java b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/StatsOptimizer.java
index 0cfd529..7febfd5 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/StatsOptimizer.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/StatsOptimizer.java
@@ -22,7 +22,6 @@ import java.util.Collection;
 import java.util.Collections;
 import java.util.Comparator;
 import java.util.HashMap;
-import java.util.HashSet;
 import java.util.LinkedHashMap;
 import java.util.List;
 import java.util.Map;
@@ -30,8 +29,6 @@ import java.util.Map.Entry;
 import java.util.Set;
 import java.util.Stack;
 
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
 import org.apache.hadoop.hive.common.StatsSetupConst;
 import org.apache.hadoop.hive.common.type.HiveDecimal;
 import org.apache.hadoop.hive.metastore.api.ColumnStatisticsData;
@@ -67,6 +64,7 @@ import org.apache.hadoop.hive.ql.plan.AggregationDesc;
 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.ExprNodeDescUtils;
 import org.apache.hadoop.hive.ql.plan.FetchWork;
 import org.apache.hadoop.hive.ql.plan.GroupByDesc;
 import org.apache.hadoop.hive.ql.udf.generic.GenericUDAFCount;
@@ -81,6 +79,8 @@ import org.apache.hadoop.hive.serde2.objectinspector.PrimitiveObjectInspector.Pr
 import org.apache.hadoop.hive.serde2.objectinspector.StandardStructObjectInspector;
 import org.apache.hadoop.hive.serde2.typeinfo.TypeInfoUtils;
 import org.apache.thrift.TException;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 import com.google.common.collect.Lists;
 
@@ -213,13 +213,17 @@ public class StatsOptimizer extends Transform {
 
     private boolean hasNullOrConstantGbyKey(GroupByOperator gbyOp) {
       GroupByDesc gbyDesc = gbyOp.getConf();
+      int numCols = gbyDesc.getOutputColumnNames().size();
+      int aggCols = gbyDesc.getAggregators().size();
       // If the Group by operator has null key
-      if (gbyDesc.getOutputColumnNames().size() ==
-        gbyDesc.getAggregators().size()) {
+      if (numCols == aggCols) {
         return true;
       }
-      for (ExprNodeDesc en :gbyDesc.getKeys()) {
-        if (!(en instanceof ExprNodeConstantDesc)) {
+      // If the Gby key is a constant
+      List<String> dpCols = gbyOp.getSchema().getColumnNames().subList(0, numCols - aggCols);
+      for(String dpCol : dpCols) {
+        ExprNodeDesc end = ExprNodeDescUtils.findConstantExprOrigin(dpCol, gbyOp);
+        if (!(end instanceof ExprNodeConstantDesc)) {
           return false;
         }
       }

http://git-wip-us.apache.org/repos/asf/hive/blob/10423f51/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/HiveRelOptUtil.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/HiveRelOptUtil.java b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/HiveRelOptUtil.java
index c6d1d46..4c154d0 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/HiveRelOptUtil.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/HiveRelOptUtil.java
@@ -35,12 +35,12 @@ import org.apache.calcite.sql.SqlKind;
 import org.apache.calcite.sql.SqlOperator;
 import org.apache.calcite.sql.fun.SqlStdOperatorTable;
 import org.apache.calcite.util.ImmutableBitSet;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
 import org.apache.hadoop.hive.ql.exec.FunctionRegistry;
 import org.apache.hadoop.hive.ql.optimizer.calcite.translator.TypeConverter;
 import org.apache.hadoop.hive.serde2.typeinfo.TypeInfo;
 import org.apache.hadoop.hive.serde2.typeinfo.TypeInfoUtils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 
 public class HiveRelOptUtil extends RelOptUtil {
@@ -105,23 +105,6 @@ public class HiveRelOptUtil extends RelOptUtil {
     final RelOptCluster cluster = inputs.get(0).getCluster();
     final RexBuilder rexBuilder = cluster.getRexBuilder();
 
-    final ImmutableBitSet[] inputsRange = new ImmutableBitSet[inputs.size()];
-    int totalFieldCount = 0;
-    for (int i = 0; i < inputs.size(); i++) {
-      final int firstField = totalFieldCount + sysFieldCount;
-      totalFieldCount = firstField + inputs.get(i).getRowType().getFieldCount();
-      inputsRange[i] = ImmutableBitSet.range(firstField, totalFieldCount);
-    }
-
-    // adjustment array
-    int[] adjustments = new int[totalFieldCount];
-    for (int i = 0; i < inputs.size(); i++) {
-      final int adjustment = inputsRange[i].nextSetBit(0);
-      for (int j = adjustment; j < inputsRange[i].length(); j++) {
-        adjustments[j] = -adjustment;
-      }
-    }
-
     if (condition instanceof RexCall) {
       RexCall call = (RexCall) condition;
       if (call.getOperator() == SqlStdOperatorTable.AND) {
@@ -165,6 +148,14 @@ public class HiveRelOptUtil extends RelOptUtil {
         final ImmutableBitSet projRefs0 = InputFinder.bits(op0);
         final ImmutableBitSet projRefs1 = InputFinder.bits(op1);
 
+        final ImmutableBitSet[] inputsRange = new ImmutableBitSet[inputs.size()];
+        int totalFieldCount = 0;
+        for (int i = 0; i < inputs.size(); i++) {
+          final int firstField = totalFieldCount + sysFieldCount;
+          totalFieldCount = firstField + inputs.get(i).getRowType().getFieldCount();
+          inputsRange[i] = ImmutableBitSet.range(firstField, totalFieldCount);
+        }
+
         boolean foundBothInputs = false;
         for (int i = 0; i < inputs.size() && !foundBothInputs; i++) {
           if (projRefs0.intersects(inputsRange[i])
@@ -196,6 +187,15 @@ public class HiveRelOptUtil extends RelOptUtil {
         }
 
         if ((leftKey != null) && (rightKey != null)) {
+          // adjustment array
+          int[] adjustments = new int[totalFieldCount];
+          for (int i = 0; i < inputs.size(); i++) {
+            final int adjustment = inputsRange[i].nextSetBit(0);
+            for (int j = adjustment; j < inputsRange[i].length(); j++) {
+              adjustments[j] = -adjustment;
+            }
+          }
+
           // replace right Key input ref
           rightKey =
               rightKey.accept(

http://git-wip-us.apache.org/repos/asf/hive/blob/10423f51/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/HiveRexUtil.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/HiveRexUtil.java b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/HiveRexUtil.java
index 2f309f3..6933fec 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/HiveRexUtil.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/HiveRexUtil.java
@@ -19,8 +19,11 @@ package org.apache.hadoop.hive.ql.optimizer.calcite;
 
 import java.util.ArrayList;
 import java.util.Collections;
+import java.util.HashMap;
 import java.util.HashSet;
+import java.util.LinkedHashSet;
 import java.util.List;
+import java.util.Map;
 import java.util.Set;
 
 import org.apache.calcite.linq4j.Ord;
@@ -29,12 +32,16 @@ import org.apache.calcite.rex.RexBuilder;
 import org.apache.calcite.rex.RexCall;
 import org.apache.calcite.rex.RexLiteral;
 import org.apache.calcite.rex.RexNode;
+import org.apache.calcite.rex.RexShuttle;
 import org.apache.calcite.rex.RexUtil;
 import org.apache.calcite.sql.SqlKind;
+import org.apache.calcite.sql.SqlOperator;
 import org.apache.calcite.sql.fun.SqlStdOperatorTable;
 import org.apache.calcite.sql.type.SqlTypeName;
 import org.apache.calcite.util.Pair;
 import org.apache.calcite.util.Util;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 import com.google.common.collect.ImmutableList;
 import com.google.common.collect.Lists;
@@ -42,6 +49,8 @@ import com.google.common.collect.Lists;
 
 public class HiveRexUtil {
 
+  protected static final Logger LOG = LoggerFactory.getLogger(HiveRexUtil.class);
+
   /**
    * Simplifies a boolean expression.
    *
@@ -54,13 +63,20 @@ public class HiveRexUtil {
    * </ul>
    */
   public static RexNode simplify(RexBuilder rexBuilder, RexNode e) {
+    return simplify(rexBuilder, e, false);
+  }
+
+  public static RexNode simplify(RexBuilder rexBuilder, RexNode e,
+          boolean unknownAsFalse) {
     switch (e.getKind()) {
     case AND:
-      return simplifyAnd(rexBuilder, (RexCall) e);
+      return simplifyAnd(rexBuilder, (RexCall) e, unknownAsFalse);
     case OR:
       return simplifyOr(rexBuilder, (RexCall) e);
+    case NOT:
+      return simplifyNot(rexBuilder, (RexCall) e);
     case CASE:
-      return simplifyCase(rexBuilder, (RexCall) e);
+      return simplifyCase(rexBuilder, (RexCall) e, unknownAsFalse);
     case IS_NULL:
       return ((RexCall) e).getOperands().get(0).getType().isNullable()
           ? e : rexBuilder.makeLiteral(false);
@@ -72,32 +88,67 @@ public class HiveRexUtil {
     }
   }
 
-  private static RexNode simplifyCase(RexBuilder rexBuilder, RexCall call) {
+  private static RexNode simplifyNot(RexBuilder rexBuilder, RexCall call) {
+    final RexNode a = call.getOperands().get(0);
+    switch (a.getKind()) {
+    case NOT:
+      // NOT NOT x ==> x
+      return simplify(rexBuilder, ((RexCall) a).getOperands().get(0));
+    }
+    final SqlKind negateKind = a.getKind().negate();
+    if (a.getKind() != negateKind) {
+      return simplify(rexBuilder,
+          rexBuilder.makeCall(op(negateKind),
+              ImmutableList.of(((RexCall) a).getOperands().get(0))));
+    }
+    final SqlKind negateKind2 = negate(a.getKind());
+    if (a.getKind() != negateKind2) {
+      return simplify(rexBuilder,
+          rexBuilder.makeCall(op(negateKind2), ((RexCall) a).getOperands()));
+    }
+    return call;
+  }
+
+  private static RexNode simplifyCase(RexBuilder rexBuilder, RexCall call,
+          boolean unknownAsFalse) {
     final List<RexNode> operands = call.getOperands();
     final List<RexNode> newOperands = new ArrayList<>();
+    final Set<String> values = new HashSet<>();
+    boolean constainsNullableCase = false;
     for (int i = 0; i < operands.size(); i++) {
       RexNode operand = operands.get(i);
       if (RexUtil.isCasePredicate(call, i)) {
         if (operand.isAlwaysTrue()) {
           // Predicate is always TRUE. Make value the ELSE and quit.
           newOperands.add(operands.get(i + 1));
+          if (operand.getType().isNullable()) {
+            constainsNullableCase = true;
+          }
           break;
-        }
-        if (operand.isAlwaysFalse()) {
-          // Predicate is always FALSE. Skip predicate and value.
+        } else if (operand.isAlwaysFalse() || RexUtil.isNull(operand)) {
+          // Predicate is always FALSE or NULL. Skip predicate and value.
           ++i;
           continue;
         }
+        if (operand.getType().isNullable()) {
+          constainsNullableCase = true;
+        }
+      } else {
+        if (unknownAsFalse && RexUtil.isNull(operand)) {
+          values.add(rexBuilder.makeLiteral(false).toString());
+        } else {
+          values.add(operand.toString());
+        }
       }
       newOperands.add(operand);
     }
     assert newOperands.size() % 2 == 1;
-    switch (newOperands.size()) {
-    case 1:
-      return rexBuilder.makeCast(call.getType(), newOperands.get(0));
+    if (newOperands.size() == 1 || values.size() == 1) {
+      return rexBuilder.makeCast(call.getType(), newOperands.get(newOperands.size() - 1));
     }
   trueFalse:
-    if (call.getType().getSqlTypeName() == SqlTypeName.BOOLEAN) {
+    if (call.getType().getSqlTypeName() == SqlTypeName.BOOLEAN &&
+            (!constainsNullableCase || unknownAsFalse)) {
       // Optimize CASE where every branch returns constant true or constant
       // false:
       //   CASE
@@ -110,7 +161,8 @@ public class HiveRexUtil {
           casePairs(rexBuilder, newOperands);
       for (Ord<Pair<RexNode, RexNode>> pair : Ord.zip(pairs)) {
         if (!pair.e.getValue().isAlwaysTrue()
-            && !pair.e.getValue().isAlwaysFalse()) {
+            && !pair.e.getValue().isAlwaysFalse()
+            && (!unknownAsFalse || !RexUtil.isNull(pair.e.getValue()))) {
           break trueFalse;
         }
       }
@@ -145,33 +197,76 @@ public class HiveRexUtil {
     return builder.build();
   }
 
-  public static RexNode simplifyAnd(RexBuilder rexBuilder, RexCall e) {
-    final List<RexNode> terms = RelOptUtil.conjunctions(e);
+  public static RexNode simplifyAnd(RexBuilder rexBuilder, RexCall e,
+          boolean unknownAsFalse) {
+    final List<RexNode> terms = new ArrayList<>();
     final List<RexNode> notTerms = new ArrayList<>();
-    final List<RexNode> negatedTerms = new ArrayList<>();
-    final List<RexNode> nullOperands = new ArrayList<>();
-    final List<RexNode> notNullOperands = new ArrayList<>();
-    final Set<RexNode> comparedOperands = new HashSet<>();
+    RelOptUtil.decomposeConjunction(e, terms, notTerms);
+    if (unknownAsFalse) {
+      return simplifyAnd2ForUnknownAsFalse(rexBuilder, terms, notTerms);
+    }
+    return simplifyAnd2(rexBuilder, terms, notTerms);
+  }
+
+  public static RexNode simplifyAnd2(RexBuilder rexBuilder,
+      List<RexNode> terms, List<RexNode> notTerms) {
+    if (terms.contains(rexBuilder.makeLiteral(false))) {
+      return rexBuilder.makeLiteral(false);
+    }
+    if (terms.isEmpty() && notTerms.isEmpty()) {
+      return rexBuilder.makeLiteral(true);
+    }
+    if (terms.size() == 1 && notTerms.isEmpty()) {
+      // Make sure "x OR y OR x" (a single-term conjunction) gets simplified.
+      return simplify(rexBuilder, terms.get(0));
+    }
+    // If one of the not-disjunctions is a disjunction that is wholly
+    // contained in the disjunctions list, the expression is not
+    // satisfiable.
+    //
+    // Example #1. x AND y AND z AND NOT (x AND y)  - not satisfiable
+    // Example #2. x AND y AND NOT (x AND y)        - not satisfiable
+    // Example #3. x AND y AND NOT (x AND y AND z)  - may be satisfiable
+    for (RexNode notDisjunction : notTerms) {
+      final List<RexNode> terms2 = RelOptUtil.conjunctions(notDisjunction);
+      if (terms.containsAll(terms2)) {
+        return rexBuilder.makeLiteral(false);
+      }
+    }
+    // Add the NOT disjunctions back in.
+    for (RexNode notDisjunction : notTerms) {
+      terms.add(
+          simplify(rexBuilder,
+              rexBuilder.makeCall(SqlStdOperatorTable.NOT, notDisjunction)));
+    }
+    return RexUtil.composeConjunction(rexBuilder, terms, false);
+  }
+
+  /** As {@link #simplifyAnd2(RexBuilder, List, List)} but we assume that if the expression returns
+   * UNKNOWN it will be interpreted as FALSE. */
+  public static RexNode simplifyAnd2ForUnknownAsFalse(RexBuilder rexBuilder,
+      List<RexNode> terms, List<RexNode> notTerms) {
+    if (terms.contains(rexBuilder.makeLiteral(false))) {
+      return rexBuilder.makeLiteral(false);
+    }
+    if (terms.isEmpty() && notTerms.isEmpty()) {
+      return rexBuilder.makeLiteral(true);
+    }
+    if (terms.size() == 1 && notTerms.isEmpty()) {
+      // Make sure "x OR y OR x" (a single-term conjunction) gets simplified.
+      return simplify(rexBuilder, terms.get(0), true);
+    }
+    // Try to simplify the expression
+    final Set<String> negatedTerms = new HashSet<>();
+    final Set<String> nullOperands = new HashSet<>();
+    final Set<RexNode> notNullOperands = new LinkedHashSet<>();
+    final Set<String> comparedOperands = new HashSet<>();
     for (int i = 0; i < terms.size(); i++) {
       final RexNode term = terms.get(i);
       if (!HiveCalciteUtil.isDeterministic(term)) {
         continue;
       }
       switch (term.getKind()) {
-      case NOT:
-        notTerms.add(
-            ((RexCall) term).getOperands().get(0));
-        terms.remove(i);
-        --i;
-        break;
-      case LITERAL:
-        if (!RexLiteral.booleanValue(term)) {
-          return term; // false
-        } else {
-          terms.remove(i);
-          --i;
-        }
-        break;
       case EQUALS:
       case NOT_EQUALS:
       case LESS_THAN:
@@ -180,53 +275,48 @@ public class HiveRexUtil {
       case GREATER_THAN_OR_EQUAL:
         RexCall call = (RexCall) term;
         RexNode left = call.getOperands().get(0);
-        comparedOperands.add(left);
+        comparedOperands.add(left.toString());
         // if it is a cast, we include the inner reference
         if (left.getKind() == SqlKind.CAST) {
           RexCall leftCast = (RexCall) left;
-          comparedOperands.add(leftCast.getOperands().get(0));
+          comparedOperands.add(leftCast.getOperands().get(0).toString());
         }
         RexNode right = call.getOperands().get(1);
-        comparedOperands.add(right);
+        comparedOperands.add(right.toString());
         // if it is a cast, we include the inner reference
         if (right.getKind() == SqlKind.CAST) {
           RexCall rightCast = (RexCall) right;
-          comparedOperands.add(rightCast.getOperands().get(0));
+          comparedOperands.add(rightCast.getOperands().get(0).toString());
         }
-        // Assume we have the expression a > 5.
-        // Then we can derive the negated term: NOT(a <= 5).
+        // Assume the expression a > 5 is part of a Filter condition.
+        // Then we can derive the negated term: a <= 5.
         // But as the comparison is string based and thus operands order dependent,
-        // we should also add the inverted negated term: NOT(5 >= a).
+        // we should also add the inverted negated term: 5 >= a.
         // Observe that for creating the inverted term we invert the list of operands.
-        RexCall negatedTerm = negate(rexBuilder, call);
+        RexNode negatedTerm = negate(rexBuilder, call);
         if (negatedTerm != null) {
-          negatedTerms.add(negatedTerm);
-          RexCall invertNegatedTerm = invert(rexBuilder, negatedTerm);
+          negatedTerms.add(negatedTerm.toString());
+          RexNode invertNegatedTerm = invert(rexBuilder, (RexCall) negatedTerm);
           if (invertNegatedTerm != null) {
-            negatedTerms.add(invertNegatedTerm);
+            negatedTerms.add(invertNegatedTerm.toString());
           }
         }
         break;
       case IN:
-        comparedOperands.add(((RexCall) term).operands.get(0));
+        comparedOperands.add(((RexCall) term).operands.get(0).toString());
         break;
       case BETWEEN:
-        comparedOperands.add(((RexCall) term).operands.get(1));
+        comparedOperands.add(((RexCall) term).operands.get(1).toString());
         break;
       case IS_NOT_NULL:
-        notNullOperands.add(
-                ((RexCall) term).getOperands().get(0));
+        notNullOperands.add(((RexCall) term).getOperands().get(0));
         terms.remove(i);
         --i;
         break;
       case IS_NULL:
-        nullOperands.add(
-                ((RexCall) term).getOperands().get(0));
+        nullOperands.add(((RexCall) term).getOperands().get(0).toString());
       }
     }
-    if (terms.isEmpty() && notTerms.isEmpty() && notNullOperands.isEmpty()) {
-      return rexBuilder.makeLiteral(true);
-    }
     // If one column should be null and is in a comparison predicate,
     // it is not satisfiable.
     // Example. IS NULL(x) AND x < 5  - not satisfiable
@@ -237,10 +327,9 @@ public class HiveRexUtil {
     //
     // Example. IS NOT NULL(x) AND x < 5  : x < 5
     for (RexNode operand : notNullOperands) {
-      if (!comparedOperands.contains(operand)) {
+      if (!comparedOperands.contains(operand.toString())) {
         terms.add(
-            rexBuilder.makeCall(
-                SqlStdOperatorTable.IS_NOT_NULL, operand));
+            rexBuilder.makeCall(SqlStdOperatorTable.IS_NOT_NULL, operand));
       }
     }
     // If one of the not-disjunctions is a disjunction that is wholly
@@ -253,23 +342,24 @@ public class HiveRexUtil {
     final Set<String> termsSet = new HashSet<String>(
             Lists.transform(terms, HiveCalciteUtil.REX_STR_FN));
     for (RexNode notDisjunction : notTerms) {
-      final Set<String> notSet = new HashSet<String>(
-              Lists.transform(RelOptUtil.conjunctions(notDisjunction), HiveCalciteUtil.REX_STR_FN));
-      if (termsSet.containsAll(notSet)) {
+      if (!HiveCalciteUtil.isDeterministic(notDisjunction)) {
+        continue;
+      }
+      final List<String> terms2Set = Lists.transform(
+              RelOptUtil.conjunctions(notDisjunction), HiveCalciteUtil.REX_STR_FN);
+      if (termsSet.containsAll(terms2Set)) {
         return rexBuilder.makeLiteral(false);
       }
     }
     // Add the NOT disjunctions back in.
     for (RexNode notDisjunction : notTerms) {
       terms.add(
-          rexBuilder.makeCall(
-              SqlStdOperatorTable.NOT, notDisjunction));
+          simplify(rexBuilder,
+              rexBuilder.makeCall(SqlStdOperatorTable.NOT, notDisjunction), true));
     }
-    // The negated terms
-    for (RexNode notDisjunction : negatedTerms) {
-      final Set<String> notSet = new HashSet<String>(
-              Lists.transform(RelOptUtil.conjunctions(notDisjunction), HiveCalciteUtil.REX_STR_FN));
-      if (termsSet.containsAll(notSet)) {
+    // The negated terms: only deterministic expressions
+    for (String negatedTerm : negatedTerms) {
+      if (termsSet.contains(negatedTerm)) {
         return rexBuilder.makeLiteral(false);
       }
     }
@@ -284,11 +374,13 @@ public class HiveRexUtil {
       final RexNode term = terms.get(i);
       switch (term.getKind()) {
       case LITERAL:
-        if (RexLiteral.booleanValue(term)) {
-          return term; // true
-        } else {
-          terms.remove(i);
-          --i;
+        if (!RexLiteral.isNullLiteral(term)) {
+          if (RexLiteral.booleanValue(term)) {
+            return term; // true
+          } else {
+            terms.remove(i);
+            --i;
+          }
         }
       }
     }
@@ -298,21 +390,34 @@ public class HiveRexUtil {
   private static RexCall negate(RexBuilder rexBuilder, RexCall call) {
     switch (call.getKind()) {
       case EQUALS:
-        return (RexCall) rexBuilder.makeCall(SqlStdOperatorTable.NOT_EQUALS, call.getOperands());
       case NOT_EQUALS:
-        return (RexCall) rexBuilder.makeCall(SqlStdOperatorTable.EQUALS, call.getOperands());
       case LESS_THAN:
-        return (RexCall) rexBuilder.makeCall(SqlStdOperatorTable.GREATER_THAN_OR_EQUAL, call.getOperands());
       case GREATER_THAN:
-        return (RexCall) rexBuilder.makeCall(SqlStdOperatorTable.LESS_THAN_OR_EQUAL, call.getOperands());
       case LESS_THAN_OR_EQUAL:
-        return (RexCall) rexBuilder.makeCall(SqlStdOperatorTable.GREATER_THAN, call.getOperands());
       case GREATER_THAN_OR_EQUAL:
-        return (RexCall) rexBuilder.makeCall(SqlStdOperatorTable.LESS_THAN, call.getOperands());
+        return (RexCall) rexBuilder.makeCall(op(negate(call.getKind())), call.getOperands());
     }
     return null;
   }
 
+  private static SqlKind negate(SqlKind kind) {
+    switch (kind) {
+      case EQUALS:
+        return SqlKind.NOT_EQUALS;
+      case NOT_EQUALS:
+        return SqlKind.EQUALS;
+      case LESS_THAN:
+        return SqlKind.GREATER_THAN_OR_EQUAL;
+      case GREATER_THAN:
+        return SqlKind.LESS_THAN_OR_EQUAL;
+      case LESS_THAN_OR_EQUAL:
+        return SqlKind.GREATER_THAN;
+      case GREATER_THAN_OR_EQUAL:
+        return SqlKind.LESS_THAN;
+    }
+    return kind;
+  }
+
   private static RexCall invert(RexBuilder rexBuilder, RexCall call) {
     switch (call.getKind()) {
       case LESS_THAN:
@@ -330,4 +435,81 @@ public class HiveRexUtil {
     }
     return null;
   }
+
+  private static SqlOperator op(SqlKind kind) {
+    switch (kind) {
+    case IS_FALSE:
+      return SqlStdOperatorTable.IS_FALSE;
+    case IS_TRUE:
+      return SqlStdOperatorTable.IS_TRUE;
+    case IS_UNKNOWN:
+      return SqlStdOperatorTable.IS_UNKNOWN;
+    case IS_NULL:
+      return SqlStdOperatorTable.IS_NULL;
+    case IS_NOT_FALSE:
+      return SqlStdOperatorTable.IS_NOT_FALSE;
+    case IS_NOT_TRUE:
+      return SqlStdOperatorTable.IS_NOT_TRUE;
+    case IS_NOT_NULL:
+      return SqlStdOperatorTable.IS_NOT_NULL;
+    case EQUALS:
+      return SqlStdOperatorTable.EQUALS;
+    case NOT_EQUALS:
+      return SqlStdOperatorTable.NOT_EQUALS;
+    case LESS_THAN:
+      return SqlStdOperatorTable.LESS_THAN;
+    case GREATER_THAN:
+      return SqlStdOperatorTable.GREATER_THAN;
+    case LESS_THAN_OR_EQUAL:
+      return SqlStdOperatorTable.LESS_THAN_OR_EQUAL;
+    case GREATER_THAN_OR_EQUAL:
+      return SqlStdOperatorTable.GREATER_THAN_OR_EQUAL;
+    default:
+      throw new AssertionError(kind);
+    }
+  }
+
+  public static class ExprSimplifier extends RexShuttle {
+    private final RexBuilder rexBuilder;
+    private final boolean unknownAsFalse;
+    private final Map<RexNode,Boolean> unknownAsFalseMap;
+
+    public ExprSimplifier(RexBuilder rexBuilder, boolean unknownAsFalse) {
+      this.rexBuilder = rexBuilder;
+      this.unknownAsFalse = unknownAsFalse;
+      this.unknownAsFalseMap = new HashMap<>();
+    }
+
+    @Override
+    public RexNode visitCall(RexCall call) {
+      Boolean unknownAsFalseCall = unknownAsFalse;
+      if (unknownAsFalseCall) {
+        switch (call.getKind()) {
+        case AND:
+        case CASE:
+          unknownAsFalseCall = this.unknownAsFalseMap.get(call);
+          if (unknownAsFalseCall == null) {
+            // Top operator
+            unknownAsFalseCall = true;
+          }
+          break;
+        default:
+          unknownAsFalseCall = false;
+        }
+        for (RexNode operand : call.operands) {
+          this.unknownAsFalseMap.put(operand, unknownAsFalseCall);
+        }
+      }
+      RexNode node = super.visitCall(call);
+      RexNode simplifiedNode = HiveRexUtil.simplify(rexBuilder, node, unknownAsFalseCall);
+      if (node == simplifiedNode) {
+        return node;
+      }
+      if (simplifiedNode.getType().equals(call.getType())) {
+        return simplifiedNode;
+      }
+      return rexBuilder.makeCast(call.getType(), simplifiedNode, true);
+    }
+  }
+
 }

http://git-wip-us.apache.org/repos/asf/hive/blob/10423f51/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/rules/HiveAggregatePullUpConstantsRule.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/rules/HiveAggregatePullUpConstantsRule.java b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/rules/HiveAggregatePullUpConstantsRule.java
new file mode 100644
index 0000000..370c0ec
--- /dev/null
+++ b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/rules/HiveAggregatePullUpConstantsRule.java
@@ -0,0 +1,47 @@
+/**
+ * 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.calcite.rules;
+
+import org.apache.calcite.plan.RelOptRuleCall;
+import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rel.core.Aggregate;
+import org.apache.calcite.rel.rules.AggregateProjectPullUpConstantsRule;
+import org.apache.hadoop.hive.ql.optimizer.calcite.HiveRelFactories;
+import org.apache.hadoop.hive.ql.optimizer.calcite.reloperators.HiveAggregate;
+
+public class HiveAggregatePullUpConstantsRule extends AggregateProjectPullUpConstantsRule {
+
+  public static final HiveAggregatePullUpConstantsRule INSTANCE =
+          new HiveAggregatePullUpConstantsRule();
+
+  public HiveAggregatePullUpConstantsRule() {
+    super(HiveAggregate.class, RelNode.class,
+            HiveRelFactories.HIVE_BUILDER, "HiveAggregatePullUpConstantsRule");
+  }
+
+  @Override
+  public boolean matches(RelOptRuleCall call) {
+    final Aggregate aggregate = call.rel(0);
+    // Rule cannot be applied if there are GroupingSets
+    if (aggregate.indicator) {
+      return false;
+    }
+    return super.matches(call);
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hive/blob/10423f51/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/rules/HiveProjectFilterPullUpConstantsRule.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/rules/HiveProjectFilterPullUpConstantsRule.java b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/rules/HiveProjectFilterPullUpConstantsRule.java
new file mode 100644
index 0000000..defab09
--- /dev/null
+++ b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/rules/HiveProjectFilterPullUpConstantsRule.java
@@ -0,0 +1,177 @@
+/**
+ * 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.calcite.rules;
+
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+
+import org.apache.calcite.plan.RelOptRule;
+import org.apache.calcite.plan.RelOptRuleCall;
+import org.apache.calcite.plan.RelOptUtil;
+import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rel.core.Filter;
+import org.apache.calcite.rel.core.Project;
+import org.apache.calcite.rex.RexBuilder;
+import org.apache.calcite.rex.RexCall;
+import org.apache.calcite.rex.RexInputRef;
+import org.apache.calcite.rex.RexNode;
+import org.apache.calcite.rex.RexShuttle;
+import org.apache.calcite.rex.RexUtil;
+import org.apache.calcite.tools.RelBuilder;
+import org.apache.calcite.tools.RelBuilderFactory;
+import org.apache.hadoop.hive.ql.optimizer.calcite.HiveCalciteUtil;
+import org.apache.hadoop.hive.ql.optimizer.calcite.HiveRelFactories;
+import org.apache.hadoop.hive.ql.optimizer.calcite.reloperators.HiveFilter;
+import org.apache.hadoop.hive.ql.optimizer.calcite.reloperators.HiveProject;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import com.google.common.collect.Lists;
+
+/**
+ * Planner rule that infers constant expressions from Filter into
+ * a Project operator.
+ */
+public class HiveProjectFilterPullUpConstantsRule extends RelOptRule {
+
+  protected static final Logger LOG = LoggerFactory.getLogger(
+          HiveProjectFilterPullUpConstantsRule.class);
+
+  public static final HiveProjectFilterPullUpConstantsRule INSTANCE =
+          new HiveProjectFilterPullUpConstantsRule(HiveProject.class, HiveFilter.class,
+                  HiveRelFactories.HIVE_BUILDER);
+
+  public HiveProjectFilterPullUpConstantsRule(
+      Class<? extends Project> projectClass,
+      Class<? extends Filter> filterClass,
+      RelBuilderFactory relBuilderFactory) {
+    super(operand(projectClass,
+            operand(filterClass, any())),
+            relBuilderFactory, null);
+  }
+
+  @Override
+  public boolean matches(RelOptRuleCall call) {
+    final Filter filterRel = call.rel(1);
+    RexNode condition = filterRel.getCondition();
+    if (!HiveCalciteUtil.isDeterministic(condition)) {
+      return false;
+    }
+
+    return super.matches(call);
+  }
+
+  public void onMatch(RelOptRuleCall call) {
+    final Project project = call.rel(0);
+    final Filter filter = call.rel(1);
+    final RelBuilder builder = call.builder();
+
+    List<RexNode> projects = project.getChildExps();
+    List<RexNode> newProjects = rewriteProjects(projects, filter.getCondition(), builder);
+    if (newProjects == null) {
+      return;
+    }
+
+    RelNode newProjRel = builder.push(filter)
+            .project(newProjects, project.getRowType().getFieldNames()).build();
+    call.transformTo(newProjRel);
+  }
+
+  // Rewrite projects to replace column references by constants when possible
+  @SuppressWarnings("incomplete-switch")
+  private static List<RexNode> rewriteProjects(List<RexNode> projects,
+          RexNode newPushedCondition, RelBuilder relBuilder) {
+    final List<RexNode> conjunctions = RelOptUtil.conjunctions(newPushedCondition);
+    final Map<String, RexNode> conditions = new HashMap<String, RexNode>();
+    for (RexNode conjunction: conjunctions) {
+      // 1.1. If it is not a RexCall, we continue
+      if (!(conjunction instanceof RexCall)) {
+        continue;
+      }
+      // 1.2. We extract the information that we need
+      RexCall conjCall = (RexCall) conjunction;
+      switch (conjCall.getOperator().getKind()) {
+        case EQUALS:
+          if (!(RexUtil.isConstant(conjCall.operands.get(0))) &&
+                  RexUtil.isConstant(conjCall.operands.get(1))) {
+            conditions.put(conjCall.operands.get(0).toString(), conjCall.operands.get(1));
+          } else if (!(RexUtil.isConstant(conjCall.operands.get(1))) &&
+                  RexUtil.isConstant(conjCall.operands.get(0))) {
+            conditions.put(conjCall.operands.get(1).toString(), conjCall.operands.get(0));
+          }
+          break;
+        case IS_NULL:
+          conditions.put(conjCall.operands.get(0).toString(),
+                  relBuilder.getRexBuilder().makeNullLiteral(
+                          conjCall.operands.get(0).getType().getSqlTypeName()));
+      }
+    }
+
+    RexReplacer replacer = new RexReplacer(relBuilder.getRexBuilder(), conditions);
+    List<RexNode> newProjects = Lists.newArrayList(projects);
+    replacer.mutate(newProjects);
+    if (replacer.replaced) {
+      return newProjects;
+    }
+    return null;
+  }
+
+  protected static class RexReplacer extends RexShuttle {
+    private final RexBuilder rexBuilder;
+    private final Map<String, RexNode> replacements;
+    private boolean replaced;
+
+    RexReplacer(
+        RexBuilder rexBuilder,
+        Map<String, RexNode> replacements) {
+      this.rexBuilder = rexBuilder;
+      this.replacements = replacements;
+      this.replaced = false;
+    }
+
+    @Override public RexNode visitInputRef(RexInputRef inputRef) {
+      RexNode node = visit(inputRef);
+      if (node == null) {
+        return super.visitInputRef(inputRef);
+      }
+      this.replaced = true;
+      return node;
+    }
+
+    @Override public RexNode visitCall(RexCall call) {
+      RexNode node = visit(call);
+      if (node != null) {
+        this.replaced = true;
+        return node;
+      }
+      return super.visitCall(call);
+    }
+
+    private RexNode visit(final RexNode call) {
+      RexNode replacement = replacements.get(call.toString());
+      if (replacement == null) {
+        return null;
+      }
+      if (replacement.getType().equals(call.getType())) {
+        return replacement;
+      }
+      return rexBuilder.makeCast(call.getType(), replacement, true);
+    }
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hive/blob/10423f51/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/rules/HiveReduceExpressionsRule.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/rules/HiveReduceExpressionsRule.java b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/rules/HiveReduceExpressionsRule.java
index 2fe9b75..514ae62 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/rules/HiveReduceExpressionsRule.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/rules/HiveReduceExpressionsRule.java
@@ -59,11 +59,15 @@ import org.apache.calcite.tools.RelBuilderFactory;
 import org.apache.calcite.util.Pair;
 import org.apache.calcite.util.Stacks;
 import org.apache.calcite.util.Util;
+import org.apache.hadoop.hive.ql.optimizer.calcite.HiveCalciteUtil;
 import org.apache.hadoop.hive.ql.optimizer.calcite.HiveRelFactories;
 import org.apache.hadoop.hive.ql.optimizer.calcite.HiveRexUtil;
+import org.apache.hadoop.hive.ql.optimizer.calcite.HiveRexUtil.ExprSimplifier;
 import org.apache.hadoop.hive.ql.optimizer.calcite.reloperators.HiveFilter;
 import org.apache.hadoop.hive.ql.optimizer.calcite.reloperators.HiveJoin;
 import org.apache.hadoop.hive.ql.optimizer.calcite.reloperators.HiveProject;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 import com.google.common.collect.ImmutableMap;
 import com.google.common.collect.Lists;
@@ -80,6 +84,9 @@ import com.google.common.collect.Lists;
  * </ul>
  */
 public abstract class HiveReduceExpressionsRule extends RelOptRule {
+
+  protected static final Logger LOG = LoggerFactory.getLogger(HiveReduceExpressionsRule.class);
+
   //~ Static fields/initializers ---------------------------------------------
 
   /**
@@ -125,17 +132,23 @@ public abstract class HiveReduceExpressionsRule extends RelOptRule {
 
     @Override public void onMatch(RelOptRuleCall call) {
       final Filter filter = call.rel(0);
-      final RexBuilder rexBuilder = filter.getCluster().getRexBuilder();
-
-      RexNode newConditionExp = HiveRexUtil.simplify(rexBuilder, filter.getCondition());
-      final List<RexNode> expList = Lists.newArrayList(newConditionExp);
-      boolean reduced = false;
+      final List<RexNode> expList =
+          Lists.newArrayList(filter.getCondition());
+      RexNode newConditionExp;
+      boolean reduced;
       final RelOptPredicateList predicates =
           RelMetadataQuery.instance().getPulledUpPredicates(filter.getInput());
-      if (reduceExpressions(filter, expList, predicates)) {
+      if (reduceExpressions(filter, expList, predicates, true)) {
         assert expList.size() == 1;
         newConditionExp = expList.get(0);
         reduced = true;
+      } else {
+        // No reduction, but let's still test the original
+        // predicate to see if it was already a constant,
+        // in which case we don't need any runtime decision
+        // about filtering.
+        newConditionExp = filter.getCondition();
+        reduced = false;
       }
 
       // Even if no reduction, let's still test the original
@@ -146,8 +159,7 @@ public abstract class HiveReduceExpressionsRule extends RelOptRule {
       if (newConditionExp.isAlwaysTrue()) {
         call.transformTo(
             filter.getInput());
-      } else if (reduced
-              || !newConditionExp.toString().equals(filter.getCondition().toString())) {
+      } else if (reduced) {
         call.transformTo(call.builder().
             push(filter.getInput()).filter(newConditionExp).build());
       } else {
@@ -169,26 +181,8 @@ public abstract class HiveReduceExpressionsRule extends RelOptRule {
       super(projectClass, relBuilderFactory, "HiveReduceExpressionsRule(Project)");
     }
 
-    public boolean matches(RelOptRuleCall call) {
-      Project project = call.rel(0);
-      HiveRulesRegistry registry = call.getPlanner().getContext().unwrap(HiveRulesRegistry.class);
-
-      // If this operator has been visited already by the rule,
-      // we do not need to apply the optimization
-      if (registry != null && registry.getVisited(this).contains(project)) {
-        return false;
-      }
-
-      return true;
-    }
-
     @Override public void onMatch(RelOptRuleCall call) {
       Project project = call.rel(0);
-      // Register that we have visited this operator in this rule
-      HiveRulesRegistry registry = call.getPlanner().getContext().unwrap(HiveRulesRegistry.class);
-      if (registry != null) {
-        registry.registerVisited(this, project);
-      }
       final RelOptPredicateList predicates =
           RelMetadataQuery.instance().getPulledUpPredicates(project.getInput());
       final List<RexNode> expList =
@@ -196,9 +190,6 @@ public abstract class HiveReduceExpressionsRule extends RelOptRule {
       if (reduceExpressions(project, expList, predicates)) {
         RelNode newProject = call.builder().push(project.getInput())
             .project(expList, project.getRowType().getFieldNames()).build();
-        if (registry != null) {
-          registry.registerVisited(this, newProject);
-        }
         call.transformTo(newProject);
 
         // New plan is absolutely better than old plan.
@@ -275,6 +266,43 @@ public abstract class HiveReduceExpressionsRule extends RelOptRule {
    */
   protected static boolean reduceExpressions(RelNode rel, List<RexNode> expList,
       RelOptPredicateList predicates) {
+    return reduceExpressions(rel, expList, predicates, false);
+  }
+
+  /**
+   * Reduces a list of expressions.
+   *
+   * @param rel     Relational expression
+   * @param expList List of expressions, modified in place
+   * @param predicates Constraints known to hold on input expressions
+   * @param unknownAsFalse Whether UNKNOWN will be treated as FALSE
+   *
+   * @return whether reduction found something to change, and succeeded
+   */
+  protected static boolean reduceExpressions(RelNode rel, List<RexNode> expList,
+      RelOptPredicateList predicates, boolean unknownAsFalse) {
+    RexBuilder rexBuilder = rel.getCluster().getRexBuilder();
+
+    boolean reduced = reduceExpressionsInternal(rel, expList, predicates);
+
+    // Simplify preds in place
+    ExprSimplifier simplifier = new ExprSimplifier(rexBuilder, unknownAsFalse);
+    List<RexNode> expList2 = Lists.newArrayList(expList);
+    simplifier.mutate(expList2);
+    boolean simplified = false;
+    for (int i = 0; i < expList.size(); i++) {
+      if (!expList2.get(i).toString().equals(expList.get(i).toString())) {
+        expList.remove(i);
+        expList.add(i, expList2.get(i));
+        simplified = true;
+      }
+    }
+
+    return reduced || simplified;
+  }
+
+  protected static boolean reduceExpressionsInternal(RelNode rel, List<RexNode> expList,
+      RelOptPredicateList predicates) {
     RexBuilder rexBuilder = rel.getCluster().getRexBuilder();
 
     // Replace predicates on CASE to CASE on predicates.
@@ -284,8 +312,8 @@ public abstract class HiveReduceExpressionsRule extends RelOptRule {
     final List<RexNode> constExps = Lists.newArrayList();
     List<Boolean> addCasts = Lists.newArrayList();
     final List<RexNode> removableCasts = Lists.newArrayList();
-    final ImmutableMap<RexNode, RexLiteral> constants =
-        predicateConstants(predicates);
+    final ImmutableMap<RexNode, RexNode> constants =
+        predicateConstants(RexNode.class, rexBuilder, predicates);
     findReducibleExps(rel.getCluster().getTypeFactory(), expList, constants,
         constExps, addCasts, removableCasts);
     if (constExps.isEmpty() && removableCasts.isEmpty()) {
@@ -347,6 +375,13 @@ public abstract class HiveReduceExpressionsRule extends RelOptRule {
     final List<RexNode> reducedValues = Lists.newArrayList();
     executor.reduce(rexBuilder, constExps2, reducedValues);
 
+    // Use RexNode.digest to judge whether each newly generated RexNode
+    // is equivalent to the original one.
+    if (Lists.transform(constExps, HiveCalciteUtil.REX_STR_FN).equals(
+            Lists.transform(reducedValues, HiveCalciteUtil.REX_STR_FN))) {
+      return false;
+    }
+
     // For Project, we have to be sure to preserve the result
     // types, so always cast regardless of the expression type.
     // For other RelNodes like Filter, in general, this isn't necessary,
@@ -384,7 +419,7 @@ public abstract class HiveReduceExpressionsRule extends RelOptRule {
    * @param removableCasts returns the list of cast expressions where the cast
    */
   protected static void findReducibleExps(RelDataTypeFactory typeFactory,
-      List<RexNode> exps, ImmutableMap<RexNode, RexLiteral> constants,
+      List<RexNode> exps, ImmutableMap<RexNode, RexNode> constants,
       List<RexNode> constExps, List<Boolean> addCasts,
       List<RexNode> removableCasts) {
     ReducibleExprLocator gardener =
@@ -437,18 +472,26 @@ public abstract class HiveReduceExpressionsRule extends RelOptRule {
   private static <C extends RexNode> void gatherConstraints(Class<C> clazz,
       RexNode predicate, Map<RexNode, C> map, Set<RexNode> excludeSet,
       RexBuilder rexBuilder) {
-    if (predicate.getKind() != SqlKind.EQUALS) {
+    if (predicate.getKind() != SqlKind.EQUALS
+            && predicate.getKind() != SqlKind.IS_NULL) {
       decompose(excludeSet, predicate);
       return;
     }
     final List<RexNode> operands = ((RexCall) predicate).getOperands();
-    if (operands.size() != 2) {
+    if (operands.size() != 2 && predicate.getKind() == SqlKind.EQUALS) {
       decompose(excludeSet, predicate);
       return;
     }
     // if it reaches here, we have rexNode equals rexNode
-    final RexNode left = operands.get(0);
-    final RexNode right = operands.get(1);
+    final RexNode left;
+    final RexNode right;
+    if (predicate.getKind() == SqlKind.EQUALS) {
+      left = operands.get(0);
+      right = operands.get(1);
+    } else {
+      left = operands.get(0);
+      right = rexBuilder.makeNullLiteral(left.getType().getSqlTypeName());
+    }
     // note that literals are immutable too and they can only be compared through
     // values.
     gatherConstraint(clazz, left, right, map, excludeSet, rexBuilder);
@@ -520,33 +563,6 @@ public abstract class HiveReduceExpressionsRule extends RelOptRule {
       }
     }
   }
-  
-  protected static ImmutableMap<RexNode, RexLiteral> predicateConstants(
-      RelOptPredicateList predicates) {
-    // We cannot use an ImmutableMap.Builder here. If there are multiple entries
-    // with the same key (e.g. "WHERE deptno = 1 AND deptno = 2"), it doesn't
-    // matter which we take, so the latter will replace the former.
-    // The basic idea is to find all the pairs of RexNode = RexLiteral
-    // (1) If 'predicates' contain a non-EQUALS, we bail out.
-    // (2) It is OK if a RexNode is equal to the same RexLiteral several times,
-    // (e.g. "WHERE deptno = 1 AND deptno = 1")
-    // (3) It will return false if there are inconsistent constraints (e.g.
-    // "WHERE deptno = 1 AND deptno = 2")
-    final Map<RexNode, RexLiteral> map = new HashMap<>();
-    final Set<RexNode> excludeSet = new HashSet<>();
-    for (RexNode predicate : predicates.pulledUpPredicates) {
-      gatherConstraints(map, predicate, excludeSet);
-    }
-    final ImmutableMap.Builder<RexNode, RexLiteral> builder =
-        ImmutableMap.builder();
-    for (Map.Entry<RexNode, RexLiteral> entry : map.entrySet()) {
-      RexNode rexNode = entry.getKey();
-      if (!overlap(rexNode, excludeSet)) {
-        builder.put(rexNode, entry.getValue());
-      }
-    }
-    return builder.build();
-  }
 
   private static boolean overlap(RexNode rexNode, Set<RexNode> set) {
     if (rexNode instanceof RexCall) {
@@ -573,46 +589,6 @@ public abstract class HiveReduceExpressionsRule extends RelOptRule {
     }
   }
 
-  private static void gatherConstraints(Map<RexNode, RexLiteral> map,
-      RexNode predicate, Set<RexNode> excludeSet) {
-    if (predicate.getKind() != SqlKind.EQUALS) {
-      decompose(excludeSet, predicate);
-      return;
-    }
-    final List<RexNode> operands = ((RexCall) predicate).getOperands();
-    if (operands.size() != 2) {
-      decompose(excludeSet, predicate);
-      return;
-    }
-    // if it reaches here, we have rexNode equals rexNode
-    final RexNode left = operands.get(0);
-    final RexNode right = operands.get(1);
-    // note that literals are immutable too and they can only be compared through
-    // values.
-    if (right instanceof RexLiteral && !excludeSet.contains(left)) {
-      RexLiteral existedValue = map.get(left);
-      if (existedValue == null) {
-        map.put(left, (RexLiteral) right);
-      } else {
-        if (!existedValue.getValue().equals(((RexLiteral) right).getValue())) {
-          // we found conflict values.
-          map.remove(left);
-          excludeSet.add(left);
-        }
-      }
-    } else if (left instanceof RexLiteral && !excludeSet.contains(right)) {
-      RexLiteral existedValue = map.get(right);
-      if (existedValue == null) {
-        map.put(right, (RexLiteral) left);
-      } else {
-        if (!existedValue.getValue().equals(((RexLiteral) left).getValue())) {
-          map.remove(right);
-          excludeSet.add(right);
-        }
-      }
-    }
-  }
-
   /** Pushes predicates into a CASE.
    *
    * <p>We have a loose definition of 'predicate': any boolean expression will
@@ -743,7 +719,7 @@ public abstract class HiveReduceExpressionsRule extends RelOptRule {
 
     private final List<Constancy> stack;
 
-    private final ImmutableMap<RexNode, RexLiteral> constants;
+    private final ImmutableMap<RexNode, RexNode> constants;
 
     private final List<RexNode> constExprs;
 
@@ -754,7 +730,7 @@ public abstract class HiveReduceExpressionsRule extends RelOptRule {
     private final List<SqlOperator> parentCallTypeStack;
 
     ReducibleExprLocator(RelDataTypeFactory typeFactory,
-        ImmutableMap<RexNode, RexLiteral> constants, List<RexNode> constExprs,
+        ImmutableMap<RexNode, RexNode> constants, List<RexNode> constExprs,
         List<Boolean> addCasts, List<RexNode> removableCasts) {
       // go deep
       super(true);

http://git-wip-us.apache.org/repos/asf/hive/blob/10423f51/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/rules/HiveSortLimitPullUpConstantsRule.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/rules/HiveSortLimitPullUpConstantsRule.java b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/rules/HiveSortLimitPullUpConstantsRule.java
index 3be9b0a..ebfabac 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/rules/HiveSortLimitPullUpConstantsRule.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/rules/HiveSortLimitPullUpConstantsRule.java
@@ -92,8 +92,15 @@ public class HiveSortLimitPullUpConstantsRule extends RelOptRule {
       return;
     }
 
-    Map<RexNode, RexNode> constants = HiveReduceExpressionsRule.predicateConstants(
+    Map<RexNode, RexNode> conditionsExtracted = HiveReduceExpressionsRule.predicateConstants(
             RexNode.class, rexBuilder, predicates);
+    Map<RexNode, RexNode> constants = new HashMap<>();
+    for (int i = 0; i < count ; i++) {
+      RexNode expr = rexBuilder.makeInputRef(sort.getInput(), i);
+      if (conditionsExtracted.containsKey(expr)) {
+        constants.put(expr, conditionsExtracted.get(expr));
+      }
+    }
 
     // None of the expressions are constant. Nothing to do.
     if (constants.isEmpty()) {
@@ -102,13 +109,10 @@ public class HiveSortLimitPullUpConstantsRule extends RelOptRule {
 
     if (count == constants.size()) {
       // At least a single item in project is required.
-      final Map<RexNode, RexNode> map = new HashMap<>(constants);
-      map.remove(map.keySet().iterator().next());
-      constants = map;
+      constants.remove(constants.keySet().iterator().next());
     }
 
     // Create expressions for Project operators before and after the Sort
-    boolean atLeastOneConstant = false;
     List<RelDataTypeField> fields = sort.getInput().getRowType().getFieldList();
     List<Pair<RexNode, String>> newChildExprs = new ArrayList<>();
     List<RexNode> topChildExprs = new ArrayList<>();
@@ -117,7 +121,6 @@ public class HiveSortLimitPullUpConstantsRule extends RelOptRule {
       RexNode expr = rexBuilder.makeInputRef(sort.getInput(), i);
       RelDataTypeField field = fields.get(i);
       if (constants.containsKey(expr)) {
-        atLeastOneConstant = true;
         topChildExprs.add(constants.get(expr));
         topChildExprsFields.add(field.getName());
       } else {
@@ -127,11 +130,6 @@ public class HiveSortLimitPullUpConstantsRule extends RelOptRule {
       }
     }
 
-    // No constants were found
-    if (!atLeastOneConstant) {
-      return;
-    }
-
     // Update field collations
     final Mappings.TargetMapping mapping =
             RelOptUtil.permutation(Pair.left(newChildExprs), sort.getInput().getRowType()).inverse();

http://git-wip-us.apache.org/repos/asf/hive/blob/10423f51/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/rules/HiveUnionPullUpConstantsRule.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/rules/HiveUnionPullUpConstantsRule.java b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/rules/HiveUnionPullUpConstantsRule.java
index 2552f87..f071ddd 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/rules/HiveUnionPullUpConstantsRule.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/rules/HiveUnionPullUpConstantsRule.java
@@ -80,32 +80,30 @@ public class HiveUnionPullUpConstantsRule extends RelOptRule {
       return;
     }
 
-    Map<RexNode, RexNode> constants = HiveReduceExpressionsRule.predicateConstants(
+    Map<RexNode, RexNode> conditionsExtracted = HiveReduceExpressionsRule.predicateConstants(
             RexNode.class, rexBuilder, predicates);
+    Map<RexNode, RexNode> constants = new HashMap<>();
+    for (int i = 0; i < count ; i++) {
+      RexNode expr = rexBuilder.makeInputRef(union, i);
+      if (conditionsExtracted.containsKey(expr)) {
+        constants.put(expr, conditionsExtracted.get(expr));
+      }
+    }
 
     // None of the expressions are constant. Nothing to do.
     if (constants.isEmpty()) {
       return;
     }
 
-    if (count == constants.size()) {
-      // At least a single item in project is required.
-      final Map<RexNode, RexNode> map = new HashMap<>(constants);
-      map.remove(map.keySet().iterator().next());
-      constants = map;
-    }
-
     // Create expressions for Project operators before and after the Union
-    boolean atLeastOneConstant = false;
-    List<RelDataTypeField> fields = union.getRowType().getFieldList();
     List<Pair<RexNode, String>> newChildExprs = new ArrayList<>();
+    List<RelDataTypeField> fields = union.getRowType().getFieldList();
     List<RexNode> topChildExprs = new ArrayList<>();
     List<String> topChildExprsFields = new ArrayList<>();
     for (int i = 0; i < count ; i++) {
       RexNode expr = rexBuilder.makeInputRef(union, i);
       RelDataTypeField field = fields.get(i);
       if (constants.containsKey(expr)) {
-        atLeastOneConstant = true;
         topChildExprs.add(constants.get(expr));
         topChildExprsFields.add(field.getName());
       } else {
@@ -115,9 +113,10 @@ public class HiveUnionPullUpConstantsRule extends RelOptRule {
       }
     }
 
-    // No constants were found
-    if (!atLeastOneConstant) {
-      return;
+    if (newChildExprs.isEmpty()) {
+      // At least a single item in project is required.
+      newChildExprs.add(Pair.<RexNode,String>of(
+              topChildExprs.get(0), topChildExprsFields.get(0)));
     }
 
     // Update top Project positions