You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hive.apache.org by xu...@apache.org on 2015/05/29 06:47:42 UTC

[63/84] [abbrv] hive git commit: HIVE-10716 : Fold case/when udf for expression involving nulls in filter operator (Ashutosh Chauhan via Gopal V)

HIVE-10716 : Fold case/when udf for expression involving nulls in filter operator (Ashutosh Chauhan via Gopal V)

Signed-off-by: Ashutosh Chauhan <ha...@apache.org>


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

Branch: refs/heads/spark
Commit: b615de25bc2282b158ed7acc67fa0faaa061ff0b
Parents: 9d9fa7b
Author: Ashutosh Chauhan <ha...@apache.org>
Authored: Thu May 14 14:06:18 2015 -0700
Committer: Ashutosh Chauhan <ha...@apache.org>
Committed: Tue May 26 22:48:28 2015 -0700

----------------------------------------------------------------------
 .../optimizer/ConstantPropagateProcFactory.java |  33 +++-
 ql/src/test/queries/clientpositive/fold_case.q  |   8 +
 .../test/results/clientpositive/fold_case.q.out | 184 ++++++++++++++++++-
 .../clientpositive/fold_eq_with_case_when.q.out |   2 +-
 .../test/results/clientpositive/fold_when.q.out |   6 +-
 5 files changed, 221 insertions(+), 12 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hive/blob/b615de25/ql/src/java/org/apache/hadoop/hive/ql/optimizer/ConstantPropagateProcFactory.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/ConstantPropagateProcFactory.java b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/ConstantPropagateProcFactory.java
index 209f717..e66de1a 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/ConstantPropagateProcFactory.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/ConstantPropagateProcFactory.java
@@ -58,6 +58,7 @@ import org.apache.hadoop.hive.ql.plan.FileSinkDesc;
 import org.apache.hadoop.hive.ql.plan.GroupByDesc;
 import org.apache.hadoop.hive.ql.plan.JoinCondDesc;
 import org.apache.hadoop.hive.ql.plan.JoinDesc;
+import org.apache.hadoop.hive.ql.plan.OperatorDesc;
 import org.apache.hadoop.hive.ql.plan.ReduceSinkDesc;
 import org.apache.hadoop.hive.ql.plan.TableScanDesc;
 import org.apache.hadoop.hive.ql.udf.UDFType;
@@ -233,7 +234,7 @@ public final class ConstantPropagateProcFactory {
       } else {
 
         // Check if the function can be short cut.
-        ExprNodeDesc shortcut = shortcutFunction(udf, newExprs);
+        ExprNodeDesc shortcut = shortcutFunction(udf, newExprs, op);
         if (shortcut != null) {
           LOG.debug("Folding expression:" + desc + " -> " + shortcut);
           return shortcut;
@@ -362,7 +363,8 @@ public final class ConstantPropagateProcFactory {
     return (expr instanceof ExprNodeColumnDesc) ? (ExprNodeColumnDesc)expr : null;
   }
 
-  private static ExprNodeDesc shortcutFunction(GenericUDF udf, List<ExprNodeDesc> newExprs) throws UDFArgumentException {
+  private static ExprNodeDesc shortcutFunction(GenericUDF udf, List<ExprNodeDesc> newExprs,
+    Operator<? extends Serializable> op) throws UDFArgumentException {
 
     if (udf instanceof GenericUDFOPEqual) {
      assert newExprs.size() == 2;
@@ -486,7 +488,20 @@ public final class ConstantPropagateProcFactory {
         Object thenVal = constThen.getValue();
         Object elseVal = constElse.getValue();
         if (thenVal == null) {
-          return elseVal == null ? thenExpr : null;
+          if (elseVal == null) {
+            // both branches are null.
+            return thenExpr;
+          } else if (op instanceof FilterOperator) {
+            // we can still fold, since here null is equivalent to false.
+            return Boolean.TRUE.equals(elseVal) ?
+              ExprNodeGenericFuncDesc.newInstance(new GenericUDFOPNot(), newExprs.subList(0, 1)) : Boolean.FALSE.equals(elseVal) ?
+              elseExpr : null;
+          } else {
+            // can't do much, expression is not in context of filter, so we can't treat null as equivalent to false here.
+            return null;
+          }
+        } else if (elseVal == null && op instanceof FilterOperator) {
+          return Boolean.TRUE.equals(thenVal) ? whenExpr : Boolean.FALSE.equals(thenVal) ? thenExpr : null;
         } else if(thenVal.equals(elseVal)){
           return thenExpr;
         } else if (thenVal instanceof Boolean && elseVal instanceof Boolean) {
@@ -516,7 +531,17 @@ public final class ConstantPropagateProcFactory {
         Object thenVal = constThen.getValue();
         Object elseVal = constElse.getValue();
         if (thenVal == null) {
-          return elseVal == null ? thenExpr : null;
+          if (null == elseVal) {
+            return thenExpr;
+          } else if (op instanceof FilterOperator) {
+            return Boolean.TRUE.equals(elseVal) ? ExprNodeGenericFuncDesc.newInstance(new GenericUDFOPNotEqual(), newExprs.subList(0, 2)) :
+              Boolean.FALSE.equals(elseVal) ? elseExpr : null;
+          } else {
+            return null;
+          }
+        } else if (null == elseVal && op instanceof FilterOperator) {
+            return Boolean.TRUE.equals(thenVal) ? ExprNodeGenericFuncDesc.newInstance(new GenericUDFOPEqual(), newExprs.subList(0, 2)) :
+              Boolean.FALSE.equals(thenVal) ? thenExpr : null;
         } else if(thenVal.equals(elseVal)){
           return thenExpr;
         } else if (thenVal instanceof Boolean && elseVal instanceof Boolean) {

http://git-wip-us.apache.org/repos/asf/hive/blob/b615de25/ql/src/test/queries/clientpositive/fold_case.q
----------------------------------------------------------------------
diff --git a/ql/src/test/queries/clientpositive/fold_case.q b/ql/src/test/queries/clientpositive/fold_case.q
index 3f9e3a3..4732aa5 100644
--- a/ql/src/test/queries/clientpositive/fold_case.q
+++ b/ql/src/test/queries/clientpositive/fold_case.q
@@ -9,4 +9,12 @@ select count(1) from src where (case key when '238' then true else 1=1 end);
 explain
 select count(1) from src where (case key when '238' then 1=1 else 1=null end);
 explain 
+select count(1) from src where (case key when '238' then 1=null  end);
+explain 
+select count(1) from src where (case key when '238' then 2 = cast('2' as bigint) end);
+explain 
+select (case key when '238' then null else false end) from src where (case key when '238' then 2 = cast('1' as bigint)  else true end);
+explain 
+select (case key when '238' then null else null end) from src where (case key when '238' then 2 = null else 3 = null  end);
+explain 
 select count(1) from src where (case key when '238' then null else 1=1 end);

http://git-wip-us.apache.org/repos/asf/hive/blob/b615de25/ql/src/test/results/clientpositive/fold_case.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/fold_case.q.out b/ql/src/test/results/clientpositive/fold_case.q.out
index 39f2e6c..a6faca0 100644
--- a/ql/src/test/results/clientpositive/fold_case.q.out
+++ b/ql/src/test/results/clientpositive/fold_case.q.out
@@ -215,7 +215,7 @@ STAGE PLANS:
             alias: src
             Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
             Filter Operator
-              predicate: CASE (key) WHEN ('238') THEN (true) ELSE (null) END (type: boolean)
+              predicate: (key = '238') (type: boolean)
               Statistics: Num rows: 250 Data size: 2656 Basic stats: COMPLETE Column stats: NONE
               Select Operator
                 Statistics: Num rows: 250 Data size: 2656 Basic stats: COMPLETE Column stats: NONE
@@ -249,10 +249,59 @@ STAGE PLANS:
         ListSink
 
 PREHOOK: query: explain 
-select count(1) from src where (case key when '238' then null else 1=1 end)
+select count(1) from src where (case key when '238' then 1=null  end)
 PREHOOK: type: QUERY
 POSTHOOK: query: explain 
-select count(1) from src where (case key when '238' then null else 1=1 end)
+select count(1) from src where (case key when '238' then 1=null  end)
+POSTHOOK: type: QUERY
+STAGE DEPENDENCIES:
+  Stage-1 is a root stage
+  Stage-0 depends on stages: Stage-1
+
+STAGE PLANS:
+  Stage: Stage-1
+    Map Reduce
+      Map Operator Tree:
+          TableScan
+            alias: src
+            Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: COMPLETE
+            Filter Operator
+              predicate: false (type: boolean)
+              Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: COMPLETE
+              Group By Operator
+                aggregations: count(1)
+                mode: hash
+                outputColumnNames: _col0
+                Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE
+                Reduce Output Operator
+                  sort order: 
+                  Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE
+                  value expressions: _col0 (type: bigint)
+      Reduce Operator Tree:
+        Group By Operator
+          aggregations: count(VALUE._col0)
+          mode: mergepartial
+          outputColumnNames: _col0
+          Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE
+          File Output Operator
+            compressed: false
+            Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE
+            table:
+                input format: org.apache.hadoop.mapred.TextInputFormat
+                output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
+                serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
+
+  Stage: Stage-0
+    Fetch Operator
+      limit: -1
+      Processor Tree:
+        ListSink
+
+PREHOOK: query: explain 
+select count(1) from src where (case key when '238' then 2 = cast('2' as bigint) end)
+PREHOOK: type: QUERY
+POSTHOOK: query: explain 
+select count(1) from src where (case key when '238' then 2 = cast('2' as bigint) end)
 POSTHOOK: type: QUERY
 STAGE DEPENDENCIES:
   Stage-1 is a root stage
@@ -266,7 +315,7 @@ STAGE PLANS:
             alias: src
             Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
             Filter Operator
-              predicate: CASE (key) WHEN ('238') THEN (null) ELSE (true) END (type: boolean)
+              predicate: (key = '238') (type: boolean)
               Statistics: Num rows: 250 Data size: 2656 Basic stats: COMPLETE Column stats: NONE
               Select Operator
                 Statistics: Num rows: 250 Data size: 2656 Basic stats: COMPLETE Column stats: NONE
@@ -299,3 +348,130 @@ STAGE PLANS:
       Processor Tree:
         ListSink
 
+PREHOOK: query: explain 
+select (case key when '238' then null else false end) from src where (case key when '238' then 2 = cast('1' as bigint)  else true end)
+PREHOOK: type: QUERY
+POSTHOOK: query: explain 
+select (case key when '238' then null else false end) from src where (case key when '238' then 2 = cast('1' as bigint)  else true end)
+POSTHOOK: type: QUERY
+STAGE DEPENDENCIES:
+  Stage-1 is a root stage
+  Stage-0 depends on stages: Stage-1
+
+STAGE PLANS:
+  Stage: Stage-1
+    Map Reduce
+      Map Operator Tree:
+          TableScan
+            alias: src
+            Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
+            Filter Operator
+              predicate: (key <> '238') (type: boolean)
+              Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
+              Select Operator
+                expressions: CASE (key) WHEN ('238') THEN (null) ELSE (false) END (type: boolean)
+                outputColumnNames: _col0
+                Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
+                File Output Operator
+                  compressed: false
+                  Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
+                  table:
+                      input format: org.apache.hadoop.mapred.TextInputFormat
+                      output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
+                      serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
+
+  Stage: Stage-0
+    Fetch Operator
+      limit: -1
+      Processor Tree:
+        ListSink
+
+PREHOOK: query: explain 
+select (case key when '238' then null else null end) from src where (case key when '238' then 2 = null else 3 = null  end)
+PREHOOK: type: QUERY
+POSTHOOK: query: explain 
+select (case key when '238' then null else null end) from src where (case key when '238' then 2 = null else 3 = null  end)
+POSTHOOK: type: QUERY
+STAGE DEPENDENCIES:
+  Stage-1 is a root stage
+  Stage-0 depends on stages: Stage-1
+
+STAGE PLANS:
+  Stage: Stage-1
+    Map Reduce
+      Map Operator Tree:
+          TableScan
+            alias: src
+            Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: COMPLETE
+            Filter Operator
+              predicate: false (type: boolean)
+              Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: COMPLETE
+              Select Operator
+                expressions: null (type: void)
+                outputColumnNames: _col0
+                Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: COMPLETE
+                File Output Operator
+                  compressed: false
+                  Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: COMPLETE
+                  table:
+                      input format: org.apache.hadoop.mapred.TextInputFormat
+                      output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
+                      serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
+
+  Stage: Stage-0
+    Fetch Operator
+      limit: -1
+      Processor Tree:
+        ListSink
+
+PREHOOK: query: explain 
+select count(1) from src where (case key when '238' then null else 1=1 end)
+PREHOOK: type: QUERY
+POSTHOOK: query: explain 
+select count(1) from src where (case key when '238' then null else 1=1 end)
+POSTHOOK: type: QUERY
+STAGE DEPENDENCIES:
+  Stage-1 is a root stage
+  Stage-0 depends on stages: Stage-1
+
+STAGE PLANS:
+  Stage: Stage-1
+    Map Reduce
+      Map Operator Tree:
+          TableScan
+            alias: src
+            Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
+            Filter Operator
+              predicate: (key <> '238') (type: boolean)
+              Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
+              Select Operator
+                Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
+                Group By Operator
+                  aggregations: count(1)
+                  mode: hash
+                  outputColumnNames: _col0
+                  Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
+                  Reduce Output Operator
+                    sort order: 
+                    Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
+                    value expressions: _col0 (type: bigint)
+      Reduce Operator Tree:
+        Group By Operator
+          aggregations: count(VALUE._col0)
+          mode: mergepartial
+          outputColumnNames: _col0
+          Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
+          File Output Operator
+            compressed: false
+            Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
+            table:
+                input format: org.apache.hadoop.mapred.TextInputFormat
+                output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
+                serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
+
+  Stage: Stage-0
+    Fetch Operator
+      limit: -1
+      Processor Tree:
+        ListSink
+

http://git-wip-us.apache.org/repos/asf/hive/blob/b615de25/ql/src/test/results/clientpositive/fold_eq_with_case_when.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/fold_eq_with_case_when.q.out b/ql/src/test/results/clientpositive/fold_eq_with_case_when.q.out
index 45a0cb1..171a19f 100644
--- a/ql/src/test/results/clientpositive/fold_eq_with_case_when.q.out
+++ b/ql/src/test/results/clientpositive/fold_eq_with_case_when.q.out
@@ -42,7 +42,7 @@ STAGE PLANS:
             alias: lineitem
             Statistics: Num rows: 100 Data size: 11999 Basic stats: COMPLETE Column stats: NONE
             Filter Operator
-              predicate: (CASE WHEN (('1996-03-30' = to_date(l_shipdate))) THEN (true) ELSE (null) END and CASE WHEN (('RAIL' = l_shipmode)) THEN (true) ELSE (null) END) (type: boolean)
+              predicate: (('1996-03-30' = to_date(l_shipdate)) and ('RAIL' = l_shipmode)) (type: boolean)
               Statistics: Num rows: 25 Data size: 2999 Basic stats: COMPLETE Column stats: NONE
               Select Operator
                 expressions: l_orderkey (type: int), (UDFToDouble(l_partkey) / 1000000.0) (type: double)

http://git-wip-us.apache.org/repos/asf/hive/blob/b615de25/ql/src/test/results/clientpositive/fold_when.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/fold_when.q.out b/ql/src/test/results/clientpositive/fold_when.q.out
index e169697..0a4b328 100644
--- a/ql/src/test/results/clientpositive/fold_when.q.out
+++ b/ql/src/test/results/clientpositive/fold_when.q.out
@@ -153,7 +153,7 @@ STAGE PLANS:
             alias: src
             Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
             Filter Operator
-              predicate: CASE WHEN ((key = '238')) THEN (true) ELSE (null) END (type: int)
+              predicate: (key = '238') (type: boolean)
               Statistics: Num rows: 250 Data size: 2656 Basic stats: COMPLETE Column stats: NONE
               Select Operator
                 expressions: key (type: string)
@@ -191,10 +191,10 @@ STAGE PLANS:
             alias: src
             Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
             Filter Operator
-              predicate: CASE WHEN ((key = '238')) THEN (true) ELSE (null) END (type: boolean)
+              predicate: (key = '238') (type: boolean)
               Statistics: Num rows: 250 Data size: 2656 Basic stats: COMPLETE Column stats: NONE
               Select Operator
-                expressions: key (type: string)
+                expressions: '238' (type: string)
                 outputColumnNames: _col0
                 Statistics: Num rows: 250 Data size: 2656 Basic stats: COMPLETE Column stats: NONE
                 File Output Operator