You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@impala.apache.org by ta...@apache.org on 2017/01/24 21:38:34 UTC

[2/3] incubator-impala git commit: IMPALA-1861: Simplify conditionals with constant conditions

IMPALA-1861: Simplify conditionals with constant conditions

When there are conditionals with constant values of TRUE or
FALSE we can simplify them during analysis using the ExprRewriter.

This patch introduces the SimplifyConditionalsRule with covers IF,
OR, AND, CASE, and DECODE.

It also introduces NormalizeExprsRule which normalizes AND and OR
such that if either child is a BoolLiteral, then the left child is a
BoolLiteral.

Testing:
- Added unit tests to ExprRewriteRulesTest.
- Added functional tests to expr.test
- Ran FE planner tests and BE expr-test.

Change-Id: Id70aaf9fd99f64bd98175b7e2dbba28f350e7d3b
Reviewed-on: http://gerrit.cloudera.org:8080/5585
Reviewed-by: Jim Apple <jb...@apache.org>
Tested-by: Impala Public Jenkins


Project: http://git-wip-us.apache.org/repos/asf/incubator-impala/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-impala/commit/4b486b0f
Tree: http://git-wip-us.apache.org/repos/asf/incubator-impala/tree/4b486b0f
Diff: http://git-wip-us.apache.org/repos/asf/incubator-impala/diff/4b486b0f

Branch: refs/heads/master
Commit: 4b486b0f90936ab5bc32daf0859bc74b340dd179
Parents: a2041dd
Author: Thomas Tauber-Marshall <tm...@cloudera.com>
Authored: Wed Dec 28 15:15:10 2016 -0800
Committer: Impala Public Jenkins <im...@gerrit.cloudera.org>
Committed: Tue Jan 24 03:22:08 2017 +0000

----------------------------------------------------------------------
 .../apache/impala/analysis/AnalysisContext.java |   5 +
 .../org/apache/impala/analysis/CaseExpr.java    |   3 +-
 .../apache/impala/analysis/CaseWhenClause.java  |  14 +-
 .../impala/rewrite/NormalizeExprsRule.java      |  56 ++++++
 .../rewrite/SimplifyConditionalsRule.java       | 201 +++++++++++++++++++
 .../impala/analysis/ExprRewriteRulesTest.java   | 113 ++++++++++-
 .../queries/QueryTest/exprs.test                |  27 +++
 7 files changed, 406 insertions(+), 13 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/4b486b0f/fe/src/main/java/org/apache/impala/analysis/AnalysisContext.java
----------------------------------------------------------------------
diff --git a/fe/src/main/java/org/apache/impala/analysis/AnalysisContext.java b/fe/src/main/java/org/apache/impala/analysis/AnalysisContext.java
index c74acc5..8cbe268 100644
--- a/fe/src/main/java/org/apache/impala/analysis/AnalysisContext.java
+++ b/fe/src/main/java/org/apache/impala/analysis/AnalysisContext.java
@@ -40,6 +40,8 @@ import org.apache.impala.rewrite.ExprRewriteRule;
 import org.apache.impala.rewrite.ExprRewriter;
 import org.apache.impala.rewrite.ExtractCommonConjunctRule;
 import org.apache.impala.rewrite.FoldConstantsRule;
+import org.apache.impala.rewrite.NormalizeExprsRule;
+import org.apache.impala.rewrite.SimplifyConditionalsRule;
 import org.apache.impala.thrift.TAccessEvent;
 import org.apache.impala.thrift.TLineageGraph;
 import org.apache.impala.thrift.TQueryCtx;
@@ -79,7 +81,10 @@ public class AnalysisContext {
     List<ExprRewriteRule> rules = Lists.newArrayList(BetweenToCompoundRule.INSTANCE);
     if (queryCtx.getClient_request().getQuery_options().enable_expr_rewrites) {
       rules.add(FoldConstantsRule.INSTANCE);
+      rules.add(NormalizeExprsRule.INSTANCE);
       rules.add(ExtractCommonConjunctRule.INSTANCE);
+      // Relies on FoldConstantsRule and NormalizeExprsRule.
+      rules.add(SimplifyConditionalsRule.INSTANCE);
     }
     rewriter_ = new ExprRewriter(rules);
   }

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/4b486b0f/fe/src/main/java/org/apache/impala/analysis/CaseExpr.java
----------------------------------------------------------------------
diff --git a/fe/src/main/java/org/apache/impala/analysis/CaseExpr.java b/fe/src/main/java/org/apache/impala/analysis/CaseExpr.java
index 3d96662..e30f705 100644
--- a/fe/src/main/java/org/apache/impala/analysis/CaseExpr.java
+++ b/fe/src/main/java/org/apache/impala/analysis/CaseExpr.java
@@ -35,7 +35,8 @@ import com.google.common.collect.Lists;
 
 /**
  * CASE and DECODE are represented using this class. The backend implementation is
- * always the "case" function.
+ * always the "case" function. CASE always returns the THEN corresponding to the leftmost
+ * WHEN that is TRUE, or the ELSE (or NULL if no ELSE is provided) if no WHEN is TRUE.
  *
  * The internal representation of
  *   CASE [expr] WHEN expr THEN expr [WHEN expr THEN expr ...] [ELSE expr] END

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/4b486b0f/fe/src/main/java/org/apache/impala/analysis/CaseWhenClause.java
----------------------------------------------------------------------
diff --git a/fe/src/main/java/org/apache/impala/analysis/CaseWhenClause.java b/fe/src/main/java/org/apache/impala/analysis/CaseWhenClause.java
index e8dd948..54dd012 100644
--- a/fe/src/main/java/org/apache/impala/analysis/CaseWhenClause.java
+++ b/fe/src/main/java/org/apache/impala/analysis/CaseWhenClause.java
@@ -22,21 +22,17 @@ package org.apache.impala.analysis;
  * captures info of a single WHEN expr THEN expr clause.
  *
  */
-class CaseWhenClause {
+public class CaseWhenClause {
   private final Expr whenExpr_;
   private final Expr thenExpr_;
 
   public CaseWhenClause(Expr whenExpr, Expr thenExpr) {
     super();
-    this.whenExpr_ = whenExpr;
-    this.thenExpr_ = thenExpr;
+    whenExpr_ = whenExpr;
+    thenExpr_ = thenExpr;
   }
 
-  public Expr getWhenExpr() {
-    return whenExpr_;
-  }
+  public Expr getWhenExpr() { return whenExpr_; }
 
-  public Expr getThenExpr() {
-    return thenExpr_;
-  }
+  public Expr getThenExpr() { return thenExpr_; }
 }

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/4b486b0f/fe/src/main/java/org/apache/impala/rewrite/NormalizeExprsRule.java
----------------------------------------------------------------------
diff --git a/fe/src/main/java/org/apache/impala/rewrite/NormalizeExprsRule.java b/fe/src/main/java/org/apache/impala/rewrite/NormalizeExprsRule.java
new file mode 100644
index 0000000..6ff9ba8
--- /dev/null
+++ b/fe/src/main/java/org/apache/impala/rewrite/NormalizeExprsRule.java
@@ -0,0 +1,56 @@
+// 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.impala.rewrite;
+
+import org.apache.impala.analysis.Analyzer;
+import org.apache.impala.analysis.BoolLiteral;
+import org.apache.impala.analysis.CompoundPredicate;
+import org.apache.impala.analysis.Expr;
+import org.apache.impala.common.AnalysisException;
+
+/**
+ * Normalizes CompoundPredicates by ensuring that if either child of AND or OR is a
+ * BoolLiteral, then the left (i.e. first) child is a BoolLiteral.
+ *
+ * Examples:
+ * id = 0 && true -> true && id = 0
+ */
+public class NormalizeExprsRule implements ExprRewriteRule {
+  public static ExprRewriteRule INSTANCE = new NormalizeExprsRule();
+
+  @Override
+  public Expr apply(Expr expr, Analyzer analyzer) throws AnalysisException {
+    if (!expr.isAnalyzed()) return expr;
+
+    // TODO: add normalization for other expr types.
+    if (expr instanceof CompoundPredicate) {
+      return normalizeCompoundPredicate((CompoundPredicate) expr);
+    }
+    return expr;
+  }
+
+  private Expr normalizeCompoundPredicate(CompoundPredicate expr) {
+    if (expr.getOp() == CompoundPredicate.Operator.NOT) return expr;
+
+    if (!(expr.getChild(0) instanceof BoolLiteral)
+        && expr.getChild(1) instanceof BoolLiteral) {
+      return new CompoundPredicate(expr.getOp(), expr.getChild(1), expr.getChild(0));
+    }
+    return expr;
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/4b486b0f/fe/src/main/java/org/apache/impala/rewrite/SimplifyConditionalsRule.java
----------------------------------------------------------------------
diff --git a/fe/src/main/java/org/apache/impala/rewrite/SimplifyConditionalsRule.java b/fe/src/main/java/org/apache/impala/rewrite/SimplifyConditionalsRule.java
new file mode 100644
index 0000000..8c1de39
--- /dev/null
+++ b/fe/src/main/java/org/apache/impala/rewrite/SimplifyConditionalsRule.java
@@ -0,0 +1,201 @@
+// 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.impala.rewrite;
+
+import java.util.ArrayList;
+import java.util.List;
+
+import org.apache.impala.analysis.Analyzer;
+import org.apache.impala.analysis.BinaryPredicate;
+import org.apache.impala.analysis.BoolLiteral;
+import org.apache.impala.analysis.CaseExpr;
+import org.apache.impala.analysis.CaseWhenClause;
+import org.apache.impala.analysis.CompoundPredicate;
+import org.apache.impala.analysis.Expr;
+import org.apache.impala.analysis.FunctionCallExpr;
+import org.apache.impala.analysis.FunctionName;
+import org.apache.impala.analysis.NullLiteral;
+import org.apache.impala.common.AnalysisException;
+import com.google.common.base.Preconditions;
+
+/***
+ * This rule simplifies conditional functions with constant conditions. It relies on
+ * FoldConstantsRule to replace the constant conditions with a BoolLiteral or NullLiteral
+ * first, and on NormalizeExprsRule to normalize CompoundPredicates.
+ *
+ * Examples:
+ * if (true, 0, 1) -> 0
+ * id = 0 OR false -> id = 0
+ * false AND id = 1 -> false
+ * case when false then 0 when true then 1 end -> 1
+ */
+public class SimplifyConditionalsRule implements ExprRewriteRule {
+  public static ExprRewriteRule INSTANCE = new SimplifyConditionalsRule();
+
+  @Override
+  public Expr apply(Expr expr, Analyzer analyzer) throws AnalysisException {
+    if (!expr.isAnalyzed()) return expr;
+
+    if (expr instanceof FunctionCallExpr) {
+      return simplifyFunctionCallExpr((FunctionCallExpr) expr);
+    } else if (expr instanceof CompoundPredicate) {
+      return simplifyCompoundPredicate((CompoundPredicate) expr);
+    } else if (expr instanceof CaseExpr) {
+      return simplifyCaseExpr((CaseExpr) expr, analyzer);
+    }
+    return expr;
+  }
+
+  /**
+   * Simplifies IF by returning the corresponding child if the condition has a constant
+   * TRUE, FALSE, or NULL (equivalent to FALSE) value.
+   */
+  private Expr simplifyFunctionCallExpr(FunctionCallExpr expr) {
+    FunctionName fnName = expr.getFnName();
+
+    // TODO: Add the other conditional functions, eg. ifnull, istrue, etc.
+    if (fnName.getFunction().equals("if")) {
+      Preconditions.checkState(expr.getChildren().size() == 3);
+      if (expr.getChild(0) instanceof BoolLiteral) {
+        if (((BoolLiteral) expr.getChild(0)).getValue()) {
+          // IF(TRUE)
+          return expr.getChild(1);
+        } else {
+          // IF(FALSE)
+          return expr.getChild(2);
+        }
+      } else if (expr.getChild(0) instanceof NullLiteral) {
+        // IF(NULL)
+        return expr.getChild(2);
+      }
+    }
+    return expr;
+  }
+
+  /**
+   * Simplifies compound predicates with at least one BoolLiteral child, which
+   * NormalizeExprsRule ensures will be the left child,  according to the following rules:
+   * true AND 'expr' -> 'expr'
+   * false AND 'expr' -> false
+   * true OR 'expr' -> true
+   * false OR 'expr' -> 'expr'
+   *
+   * Unlike other rules here such as IF, we cannot in general simplify CompoundPredicates
+   * with a NullLiteral child (unless the other child is a BoolLiteral), eg. null and
+   * 'expr' is false if 'expr' is false but null if 'expr' is true.
+   *
+   * NOT is covered by FoldConstantRule.
+   */
+  private Expr simplifyCompoundPredicate(CompoundPredicate expr) {
+    Expr leftChild = expr.getChild(0);
+    if (!(leftChild instanceof BoolLiteral)) return expr;
+
+    if (expr.getOp() == CompoundPredicate.Operator.AND) {
+      if (((BoolLiteral) leftChild).getValue()) {
+        // TRUE AND 'expr', so return 'expr'.
+        return expr.getChild(1);
+      } else {
+        // FALSE AND 'expr', so return FALSE.
+        return leftChild;
+      }
+    } else if (expr.getOp() == CompoundPredicate.Operator.OR) {
+      if (((BoolLiteral) leftChild).getValue()) {
+        // TRUE OR 'expr', so return TRUE.
+        return leftChild;
+      } else {
+        // FALSE OR 'expr', so return 'expr'.
+        return expr.getChild(1);
+      }
+    }
+    return expr;
+  }
+
+  /**
+   * Simpilfies CASE and DECODE. If any of the 'when's have constant FALSE/NULL values,
+   * they are removed. If all of the 'when's are removed, just the ELSE is returned. If
+   * any of the 'when's have constant TRUE values, the leftmost one becomes the ELSE
+   * clause and all following cases are removed.
+   */
+  private Expr simplifyCaseExpr(CaseExpr expr, Analyzer analyzer)
+      throws AnalysisException {
+    Expr caseExpr = expr.hasCaseExpr() ? expr.getChild(0) : null;
+    if (expr.hasCaseExpr() && !caseExpr.isLiteral()) return expr;
+
+    int numChildren = expr.getChildren().size();
+    int loopStart = expr.hasCaseExpr() ? 1 : 0;
+    // Check and return early if there's nothing that can be simplified.
+    boolean canSimplify = false;
+    for (int i = loopStart; i < numChildren - 1; i += 2) {
+      if (expr.getChild(i).isLiteral()) {
+        canSimplify = true;
+        break;
+      }
+    }
+    if (!canSimplify) return expr;
+
+    // Contains all 'when' clauses with non-constant conditions, used to construct the new
+    // CASE expr while removing any FALSE or NULL cases.
+    List<CaseWhenClause> newWhenClauses = new ArrayList<CaseWhenClause>();
+    // Set to THEN of first constant TRUE clause, if any.
+    Expr elseExpr = null;
+    for (int i = loopStart; i < numChildren - 1; i += 2) {
+      Expr child = expr.getChild(i);
+      if (child instanceof NullLiteral) continue;
+
+      Expr whenExpr;
+      if (expr.hasCaseExpr()) {
+        if (child.isLiteral()) {
+          BinaryPredicate pred = new BinaryPredicate(
+              BinaryPredicate.Operator.EQ, caseExpr, expr.getChild(i));
+          pred.analyze(analyzer);
+          whenExpr = analyzer.getConstantFolder().rewrite(pred, analyzer);
+        } else {
+          whenExpr = null;
+        }
+      } else {
+        whenExpr = child;
+      }
+
+      if (whenExpr instanceof BoolLiteral) {
+        if (((BoolLiteral) whenExpr).getValue()) {
+          if (newWhenClauses.size() == 0) {
+            // This WHEN is always TRUE, and any cases preceding it are constant
+            // FALSE/NULL, so just return its THEN.
+            return expr.getChild(i + 1).castTo(expr.getType());
+          } else {
+            // This WHEN is always TRUE, so the cases after it can never be reached.
+            elseExpr = expr.getChild(i + 1);
+            break;
+          }
+        } else {
+          // This WHEN is always FALSE, so it can be removed.
+        }
+      } else {
+        newWhenClauses.add(new CaseWhenClause(child, expr.getChild(i + 1)));
+      }
+    }
+
+    if (expr.hasElseExpr() && elseExpr == null) elseExpr = expr.getChild(numChildren - 1);
+    if (newWhenClauses.size() == 0) {
+      // All of the WHEN clauses were FALSE, return the ELSE.
+      if (elseExpr == null) return NullLiteral.create(expr.getType());
+      return elseExpr;
+    }
+    return new CaseExpr(caseExpr, newWhenClauses, elseExpr);
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/4b486b0f/fe/src/test/java/org/apache/impala/analysis/ExprRewriteRulesTest.java
----------------------------------------------------------------------
diff --git a/fe/src/test/java/org/apache/impala/analysis/ExprRewriteRulesTest.java b/fe/src/test/java/org/apache/impala/analysis/ExprRewriteRulesTest.java
index f31d50d..3abdf67 100644
--- a/fe/src/test/java/org/apache/impala/analysis/ExprRewriteRulesTest.java
+++ b/fe/src/test/java/org/apache/impala/analysis/ExprRewriteRulesTest.java
@@ -23,10 +23,12 @@ import org.apache.impala.catalog.Catalog;
 import org.apache.impala.common.AnalysisException;
 import org.apache.impala.common.FrontendTestBase;
 import org.apache.impala.rewrite.BetweenToCompoundRule;
+import org.apache.impala.rewrite.SimplifyConditionalsRule;
 import org.apache.impala.rewrite.ExprRewriteRule;
 import org.apache.impala.rewrite.ExprRewriter;
 import org.apache.impala.rewrite.ExtractCommonConjunctRule;
 import org.apache.impala.rewrite.FoldConstantsRule;
+import org.apache.impala.rewrite.NormalizeExprsRule;
 import org.junit.Assert;
 import org.junit.Test;
 
@@ -39,15 +41,17 @@ public class ExprRewriteRulesTest extends FrontendTestBase {
 
   public Expr RewritesOk(String expr, ExprRewriteRule rule, String expectedExpr)
       throws AnalysisException {
+    return RewritesOk(expr, Lists.newArrayList(rule), expectedExpr);
+  }
+
+  public Expr RewritesOk(String expr, List<ExprRewriteRule> rules, String expectedExpr)
+      throws AnalysisException {
     String stmtStr = "select " + expr + " from functional.alltypessmall";
     SelectStmt stmt = (SelectStmt) ParsesOk(stmtStr);
     Analyzer analyzer = createAnalyzer(Catalog.DEFAULT_DB);
     stmt.analyze(analyzer);
     Expr origExpr = stmt.getResultExprs().get(0);
     String origSql = origExpr.toSql();
-    // Create a rewriter with only a single rule.
-    List<ExprRewriteRule> rules = Lists.newArrayList();
-    rules.add(rule);
     ExprRewriter rewriter = new ExprRewriter(rules);
     Expr rewrittenExpr = rewriter.rewrite(origExpr, analyzer);
     String rewrittenSql = rewrittenExpr.toSql();
@@ -230,4 +234,107 @@ public class ExprRewriteRulesTest extends FrontendTestBase {
     // Tests that exprs that warn during their evaluation are not folded.
     RewritesOk("coalesce(1.8, cast(int_col as decimal(38,38)))", rule, null);
   }
+
+  @Test
+  public void TestSimplifyConditionalsRule() throws AnalysisException {
+    ExprRewriteRule rule = SimplifyConditionalsRule.INSTANCE;
+
+    // IF
+    RewritesOk("if(true, id, id+1)", rule, "id");
+    RewritesOk("if(false, id, id+1)", rule, "id + 1");
+    RewritesOk("if(null, id, id+1)", rule, "id + 1");
+    RewritesOk("if(id = 0, true, false)", rule, null);
+
+    // CompoundPredicate
+    RewritesOk("false || id = 0", rule, "id = 0");
+    RewritesOk("true || id = 0", rule, "TRUE");
+    RewritesOk("false && id = 0", rule, "FALSE");
+    RewritesOk("true && id = 0", rule, "id = 0");
+    // NULL with a non-constant other child doesn't get rewritten.
+    RewritesOk("null && id = 0", rule, null);
+    RewritesOk("null || id = 0", rule, null);
+
+    List<ExprRewriteRule> rules = Lists.newArrayList();
+    rules.add(FoldConstantsRule.INSTANCE);
+    rules.add(rule);
+    // CASE with caseExpr
+    // Single TRUE case with no preceding non-constant cases.
+    RewritesOk("case 1 when 0 then id when 1 then id + 1 when 2 then id + 2 end", rules,
+        "id + 1");
+    // SINGLE TRUE case with preceding non-constant case.
+    RewritesOk("case 1 when id then id when 1 then id + 1 end", rules,
+        "CASE 1 WHEN id THEN id ELSE id + 1 END");
+    // Single FALSE case.
+    RewritesOk("case 0 when 1 then 1 when id then id + 1 end", rules,
+        "CASE 0 WHEN id THEN id + 1 END");
+    // All FALSE, return ELSE.
+    RewritesOk("case 2 when 0 then id when 1 then id * 2 else 0 end", rules, "0");
+    // All FALSE, return implicit NULL ELSE.
+    RewritesOk("case 3 when 0 then id when 1 then id + 1 end", rules, "NULL");
+    // Multiple TRUE, first one becomes ELSE.
+    RewritesOk("case 1 when id then id when 2 - 1 then id + 1 when 1 then id + 2 end",
+        rules, "CASE 1 WHEN id THEN id ELSE id + 1 END");
+    // When NULL.
+    RewritesOk("case 0 when null then 0 else 1 end", rules, "1");
+    // All non-constant, don't rewrite.
+    RewritesOk("case id when 1 then 1 when 2 then 2 else 3 end", rules, null);
+
+    // CASE without caseExpr
+    // Single TRUE case with no predecing non-constant case.
+    RewritesOk("case when FALSE then 0 when TRUE then 1 end", rules, "1");
+    // Single TRUE case with preceding non-constant case.
+    RewritesOk("case when id = 0 then 0 when true then 1 when id = 2 then 2 end", rules,
+        "CASE WHEN id = 0 THEN 0 ELSE 1 END");
+    // Single FALSE case.
+    RewritesOk("case when id = 0 then 0 when false then 1 when id = 2 then 2 end", rules,
+        "CASE WHEN id = 0 THEN 0 WHEN id = 2 THEN 2 END");
+    // All FALSE, return ELSE.
+    RewritesOk(
+        "case when false then 1 when false then 2 else id + 1 end", rules, "id + 1");
+    // All FALSE, return implicit NULL ELSE.
+    RewritesOk("case when false then 0 end", rules, "NULL");
+    // Multiple TRUE, first one becomes ELSE.
+    RewritesOk("case when id = 1 then 0 when 2 = 1 + 1 then 1 when true then 2 end",
+        rules, "CASE WHEN id = 1 THEN 0 ELSE 1 END");
+    // When NULL.
+    RewritesOk("case when id = 0 then 0 when null then 1 else 2 end", rules,
+        "CASE WHEN id = 0 THEN 0 ELSE 2 END");
+    // All non-constant, don't rewrite.
+    RewritesOk("case when id = 0 then 0 when id = 1 then 1 end", rules, null);
+
+    // DECODE
+    // SIngle TRUE case with no preceding non-constant case.
+    RewritesOk("decode(1, 0, id, 1, id + 1, 2, id + 2)", rules, "id + 1");
+    // Single TRUE case with predecing non-constant case.
+    RewritesOk("decode(1, id, id, 1, id + 1, 0)", rules,
+        "CASE WHEN 1 = id THEN id ELSE id + 1 END");
+    // Single FALSE case.
+    RewritesOk("decode(1, 0, id, tinyint_col, id + 1)", rules,
+        "CASE WHEN 1 = tinyint_col THEN id + 1 END");
+    // All FALSE, return ELSE.
+    RewritesOk("decode(1, 0, 0, 2, 2, 3)", rules, "3");
+    // All FALSE, return implicit NULL ELSE.
+    RewritesOk("decode(1, 1 + 1, 2, 1 + 2, 3)", rules, "NULL");
+    // Multiple TRUE, first one becomes ELSE.
+    RewritesOk("decode(1, id, id, 1 + 1, 0, 1 * 1, 1, 2 - 1, 2)", rules,
+        "CASE WHEN 1 = id THEN id ELSE 1 END");
+    // When NULL - DECODE allows the decodeExpr to equal NULL (see CaseExpr.java), so the
+    // NULL case is not treated as a constant FALSE and removed.
+    RewritesOk("decode(id, null, 0, 1)", rules, null);
+    // All non-constant, don't rewrite.
+    RewritesOk("decode(id, 1, 1, 2, 2)", rules, null);
+  }
+
+  @Test
+  public void TestNormalizeExprsRule() throws AnalysisException {
+    ExprRewriteRule rule = NormalizeExprsRule.INSTANCE;
+
+    // CompoundPredicate
+    RewritesOk("id = 0 OR false", rule, "FALSE OR id = 0");
+    RewritesOk("null AND true", rule, "TRUE AND NULL");
+    // The following already have a BoolLiteral left child and don't get rewritten.
+    RewritesOk("true and id = 0", rule, null);
+    RewritesOk("false or id = 1", rule, null);
+    RewritesOk("false or true", rule, null);
+  }
 }

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/4b486b0f/testdata/workloads/functional-query/queries/QueryTest/exprs.test
----------------------------------------------------------------------
diff --git a/testdata/workloads/functional-query/queries/QueryTest/exprs.test b/testdata/workloads/functional-query/queries/QueryTest/exprs.test
index 9e5d3dc..3b2383e 100644
--- a/testdata/workloads/functional-query/queries/QueryTest/exprs.test
+++ b/testdata/workloads/functional-query/queries/QueryTest/exprs.test
@@ -2665,3 +2665,30 @@ select decode(0, 1, 0, id, 1, 2) a from functional.alltypestiny order by a
 ---- TYPES
 TINYINT
 ====
+---- QUERY
+# Test expr rewrite behavior of IF with constant conditions.
+select if(true, id * 10, 0) from functional.alltypestiny where id = if(false, 0, 1)
+---- RESULTS
+10
+---- TYPES
+BIGINT
+====
+---- QUERY
+# Test expr rewrite behavior of CompoundPredicate with constant conditions.
+select id, null and id = 0 from functional.alltypestiny where id < 2 or false
+---- RESULTS
+0,NULL
+1,false
+---- TYPES
+INT,BOOLEAN
+====
+---- QUERY
+# Test expr rewrite behavior of CASE with constant conditions.
+select id, case id when null then null when 1 then 10 when true then 20 end
+from functional.alltypestiny
+where id = case when false then 0 when 1 = 1 then 1 else 2 end
+---- RESULTS
+1,10
+---- TYPES
+INT,TINYINT
+====