You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@calcite.apache.org by mm...@apache.org on 2017/09/05 14:36:51 UTC

[08/16] calcite git commit: [CALCITE-1969] Annotate user-defined functions as strict and semi-strict

[CALCITE-1969] Annotate user-defined functions as strict and semi-strict

Also add an "Experimental" annotation.


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

Branch: refs/heads/branch-1.14
Commit: b2bf1ca0a40762b904f7334c92e851b21f6b2844
Parents: 4208d80
Author: Julian Hyde <jh...@apache.org>
Authored: Fri Aug 25 16:17:53 2017 -0700
Committer: Julian Hyde <jh...@apache.org>
Committed: Tue Aug 29 10:15:18 2017 -0700

----------------------------------------------------------------------
 .../calcite/adapter/enumerable/NullPolicy.java  |  2 +
 .../calcite/adapter/enumerable/RexImpTable.java | 10 ++--
 .../calcite/prepare/CalciteCatalogReader.java   |  9 +++-
 .../calcite/schema/impl/ScalarFunctionImpl.java | 44 ++++++++++++++--
 .../sql/type/SqlTypeExplicitPrecedenceList.java |  4 +-
 .../java/org/apache/calcite/test/UdfTest.java   | 53 ++++++++++++++++++++
 .../java/org/apache/calcite/util/Smalls.java    | 24 +++++++++
 .../calcite/linq4j/function/Experimental.java   | 42 ++++++++++++++++
 .../calcite/linq4j/function/SemiStrict.java     | 43 ++++++++++++++++
 .../apache/calcite/linq4j/function/Strict.java  | 39 ++++++++++++++
 10 files changed, 261 insertions(+), 9 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/calcite/blob/b2bf1ca0/core/src/main/java/org/apache/calcite/adapter/enumerable/NullPolicy.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/adapter/enumerable/NullPolicy.java b/core/src/main/java/org/apache/calcite/adapter/enumerable/NullPolicy.java
index dedf300..43c3431 100644
--- a/core/src/main/java/org/apache/calcite/adapter/enumerable/NullPolicy.java
+++ b/core/src/main/java/org/apache/calcite/adapter/enumerable/NullPolicy.java
@@ -27,6 +27,8 @@ package org.apache.calcite.adapter.enumerable;
 public enum NullPolicy {
   /** Returns null if and only if one of the arguments are null. */
   STRICT,
+  /** Returns null if one of the arguments is null, and possibly other times. */
+  SEMI_STRICT,
   /** If any of the arguments are null, return null. */
   ANY,
   /** If any of the arguments are false, result is false; else if any

http://git-wip-us.apache.org/repos/asf/calcite/blob/b2bf1ca0/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 00e9764..17cb540 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
@@ -475,6 +475,7 @@ public class RexImpTable {
     switch (nullPolicy) {
     case ANY:
     case STRICT:
+    case SEMI_STRICT:
       return new CallImplementor() {
         public Expression implement(
             RexToLixTranslator translator, RexCall call, NullAs nullAs) {
@@ -812,7 +813,8 @@ public class RexImpTable {
     case IS_NOT_NULL:
       // If "f" is strict, then "f(a0, a1) IS NOT NULL" is
       // equivalent to "a0 IS NOT NULL AND a1 IS NOT NULL".
-      if (nullPolicy == NullPolicy.STRICT) {
+      switch (nullPolicy) {
+      case STRICT:
         return Expressions.foldAnd(
             translator.translateList(
                 call.getOperands(), nullAs));
@@ -821,7 +823,8 @@ public class RexImpTable {
     case IS_NULL:
       // If "f" is strict, then "f(a0, a1) IS NULL" is
       // equivalent to "a0 IS NULL OR a1 IS NULL".
-      if (nullPolicy == NullPolicy.STRICT) {
+      switch (nullPolicy) {
+      case STRICT:
         return Expressions.foldOr(
             translator.translateList(
                 call.getOperands(), nullAs));
@@ -904,7 +907,8 @@ public class RexImpTable {
       // RexNode can be referred via multiple ways: RexNode itself, RexLocalRef,
       // and may be others.
       final Map<RexNode, Boolean> nullable = new HashMap<>();
-      if (nullPolicy == NullPolicy.STRICT) {
+      switch (nullPolicy) {
+      case STRICT:
         // The arguments should be not nullable if STRICT operator is computed
         // in nulls NOT_POSSIBLE mode
         for (RexNode arg : call.getOperands()) {

http://git-wip-us.apache.org/repos/asf/calcite/blob/b2bf1ca0/core/src/main/java/org/apache/calcite/prepare/CalciteCatalogReader.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/prepare/CalciteCatalogReader.java b/core/src/main/java/org/apache/calcite/prepare/CalciteCatalogReader.java
index ab1bbe4..b7f32e1 100644
--- a/core/src/main/java/org/apache/calcite/prepare/CalciteCatalogReader.java
+++ b/core/src/main/java/org/apache/calcite/prepare/CalciteCatalogReader.java
@@ -33,6 +33,7 @@ import org.apache.calcite.schema.Table;
 import org.apache.calcite.schema.TableFunction;
 import org.apache.calcite.schema.TableMacro;
 import org.apache.calcite.schema.Wrapper;
+import org.apache.calcite.schema.impl.ScalarFunctionImpl;
 import org.apache.calcite.sql.SqlFunctionCategory;
 import org.apache.calcite.sql.SqlIdentifier;
 import org.apache.calcite.sql.SqlOperator;
@@ -354,7 +355,13 @@ public class CalciteCatalogReader implements Prepare.CatalogReader {
   private SqlReturnTypeInference infer(final ScalarFunction function) {
     return new SqlReturnTypeInference() {
       public RelDataType inferReturnType(SqlOperatorBinding opBinding) {
-        final RelDataType type = function.getReturnType(typeFactory);
+        final RelDataType type;
+        if (function instanceof ScalarFunctionImpl) {
+          type = ((ScalarFunctionImpl) function).getReturnType(typeFactory,
+              opBinding);
+        } else {
+          type = function.getReturnType(typeFactory);
+        }
         return toSql(type);
       }
     };

http://git-wip-us.apache.org/repos/asf/calcite/blob/b2bf1ca0/core/src/main/java/org/apache/calcite/schema/impl/ScalarFunctionImpl.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/schema/impl/ScalarFunctionImpl.java b/core/src/main/java/org/apache/calcite/schema/impl/ScalarFunctionImpl.java
index 18f398a..2429e0c 100644
--- a/core/src/main/java/org/apache/calcite/schema/impl/ScalarFunctionImpl.java
+++ b/core/src/main/java/org/apache/calcite/schema/impl/ScalarFunctionImpl.java
@@ -20,10 +20,13 @@ import org.apache.calcite.adapter.enumerable.CallImplementor;
 import org.apache.calcite.adapter.enumerable.NullPolicy;
 import org.apache.calcite.adapter.enumerable.ReflectiveCallNotNullImplementor;
 import org.apache.calcite.adapter.enumerable.RexImpTable;
+import org.apache.calcite.linq4j.function.SemiStrict;
+import org.apache.calcite.linq4j.function.Strict;
 import org.apache.calcite.rel.type.RelDataType;
 import org.apache.calcite.rel.type.RelDataTypeFactory;
 import org.apache.calcite.schema.ImplementableFunction;
 import org.apache.calcite.schema.ScalarFunction;
+import org.apache.calcite.sql.SqlOperatorBinding;
 
 import com.google.common.collect.ImmutableMultimap;
 
@@ -35,8 +38,8 @@ import static org.apache.calcite.util.Static.RESOURCE;
 /**
 * Implementation of {@link org.apache.calcite.schema.ScalarFunction}.
 */
-public class ScalarFunctionImpl extends ReflectiveFunctionBase implements
-    ScalarFunction, ImplementableFunction {
+public class ScalarFunctionImpl extends ReflectiveFunctionBase
+    implements ScalarFunction, ImplementableFunction {
   private final CallImplementor implementor;
 
   /** Private constructor. */
@@ -112,8 +115,43 @@ public class ScalarFunctionImpl extends ReflectiveFunctionBase implements
   }
 
   private static CallImplementor createImplementor(final Method method) {
+    final NullPolicy nullPolicy = getNullPolicy(method);
     return RexImpTable.createImplementor(
-        new ReflectiveCallNotNullImplementor(method), NullPolicy.NONE, false);
+        new ReflectiveCallNotNullImplementor(method), nullPolicy, false);
+  }
+
+  private static NullPolicy getNullPolicy(Method m) {
+    if (m.getAnnotation(Strict.class) != null) {
+      return NullPolicy.STRICT;
+    } else if (m.getAnnotation(SemiStrict.class) != null) {
+      return NullPolicy.SEMI_STRICT;
+    } else if (m.getDeclaringClass().getAnnotation(Strict.class) != null) {
+      return NullPolicy.STRICT;
+    } else if (m.getDeclaringClass().getAnnotation(SemiStrict.class) != null) {
+      return NullPolicy.SEMI_STRICT;
+    } else {
+      return NullPolicy.NONE;
+    }
+  }
+
+  public RelDataType getReturnType(RelDataTypeFactory typeFactory,
+      SqlOperatorBinding opBinding) {
+    // Strict and semi-strict functions can return null even if their Java
+    // functions return a primitive type. Because when one of their arguments
+    // is null, they won't even be called.
+    final RelDataType returnType = getReturnType(typeFactory);
+    switch (getNullPolicy(method)) {
+    case STRICT:
+      for (RelDataType type : opBinding.collectOperandTypes()) {
+        if (type.isNullable()) {
+          return typeFactory.createTypeWithNullability(returnType, true);
+        }
+      }
+      break;
+    case SEMI_STRICT:
+      return typeFactory.createTypeWithNullability(returnType, true);
+    }
+    return returnType;
   }
 }
 

http://git-wip-us.apache.org/repos/asf/calcite/blob/b2bf1ca0/core/src/main/java/org/apache/calcite/sql/type/SqlTypeExplicitPrecedenceList.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/sql/type/SqlTypeExplicitPrecedenceList.java b/core/src/main/java/org/apache/calcite/sql/type/SqlTypeExplicitPrecedenceList.java
index 7599bb7..8dcdc8b 100644
--- a/core/src/main/java/org/apache/calcite/sql/type/SqlTypeExplicitPrecedenceList.java
+++ b/core/src/main/java/org/apache/calcite/sql/type/SqlTypeExplicitPrecedenceList.java
@@ -151,8 +151,8 @@ public class SqlTypeExplicitPrecedenceList
 
   // implement RelDataTypePrecedenceList
   public int compareTypePrecedence(RelDataType type1, RelDataType type2) {
-    assert containsType(type1);
-    assert containsType(type2);
+    assert containsType(type1) : type1;
+    assert containsType(type2) : type2;
 
     int p1 =
         getListPosition(

http://git-wip-us.apache.org/repos/asf/calcite/blob/b2bf1ca0/core/src/test/java/org/apache/calcite/test/UdfTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/calcite/test/UdfTest.java b/core/src/test/java/org/apache/calcite/test/UdfTest.java
index 2cdc449..6b5b5e3 100644
--- a/core/src/test/java/org/apache/calcite/test/UdfTest.java
+++ b/core/src/test/java/org/apache/calcite/test/UdfTest.java
@@ -22,6 +22,7 @@ import org.apache.calcite.adapter.enumerable.RexToLixTranslator;
 import org.apache.calcite.adapter.java.ReflectiveSchema;
 import org.apache.calcite.jdbc.CalciteConnection;
 import org.apache.calcite.linq4j.Ord;
+import org.apache.calcite.linq4j.function.SemiStrict;
 import org.apache.calcite.linq4j.tree.Expression;
 import org.apache.calcite.linq4j.tree.Expressions;
 import org.apache.calcite.linq4j.tree.Types;
@@ -149,6 +150,18 @@ public class UdfTest {
         + "           methodName: 'abs'\n"
         + "         },\n"
         + "         {\n"
+        + "           name: 'NULL4',\n"
+        + "           className: '"
+        + Smalls.Null4Function.class.getName()
+        + "'\n"
+        + "         },\n"
+        + "         {\n"
+        + "           name: 'NULL8',\n"
+        + "           className: '"
+        + Smalls.Null8Function.class.getName()
+        + "'\n"
+        + "         },\n"
+        + "         {\n"
         + "           className: '"
         + Smalls.MultipleFunction.class.getName()
         + "',\n"
@@ -291,6 +304,46 @@ public class UdfTest {
         .returns("");
   }
 
+  /** Tests that we generate the appropriate checks for a "semi-strict"
+   * function.
+   *
+   * <p>The difference between "strict" and "semi-strict" functions is that a
+   * "semi-strict" function might return null even if none of its arguments
+   * are null. (Both always return null if one of their arguments is null.)
+   * Thus, a nasty function is more unpredictable.
+   *
+   * @see SemiStrict */
+  @Test public void testSemiStrict() {
+    final CalciteAssert.AssertThat with = withUdf();
+    final String sql = "select\n"
+        + "  \"adhoc\".null4(upper(\"name\")) as p\n"
+        + " from \"adhoc\".EMPLOYEES";
+    with.query(sql)
+        .returnsUnordered("P=null",
+            "P=null",
+            "P=SEBASTIAN",
+            "P=THEODORE");
+    // my_str is non-strict; it must be called when args are null
+    final String sql2 = "select\n"
+        + "  \"adhoc\".my_str(upper(\"adhoc\".null4(\"name\"))) as p\n"
+        + " from \"adhoc\".EMPLOYEES";
+    with.query(sql2)
+        .returnsUnordered("P=<null>",
+            "P=<null>",
+            "P=<SEBASTIAN>",
+            "P=<THEODORE>");
+    // null8 throws NPE if its argument is null,
+    // so we had better know that null4 might return null
+    final String sql3 = "select\n"
+        + "  \"adhoc\".null8(\"adhoc\".null4(\"name\")) as p\n"
+        + " from \"adhoc\".EMPLOYEES";
+    with.query(sql3)
+        .returnsUnordered("P=null",
+            "P=null",
+            "P=Sebastian",
+            "P=null");
+  }
+
   /** Tests derived return type of user-defined function. */
   @Test public void testUdfDerivedReturnType() {
     final CalciteAssert.AssertThat with = withUdf();

http://git-wip-us.apache.org/repos/asf/calcite/blob/b2bf1ca0/core/src/test/java/org/apache/calcite/util/Smalls.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/calcite/util/Smalls.java b/core/src/test/java/org/apache/calcite/util/Smalls.java
index 39f0263..6ea55bf 100644
--- a/core/src/test/java/org/apache/calcite/util/Smalls.java
+++ b/core/src/test/java/org/apache/calcite/util/Smalls.java
@@ -30,6 +30,7 @@ import org.apache.calcite.linq4j.function.Deterministic;
 import org.apache.calcite.linq4j.function.Function1;
 import org.apache.calcite.linq4j.function.Function2;
 import org.apache.calcite.linq4j.function.Parameter;
+import org.apache.calcite.linq4j.function.SemiStrict;
 import org.apache.calcite.linq4j.tree.Types;
 import org.apache.calcite.rel.type.RelDataType;
 import org.apache.calcite.rel.type.RelDataTypeFactory;
@@ -450,6 +451,29 @@ public class Smalls {
     }
   }
 
+  /** Example of a semi-strict UDF.
+   * (Returns null if its parameter is null or if its length is 4.) */
+  public static class Null4Function {
+    @SemiStrict public static String eval(@Parameter(name = "s") String s) {
+      if (s == null || s.length() == 4) {
+        return null;
+      }
+      return s;
+    }
+  }
+
+  /** Example of a picky, semi-strict UDF.
+   * Throws {@link NullPointerException} if argument is null.
+   * Returns null if its argument's length is 8. */
+  public static class Null8Function {
+    @SemiStrict public static String eval(@Parameter(name = "s") String s) {
+      if (s.length() == 8) {
+        return null;
+      }
+      return s;
+    }
+  }
+
   /** Example of a UDF with a static {@code eval} method. Class is abstract,
    * but code-generator should not need to instantiate it. */
   public abstract static class MyDoubleFunction {

http://git-wip-us.apache.org/repos/asf/calcite/blob/b2bf1ca0/linq4j/src/main/java/org/apache/calcite/linq4j/function/Experimental.java
----------------------------------------------------------------------
diff --git a/linq4j/src/main/java/org/apache/calcite/linq4j/function/Experimental.java b/linq4j/src/main/java/org/apache/calcite/linq4j/function/Experimental.java
new file mode 100644
index 0000000..630384d
--- /dev/null
+++ b/linq4j/src/main/java/org/apache/calcite/linq4j/function/Experimental.java
@@ -0,0 +1,42 @@
+/*
+ * 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.calcite.linq4j.function;
+
+import java.lang.annotation.Retention;
+import java.lang.annotation.RetentionPolicy;
+import java.lang.annotation.Target;
+
+import static java.lang.annotation.ElementType.FIELD;
+import static java.lang.annotation.ElementType.METHOD;
+import static java.lang.annotation.ElementType.PACKAGE;
+import static java.lang.annotation.ElementType.TYPE;
+
+/**
+ * Annotation that indicates that a class, interface, field or method
+ * is experimental, not part of the public API, and subject to change
+ * or removal.
+ *
+ * <p>And yes, it is flagged experimental. We may move it elsewhere in future,
+ * when we re-think the maturity model.
+ */
+@Target({PACKAGE, TYPE, FIELD, METHOD })
+@Retention(RetentionPolicy.SOURCE)
+@Experimental
+public @interface Experimental {
+}
+
+// End Experimental.java

http://git-wip-us.apache.org/repos/asf/calcite/blob/b2bf1ca0/linq4j/src/main/java/org/apache/calcite/linq4j/function/SemiStrict.java
----------------------------------------------------------------------
diff --git a/linq4j/src/main/java/org/apache/calcite/linq4j/function/SemiStrict.java b/linq4j/src/main/java/org/apache/calcite/linq4j/function/SemiStrict.java
new file mode 100644
index 0000000..a9d2dce
--- /dev/null
+++ b/linq4j/src/main/java/org/apache/calcite/linq4j/function/SemiStrict.java
@@ -0,0 +1,43 @@
+/*
+ * 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.calcite.linq4j.function;
+
+import java.lang.annotation.Retention;
+import java.lang.annotation.RetentionPolicy;
+import java.lang.annotation.Target;
+
+import static java.lang.annotation.ElementType.METHOD;
+import static java.lang.annotation.ElementType.TYPE;
+
+/**
+ * Annotation applied to a user-defined function that indicates that
+ * the function always returns null if one or more of its arguments
+ * are null but also may return null at other times.
+ *
+ * <p>Compare with {@link Strict}:
+ * <ul>
+ *   <li>A strict function returns null if and only if it has a null argument
+ *   <li>A semi-strict function returns null if it has a null argument
+ * </ul>
+ */
+@Target({METHOD, TYPE })
+@Retention(RetentionPolicy.RUNTIME)
+@Experimental
+public @interface SemiStrict {
+}
+
+// End SemiStrict.java

http://git-wip-us.apache.org/repos/asf/calcite/blob/b2bf1ca0/linq4j/src/main/java/org/apache/calcite/linq4j/function/Strict.java
----------------------------------------------------------------------
diff --git a/linq4j/src/main/java/org/apache/calcite/linq4j/function/Strict.java b/linq4j/src/main/java/org/apache/calcite/linq4j/function/Strict.java
new file mode 100644
index 0000000..5bddb0a
--- /dev/null
+++ b/linq4j/src/main/java/org/apache/calcite/linq4j/function/Strict.java
@@ -0,0 +1,39 @@
+/*
+ * 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.calcite.linq4j.function;
+
+import java.lang.annotation.Retention;
+import java.lang.annotation.RetentionPolicy;
+import java.lang.annotation.Target;
+
+import static java.lang.annotation.ElementType.METHOD;
+import static java.lang.annotation.ElementType.TYPE;
+
+/**
+ * Annotation applied to a user-defined function that indicates that
+ * the function returns null if and only if one or more of its arguments
+ * are null.
+ *
+ * @see SemiStrict
+ */
+@Target({METHOD, TYPE })
+@Retention(RetentionPolicy.RUNTIME)
+@Experimental
+public @interface Strict {
+}
+
+// End Strict.java