You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@calcite.apache.org by vl...@apache.org on 2015/12/10 20:58:25 UTC

calcite git commit: [CALCITE-980] Fix AND and OR implementation in Enumerable convention

Repository: calcite
Updated Branches:
  refs/heads/master 47e0e7c95 -> 963ba1b1b


[CALCITE-980] Fix AND and OR implementation in Enumerable convention

Previously it could fail with NPE or return wrong result

fixes #169


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

Branch: refs/heads/master
Commit: 963ba1b1b3d2ab95989d8383e0a855c3ae5e24cb
Parents: 47e0e7c
Author: Vladimir Sitnikov <si...@gmail.com>
Authored: Thu Dec 10 22:57:27 2015 +0300
Committer: Vladimir Sitnikov <si...@gmail.com>
Committed: Thu Dec 10 22:57:27 2015 +0300

----------------------------------------------------------------------
 .../calcite/adapter/enumerable/RexImpTable.java |  91 ++++---
 .../java/org/apache/calcite/test/JdbcTest.java  |  51 ++++
 .../calcite/test/ReflectiveSchemaTest.java      |  18 ++
 core/src/test/resources/sql/conditions.oq       | 259 +++++++++++++++++++
 4 files changed, 383 insertions(+), 36 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/calcite/blob/963ba1b1/core/src/main/java/org/apache/calcite/adapter/enumerable/RexImpTable.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/adapter/enumerable/RexImpTable.java b/core/src/main/java/org/apache/calcite/adapter/enumerable/RexImpTable.java
index dd21b9d..32c043c 100644
--- a/core/src/main/java/org/apache/calcite/adapter/enumerable/RexImpTable.java
+++ b/core/src/main/java/org/apache/calcite/adapter/enumerable/RexImpTable.java
@@ -55,10 +55,8 @@ import org.apache.calcite.sql.validate.SqlUserDefinedFunction;
 import org.apache.calcite.util.BuiltInMethod;
 import org.apache.calcite.util.Util;
 
-import com.google.common.base.Function;
 import com.google.common.base.Supplier;
 import com.google.common.collect.ImmutableList;
-import com.google.common.collect.Lists;
 import com.google.common.collect.Maps;
 
 import java.lang.reflect.Constructor;
@@ -428,22 +426,38 @@ public class RexImpTable {
       return new CallImplementor() {
         public Expression implement(
             RexToLixTranslator translator, RexCall call, NullAs nullAs) {
+          assert call.getOperator() == AND
+              : "AND null semantics is supported only for AND operator. Actual operator is "
+              + String.valueOf(call.getOperator());
           final RexCall call2 = call2(false, translator, call);
-          final NullAs nullAs2 = nullAs == NullAs.TRUE ? NullAs.NULL : nullAs;
-          final List<Expression> expressions =
-              translator.translateList(call2.getOperands(), nullAs2);
           switch (nullAs) {
-          case NOT_POSSIBLE:
+          case NOT_POSSIBLE: // Just foldAnd
           case TRUE:
+            // AND call should return false iff has FALSEs,
+            // thus if we convert nulls to true then no harm is made
+          case FALSE:
+            // AND call should return false iff has FALSEs or has NULLs,
+            // thus if we convert nulls to false, no harm is made
+            final List<Expression> expressions =
+                translator.translateList(call2.getOperands(), nullAs);
             return Expressions.foldAnd(expressions);
+          case NULL:
+          case IS_NULL:
+          case IS_NOT_NULL:
+            final List<Expression> nullAsTrue =
+                translator.translateList(call2.getOperands(), NullAs.TRUE);
+            final List<Expression> nullAsIsNull =
+                translator.translateList(call2.getOperands(), NullAs.IS_NULL);
+            Expression hasFalse = Expressions.not(Expressions.foldAnd(nullAsTrue));
+            Expression hasNull = Expressions.foldOr(nullAsIsNull);
+            Expression result = nullAs.handle(
+                Expressions.condition(hasFalse, BOXED_FALSE_EXPR,
+                    Expressions.condition(hasNull, NULL_EXPR, BOXED_TRUE_EXPR)));
+            return result;
+          default:
+            throw new IllegalArgumentException(
+                "Unknown nullAs when implementing AND: " + nullAs);
           }
-          return Expressions.foldAnd(
-              Lists.transform(expressions,
-                  new Function<Expression, Expression>() {
-                    public Expression apply(Expression e) {
-                      return nullAs2.handle(e);
-                    }
-                  }));
         }
       };
     case OR:
@@ -456,34 +470,39 @@ public class RexImpTable {
       //   : Boolean.TRUE;
       return new CallImplementor() {
         public Expression implement(
-            RexToLixTranslator translator, RexCall call, NullAs nullAs) {
+            RexToLixTranslator translator, RexCall call, final NullAs nullAs) {
+          assert call.getOperator() == OR
+              : "OR null semantics is supported only for OR operator. Actual operator is "
+              + String.valueOf(call.getOperator());
           final RexCall call2 = call2(harmonize, translator, call);
-          final NullAs nullAs2 = nullAs == NullAs.TRUE ? NullAs.NULL : nullAs;
-          final List<Expression> expressions =
-              translator.translateList(call2.getOperands(), nullAs2);
           switch (nullAs) {
-          case NOT_POSSIBLE:
+          case NOT_POSSIBLE: // Just foldOr
+          case TRUE:
+            // This should return false iff all arguments are FALSE,
+            // thus we convert nulls to TRUE and foldOr
           case FALSE:
+            // This should return true iff has TRUE arguments,
+            // thus we convert nulls to FALSE and foldOr
+            final List<Expression> expressions =
+                translator.translateList(call2.getOperands(), nullAs);
             return Expressions.foldOr(expressions);
+          case NULL:
+          case IS_NULL:
+          case IS_NOT_NULL:
+            final List<Expression> nullAsFalse =
+                translator.translateList(call2.getOperands(), NullAs.FALSE);
+            final List<Expression> nullAsIsNull =
+                translator.translateList(call2.getOperands(), NullAs.IS_NULL);
+            Expression hasTrue = Expressions.foldOr(nullAsFalse);
+            Expression hasNull = Expressions.foldOr(nullAsIsNull);
+            Expression result = nullAs.handle(
+                Expressions.condition(hasTrue, BOXED_TRUE_EXPR,
+                    Expressions.condition(hasNull, NULL_EXPR, BOXED_FALSE_EXPR)));
+            return result;
+          default:
+            throw new IllegalArgumentException(
+                "Unknown nullAs when implementing OR: " + nullAs);
           }
-          final Expression t0 = expressions.get(0);
-          final Expression t1 = expressions.get(1);
-          if (!nullable(call2, 0) && !nullable(call2, 1)) {
-            return Expressions.orElse(t0, t1);
-          }
-          return optimize(
-              Expressions.condition(
-                  Expressions.equal(t0, NULL_EXPR),
-                  Expressions.condition(
-                      Expressions.orElse(
-                          Expressions.equal(t1, NULL_EXPR),
-                          Expressions.not(t1)),
-                      NULL_EXPR,
-                      BOXED_TRUE_EXPR),
-                  Expressions.condition(
-                      Expressions.not(t0),
-                      t1,
-                      BOXED_TRUE_EXPR)));
         }
       };
     case NOT:

http://git-wip-us.apache.org/repos/asf/calcite/blob/963ba1b1/core/src/test/java/org/apache/calcite/test/JdbcTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/calcite/test/JdbcTest.java b/core/src/test/java/org/apache/calcite/test/JdbcTest.java
index 5cf7d16..4c203d6 100644
--- a/core/src/test/java/org/apache/calcite/test/JdbcTest.java
+++ b/core/src/test/java/org/apache/calcite/test/JdbcTest.java
@@ -4277,6 +4277,57 @@ public class JdbcTest {
             "empid=100; deptno=10; name=Bill; salary=10000.0; commission=1000\n");
   }
 
+  /** Tests CALCITE-980: Not (C='a' or C='b') causes NPE */
+  @Test public void testWhereOrAndNullable() {
+    /* Generates the following code:
+       public boolean moveNext() {
+         while (inputEnumerator.moveNext()) {
+           final Object[] current = (Object[]) inputEnumerator.current();
+           final String inp0_ = current[0] == null ? (String) null : current[0].toString();
+           final String inp1_ = current[1] == null ? (String) null : current[1].toString();
+           if (inp0_ != null && org.apache.calcite.runtime.SqlFunctions.eq(inp0_, "a")
+               && (inp1_ != null && org.apache.calcite.runtime.SqlFunctions.eq(inp1_, "b"))
+               || inp0_ != null && org.apache.calcite.runtime.SqlFunctions.eq(inp0_, "b")
+               && (inp1_ != null && org.apache.calcite.runtime.SqlFunctions.eq(inp1_, "c"))) {
+             return true;
+           }
+         }
+         return false;
+       }
+     */
+    CalciteAssert.that()
+        .with(CalciteAssert.Config.REGULAR)
+        .query("with tst(c) as (values('a'),('b'),('c'),(cast(null as varchar)))"
+            + " select u.c u, v.c v from tst u, tst v where ((u.c = 'a' and v.c = 'b') or (u.c = 'b' and v.c = 'c'))")
+        .returnsUnordered(
+            "U=a; V=b",
+            "U=b; V=c");
+  }
+
+  /** Tests CALCITE-980: different flavors of boolean logic */
+  @Test public void testBooleansInWhere() throws Exception {
+    checkRun("sql/conditions.oq");
+  }
+
+  /** Tests CALCITE-980: different flavors of boolean logic */
+  @Ignore("Fails with org.codehaus.commons.compiler.CompileException: Line 16, Column 112:"
+      + " Cannot compare types \"int\" and \"java.lang.String\"\n")
+  @Test public void testComparingIntAndString() throws Exception {
+    // if (((...test.ReflectiveSchemaTest.IntAndString) inputEnumerator.current()).id == "T")
+
+    CalciteAssert.that()
+        .withSchema("s",
+            new ReflectiveSchema(
+                new ReflectiveSchemaTest.CatchallSchema()))
+        .query("select a.\"value\", b.\"value\"\n"
+            + "  from \"bools\" a\n"
+            + "     , \"bools\" b\n"
+            + " where b.\"value\" = 'T'\n"
+            + " order by 1, 2")
+        .returnsUnordered(
+            "should fail with 'not a number' sql error while converting text to number");
+  }
+
   /** Tests the LIKE operator. */
   @Test public void testLike() {
     CalciteAssert.that()

http://git-wip-us.apache.org/repos/asf/calcite/blob/963ba1b1/core/src/test/java/org/apache/calcite/test/ReflectiveSchemaTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/calcite/test/ReflectiveSchemaTest.java b/core/src/test/java/org/apache/calcite/test/ReflectiveSchemaTest.java
index 95b15d6..0ebd616 100644
--- a/core/src/test/java/org/apache/calcite/test/ReflectiveSchemaTest.java
+++ b/core/src/test/java/org/apache/calcite/test/ReflectiveSchemaTest.java
@@ -798,6 +798,17 @@ public class ReflectiveSchemaTest {
     public final BitSet bitSet = new BitSet(0);
   }
 
+  /** Table that has integer and string fields */
+  public static class IntAndString {
+    public final int id;
+    public final String value;
+
+    public IntAndString(int id, String value) {
+      this.id = id;
+      this.value = value;
+    }
+  }
+
   /** Object whose fields are relations. Called "catch-all" because it's OK
    * if tests add new fields. */
   public static class CatchallSchema {
@@ -841,6 +852,13 @@ public class ReflectiveSchemaTest {
 
     public final IntHolder[] primesCustomBoxed =
         new IntHolder[]{new IntHolder(1), new IntHolder(3), new IntHolder(5)};
+
+    public final IntAndString[] nullables = new IntAndString[] {
+      new IntAndString(1, "A"), new IntAndString(2, "B"), new IntAndString(2, "C"),
+      new IntAndString(3, null)};
+
+    public final IntAndString[] bools = new IntAndString[] {
+      new IntAndString(1, "T"), new IntAndString(2, "F"), new IntAndString(3, null)};
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/calcite/blob/963ba1b1/core/src/test/resources/sql/conditions.oq
----------------------------------------------------------------------
diff --git a/core/src/test/resources/sql/conditions.oq b/core/src/test/resources/sql/conditions.oq
new file mode 100644
index 0000000..f7d7f51
--- /dev/null
+++ b/core/src/test/resources/sql/conditions.oq
@@ -0,0 +1,259 @@
+# conditions.oq - conditions
+#
+# 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.
+#
+!use catchall
+!set outputformat mysql
+
+# OR test
+
+with tmp(a, b) as (
+  values (1, 1), (1, 0), (1, cast(null as int))
+       , (0, 1), (0, 0), (0, cast(null as int))
+       , (cast(null as int), 1), (cast(null as int), 0), (cast(null as int), cast(null as int)))
+select *
+  from tmp
+ where a = 1 or b = 1
+ order by 1, 2;
+
++---+---+
+| A | B |
++---+---+
+| 0 | 1 |
+| 1 | 0 |
+| 1 | 1 |
+| 1 |   |
+|   | 1 |
++---+---+
+(5 rows)
+
+!ok
+
+with tmp(a, b) as (
+  values (1, 1), (1, 0), (1, cast(null as int))
+       , (0, 1), (0, 0), (0, cast(null as int))
+       , (cast(null as int), 1), (cast(null as int), 0), (cast(null as int), cast(null as int)))
+select *
+  from tmp
+ where not (a = 1 or b = 1)
+ order by 1, 2;
+
++---+---+
+| A | B |
++---+---+
+| 0 | 0 |
++---+---+
+(1 row)
+
+!ok
+
+# AND test
+
+with tmp(a, b) as (
+  values (1, 1), (1, 0), (1, cast(null as int))
+       , (0, 1), (0, 0), (0, cast(null as int))
+       , (cast(null as int), 1), (cast(null as int), 0), (cast(null as int), cast(null as int)))
+select *
+  from tmp
+ where a = 1 AND b = 1
+ order by 1, 2;
+
++---+---+
+| A | B |
++---+---+
+| 1 | 1 |
++---+---+
+(1 row)
+
+!ok
+
+with tmp(a, b) as (
+  values (1, 1), (1, 0), (1, cast(null as int))
+       , (0, 1), (0, 0), (0, cast(null as int))
+       , (cast(null as int), 1), (cast(null as int), 0), (cast(null as int), cast(null as int)))
+select *
+  from tmp
+ where not (a = 1 AND b = 1)
+ order by 1, 2;
+
++---+---+
+| A | B |
++---+---+
+| 0 | 0 |
+| 0 | 1 |
+| 0 |   |
+| 1 | 0 |
+|   | 0 |
++---+---+
+(5 rows)
+
+!ok
+
+# Test cases for CALCITE-980
+
+select "value" from "nullables" a where "value" = 'A' or "value" = 'B' order by 1;
+
++-------+
+| value |
++-------+
+| A     |
+| B     |
++-------+
+(2 rows)
+
+!ok
+
+select "value" from "nullables" a where not ("value" = 'A' or "value" = 'B') order by 1;
+
++-------+
+| value |
++-------+
+| C     |
++-------+
+(1 row)
+
+!ok
+
+select "value" from "nullables" a where not (not ("value" = 'A' or "value" = 'B')) order by 1;
+
++-------+
+| value |
++-------+
+| A     |
+| B     |
++-------+
+(2 rows)
+
+!ok
+
+select "value" from "nullables" a where "value" = 'A' and "value" = 'B' order by 1;
+
++-------+
+| value |
++-------+
++-------+
+(0 rows)
+
+!ok
+
+select "value" from "nullables" a where not ("value" = 'A' and "value" = 'B') order by 1;
+
++-------+
+| value |
++-------+
+| A     |
+| B     |
+| C     |
++-------+
+(3 rows)
+
+!ok
+
+select "value" from "nullables" a where not (not ("value" = 'A' and "value" = 'B')) order by 1;
+
++-------+
+| value |
++-------+
++-------+
+(0 rows)
+
+!ok
+
+select "value" from "nullables" a
+ where case when not ("value" = 'A' or "value" = 'B') then 1 else 0 end = 1
+ order by 1;
+
++-------+
+| value |
++-------+
+| C     |
++-------+
+(1 row)
+
+!ok
+
+select "value" from "nullables" a
+ where
+   case when not ("value"='A' or "value"='B')
+     then
+       case when ("value"='A' or "value"='B') then 1 else 2 end
+     else 0
+   end = 2
+ order by 1;
+
++-------+
+| value |
++-------+
+| C     |
++-------+
+(1 row)
+
+!ok
+
+select "value" from "nullables" a
+ where
+   case when not ("value"='A' or "value"='B')
+     then
+       case when not /* <--diff from above */ ("value"='A' or "value"='B') then 1 else 2 end
+     else 0
+   end = 1 /* <- diff from above*/
+ order by 1;
+
++-------+
+| value |
++-------+
+| C     |
++-------+
+(1 row)
+
+!ok
+
+select "value" from "nullables" a
+ where
+   case when not ("value"='A' or "value"='B')
+     then
+       case when not ("value"='A' or "value"='B') then 1 else 2 end
+     else 0
+   end = 0 /* <- diff from above*/
+ order by 1;
+
++-------+
+| value |
++-------+
+| A     |
+| B     |
+|       |
++-------+
+(3 rows)
+
+!ok
+
+select "value" from "nullables" a
+ where
+   case when not ("value"='A' or "value"='B')
+     then
+       case when not ("value"='A' or "value"='B') then 1 else 2 end
+     else 0
+   end = 2 /* <- diff from above*/
+ order by 1;
+
++-------+
+| value |
++-------+
++-------+
+(0 rows)
+
+!ok