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 2019/07/24 15:03:30 UTC

[calcite] branch master updated: [CALCITE-3195] Handle a UDF that throws checked exceptions in the Enumerable code generator (DonnyZone)

This is an automated email from the ASF dual-hosted git repository.

mmior pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/calcite.git


The following commit(s) were added to refs/heads/master by this push:
     new 18974ce  [CALCITE-3195] Handle a UDF that throws checked exceptions in the Enumerable code generator (DonnyZone)
18974ce is described below

commit 18974ce8a945424a8a3465c03b253e292641f420
Author: wellfengzhu <we...@gmail.com>
AuthorDate: Mon Jul 15 22:49:59 2019 +0800

    [CALCITE-3195] Handle a UDF that throws checked exceptions in the Enumerable code generator (DonnyZone)
---
 .../ReflectiveCallNotNullImplementor.java          | 21 +++++++++++--
 .../adapter/enumerable/RexToLixTranslator.java     | 28 +++++++++++++++++
 .../test/java/org/apache/calcite/test/UdfTest.java | 35 ++++++++++++++++++++++
 .../test/java/org/apache/calcite/util/Smalls.java  | 15 ++++++++++
 .../org/apache/calcite/linq4j/tree/Shuttle.java    | 13 ++++++--
 .../apache/calcite/linq4j/tree/TryStatement.java   | 13 +++++++-
 .../apache/calcite/linq4j/test/InlinerTest.java    | 28 +++++++++++++++++
 7 files changed, 148 insertions(+), 5 deletions(-)

diff --git a/core/src/main/java/org/apache/calcite/adapter/enumerable/ReflectiveCallNotNullImplementor.java b/core/src/main/java/org/apache/calcite/adapter/enumerable/ReflectiveCallNotNullImplementor.java
index 4cb57cd..c083d25 100644
--- a/core/src/main/java/org/apache/calcite/adapter/enumerable/ReflectiveCallNotNullImplementor.java
+++ b/core/src/main/java/org/apache/calcite/adapter/enumerable/ReflectiveCallNotNullImplementor.java
@@ -47,17 +47,34 @@ public class ReflectiveCallNotNullImplementor implements NotNullImplementor {
       RexCall call, List<Expression> translatedOperands) {
     translatedOperands =
         EnumUtils.fromInternal(method.getParameterTypes(), translatedOperands);
+    final Expression callExpr;
     if ((method.getModifiers() & Modifier.STATIC) != 0) {
-      return Expressions.call(method, translatedOperands);
+      callExpr = Expressions.call(method, translatedOperands);
     } else {
       // The UDF class must have a public zero-args constructor.
       // Assume that the validator checked already.
       final Expression target =
           Expressions.new_(method.getDeclaringClass());
-      return Expressions.call(target, method, translatedOperands);
+      callExpr = Expressions.call(target, method, translatedOperands);
     }
+    if (!containsCheckedException(method)) {
+      return callExpr;
+    }
+    return translator.handleMethodCheckedExceptions(callExpr);
   }
 
+  private boolean containsCheckedException(Method method) {
+    Class[] exceptions = method.getExceptionTypes();
+    if (exceptions == null || exceptions.length == 0) {
+      return false;
+    }
+    for (Class clazz : exceptions) {
+      if (!RuntimeException.class.isAssignableFrom(clazz)) {
+        return true;
+      }
+    }
+    return false;
+  }
 }
 
 // End ReflectiveCallNotNullImplementor.java
diff --git a/core/src/main/java/org/apache/calcite/adapter/enumerable/RexToLixTranslator.java b/core/src/main/java/org/apache/calcite/adapter/enumerable/RexToLixTranslator.java
index a828a34..4f7af47 100644
--- a/core/src/main/java/org/apache/calcite/adapter/enumerable/RexToLixTranslator.java
+++ b/core/src/main/java/org/apache/calcite/adapter/enumerable/RexToLixTranslator.java
@@ -22,12 +22,14 @@ import org.apache.calcite.avatica.util.ByteString;
 import org.apache.calcite.avatica.util.DateTimeUtils;
 import org.apache.calcite.linq4j.function.Function1;
 import org.apache.calcite.linq4j.tree.BlockBuilder;
+import org.apache.calcite.linq4j.tree.CatchBlock;
 import org.apache.calcite.linq4j.tree.ConstantExpression;
 import org.apache.calcite.linq4j.tree.Expression;
 import org.apache.calcite.linq4j.tree.ExpressionType;
 import org.apache.calcite.linq4j.tree.Expressions;
 import org.apache.calcite.linq4j.tree.ParameterExpression;
 import org.apache.calcite.linq4j.tree.Primitive;
+import org.apache.calcite.linq4j.tree.Statement;
 import org.apache.calcite.linq4j.tree.UnaryExpression;
 import org.apache.calcite.rel.type.RelDataType;
 import org.apache.calcite.rel.type.RelDataTypeFactoryImpl;
@@ -622,6 +624,32 @@ public class RexToLixTranslator {
     return unboxed;
   }
 
+  /**
+   * Handle checked Exceptions declared in Method. In such case,
+   * method call should be wrapped in a try...catch block.
+   * "
+   *      final Type method_call;
+   *      try {
+   *        method_call = callExpr
+   *      } catch (Exception e) {
+   *        throw new RuntimeException(e);
+   *      }
+   * "
+   */
+  Expression handleMethodCheckedExceptions(Expression callExpr) {
+    // Try statement
+    ParameterExpression methodCall = Expressions.parameter(
+        callExpr.getType(), list.newName("method_call"));
+    list.add(Expressions.declare(Modifier.FINAL, methodCall, null));
+    Statement st = Expressions.statement(Expressions.assign(methodCall, callExpr));
+    // Catch Block, wrap checked exception in unchecked exception
+    ParameterExpression e = Expressions.parameter(0, Exception.class, "e");
+    Expression uncheckedException = Expressions.new_(RuntimeException.class, e);
+    CatchBlock cb = Expressions.catch_(e, Expressions.throw_(uncheckedException));
+    list.add(Expressions.tryCatch(st, cb));
+    return methodCall;
+  }
+
   /** Translates an expression that is not in the cache.
    *
    * @param expr Expression
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 4eac690..cf9fc43 100644
--- a/core/src/test/java/org/apache/calcite/test/UdfTest.java
+++ b/core/src/test/java/org/apache/calcite/test/UdfTest.java
@@ -115,6 +115,12 @@ public class UdfTest {
         + "'\n"
         + "         },\n"
         + "         {\n"
+        + "           name: 'MY_EXCEPTION',\n"
+        + "           className: '"
+        + Smalls.MyExceptionFunction.class.getName()
+        + "'\n"
+        + "         },\n"
+        + "         {\n"
         + "           name: 'COUNT_ARGS',\n"
         + "           className: '"
         + Smalls.CountArgs0Function.class.getName()
@@ -220,6 +226,35 @@ public class UdfTest {
   }
 
   /** Test case for
+   * <a href="https://issues.apache.org/jira/browse/CALCITE-3195">[CALCITE-3195]
+   * Handle a UDF that throws checked exceptions in the Enumerable code generator</a>. */
+  @Test public void testUserDefinedFunctionWithException() throws Exception {
+    final String sql1 = "select \"adhoc\".my_exception(\"deptno\") as p\n"
+        + "from \"adhoc\".EMPLOYEES";
+    final String expected1 = "P=20\n"
+        + "P=30\n"
+        + "P=20\n"
+        + "P=20\n";
+    withUdf().query(sql1).returns(expected1);
+
+    final String sql2 = "select cast(\"adhoc\".my_exception(\"deptno\") as double) as p\n"
+        + "from \"adhoc\".EMPLOYEES";
+    final String expected2 = "P=20.0\n"
+        + "P=30.0\n"
+        + "P=20.0\n"
+        + "P=20.0\n";
+    withUdf().query(sql2).returns(expected2);
+
+    final String sql3 = "select \"adhoc\".my_exception(\"deptno\" * 2 + 11) as p\n"
+        + "from \"adhoc\".EMPLOYEES";
+    final String expected3 = "P=41\n"
+        + "P=61\n"
+        + "P=41\n"
+        + "P=41\n";
+    withUdf().query(sql3).returns(expected3);
+  }
+
+  /** Test case for
    * <a href="https://issues.apache.org/jira/browse/CALCITE-937">[CALCITE-937]
    * User-defined function within view</a>. */
   @Test public void testUserDefinedFunctionInView() throws Exception {
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 4825f75..ac5cc1e 100644
--- a/core/src/test/java/org/apache/calcite/util/Smalls.java
+++ b/core/src/test/java/org/apache/calcite/util/Smalls.java
@@ -50,6 +50,7 @@ import org.apache.calcite.sql.type.SqlTypeName;
 
 import com.google.common.collect.ImmutableList;
 
+import java.io.IOException;
 import java.lang.reflect.Method;
 import java.math.BigDecimal;
 import java.sql.Date;
@@ -468,6 +469,20 @@ public class Smalls {
     }
   }
 
+  /** User-defined function that declares exceptions. */
+  public static class MyExceptionFunction {
+    public MyExceptionFunction() {}
+
+    public static int eval(int x) throws IllegalArgumentException, IOException {
+      if (x < 0) {
+        throw new IllegalArgumentException("Illegal argument: " + x);
+      } else if (x > 100) {
+        throw new IOException("IOException when argument > 100");
+      }
+      return x + 10;
+    }
+  }
+
   /** Example of a UDF that has overloaded UDFs (same name, different args). */
   public abstract static class CountArgs0Function {
     private CountArgs0Function() {}
diff --git a/linq4j/src/main/java/org/apache/calcite/linq4j/tree/Shuttle.java b/linq4j/src/main/java/org/apache/calcite/linq4j/tree/Shuttle.java
index 08aa1ad..0cb793a 100644
--- a/linq4j/src/main/java/org/apache/calcite/linq4j/tree/Shuttle.java
+++ b/linq4j/src/main/java/org/apache/calcite/linq4j/tree/Shuttle.java
@@ -250,8 +250,17 @@ public class Shuttle {
     return switchStatement;
   }
 
-  public Statement visit(TryStatement tryStatement) {
-    return tryStatement;
+  public Shuttle preVisit(TryStatement tryStatement) {
+    return this;
+  }
+
+  public Statement visit(TryStatement tryStatement,
+      Statement body, List<CatchBlock> catchBlocks, Statement fynally) {
+    return body.equals(tryStatement.body)
+           && Objects.equals(catchBlocks, tryStatement.catchBlocks)
+           && Objects.equals(fynally, tryStatement.fynally)
+           ? tryStatement
+           : new TryStatement(body, catchBlocks, fynally);
   }
 
   public Expression visit(MemberInitExpression memberInitExpression) {
diff --git a/linq4j/src/main/java/org/apache/calcite/linq4j/tree/TryStatement.java b/linq4j/src/main/java/org/apache/calcite/linq4j/tree/TryStatement.java
index 6b6c072..a56621f 100644
--- a/linq4j/src/main/java/org/apache/calcite/linq4j/tree/TryStatement.java
+++ b/linq4j/src/main/java/org/apache/calcite/linq4j/tree/TryStatement.java
@@ -16,6 +16,7 @@
  */
 package org.apache.calcite.linq4j.tree;
 
+import java.util.ArrayList;
 import java.util.List;
 import java.util.Objects;
 
@@ -36,7 +37,17 @@ public class TryStatement extends Statement {
   }
 
   @Override public Statement accept(Shuttle shuttle) {
-    return shuttle.visit(this);
+    shuttle = shuttle.preVisit(this);
+    Statement body1 = body.accept(shuttle);
+    List<CatchBlock> catchBlocks1 = new ArrayList<>();
+    for (CatchBlock cb: catchBlocks) {
+      Statement cbBody = cb.body.accept(shuttle);
+      catchBlocks1.add(
+          Expressions.catch_(cb.parameter, cbBody));
+    }
+    Statement fynally1 =
+        fynally == null ? null : fynally.accept(shuttle);
+    return shuttle.visit(this, body1, catchBlocks1, fynally1);
   }
 
   public <R> R accept(Visitor<R> visitor) {
diff --git a/linq4j/src/test/java/org/apache/calcite/linq4j/test/InlinerTest.java b/linq4j/src/test/java/org/apache/calcite/linq4j/test/InlinerTest.java
index 79e2f7d..d389d6b 100644
--- a/linq4j/src/test/java/org/apache/calcite/linq4j/test/InlinerTest.java
+++ b/linq4j/src/test/java/org/apache/calcite/linq4j/test/InlinerTest.java
@@ -17,11 +17,13 @@
 package org.apache.calcite.linq4j.test;
 
 import org.apache.calcite.linq4j.tree.BlockBuilder;
+import org.apache.calcite.linq4j.tree.CatchBlock;
 import org.apache.calcite.linq4j.tree.DeclarationStatement;
 import org.apache.calcite.linq4j.tree.Expression;
 import org.apache.calcite.linq4j.tree.ExpressionType;
 import org.apache.calcite.linq4j.tree.Expressions;
 import org.apache.calcite.linq4j.tree.ParameterExpression;
+import org.apache.calcite.linq4j.tree.Statement;
 
 import org.hamcrest.CoreMatchers;
 import org.junit.Before;
@@ -194,6 +196,32 @@ public class InlinerTest {
             + "}\n",
         Expressions.toString(builder.toBlock()));
   }
+
+  @Test public void testInlineInTryCatchStatement() {
+    final BlockBuilder builder = new BlockBuilder(true);
+    final ParameterExpression t = Expressions.parameter(int.class, "t");
+    builder.add(Expressions.declare(Modifier.FINAL, t, ONE));
+    final ParameterExpression u = Expressions.parameter(int.class, "u");
+    builder.add(Expressions.declare(Modifier.FINAL, u, null));
+    Statement st = Expressions.statement(
+        Expressions.assign(u,
+            Expressions.makeBinary(ExpressionType.Add, t, TWO)));
+    ParameterExpression e = Expressions.parameter(0, Exception.class, "e");
+    CatchBlock cb = Expressions.catch_(e, Expressions.throw_(e));
+    builder.add(Expressions.tryCatch(st, cb));
+    builder.add(Expressions.return_(null, u));
+    assertEquals(
+        "{\n"
+            + "  final int u;\n"
+            + "  try {\n"
+            + "    u = 1 + 2;\n"
+            + "  } catch (Exception e) {\n"
+            + "    throw e;\n"
+            + "  }\n"
+            + "  return u;\n"
+            + "}\n",
+        builder.toBlock().toString());
+  }
 }
 
 // End InlinerTest.java