You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@calcite.apache.org by ho...@apache.org on 2019/05/08 15:37:37 UTC

[calcite] branch master updated: [CALCITE-3046] CompileException when inserting casted value of composited user defined type into table

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

hongze 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 b8bdfb0  [CALCITE-3046] CompileException when inserting casted value of composited user defined type into table
b8bdfb0 is described below

commit b8bdfb074be12f90c98f9f1fa2af8cf46f2d36f1
Author: Hongze Zhang <ho...@tencent.com>
AuthorDate: Sun May 5 17:23:45 2019 +0800

    [CALCITE-3046] CompileException when inserting casted value of composited user defined type into table
    
    Also, add several test cases for EnumerableRelImplementor.TypeFinder.
---
 .../enumerable/EnumerableRelImplementor.java       |  36 +++++-
 .../calcite/adapter/enumerable/TypeFinderTest.java | 132 +++++++++++++++++++++
 .../java/org/apache/calcite/test/CalciteSuite.java |   2 +
 .../java/org/apache/calcite/test/ServerTest.java   |  17 +++
 4 files changed, 184 insertions(+), 3 deletions(-)

diff --git a/core/src/main/java/org/apache/calcite/adapter/enumerable/EnumerableRelImplementor.java b/core/src/main/java/org/apache/calcite/adapter/enumerable/EnumerableRelImplementor.java
index b902926..683f53a 100644
--- a/core/src/main/java/org/apache/calcite/adapter/enumerable/EnumerableRelImplementor.java
+++ b/core/src/main/java/org/apache/calcite/adapter/enumerable/EnumerableRelImplementor.java
@@ -27,7 +27,9 @@ import org.apache.calcite.linq4j.tree.ClassDeclaration;
 import org.apache.calcite.linq4j.tree.ConditionalStatement;
 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.FunctionExpression;
 import org.apache.calcite.linq4j.tree.GotoStatement;
 import org.apache.calcite.linq4j.tree.MemberDeclaration;
 import org.apache.calcite.linq4j.tree.MethodCallExpression;
@@ -37,6 +39,7 @@ 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.Types;
+import org.apache.calcite.linq4j.tree.UnaryExpression;
 import org.apache.calcite.linq4j.tree.VisitorImpl;
 import org.apache.calcite.rex.RexBuilder;
 import org.apache.calcite.runtime.Bindable;
@@ -44,6 +47,7 @@ import org.apache.calcite.sql.validate.SqlConformance;
 import org.apache.calcite.sql.validate.SqlConformanceEnum;
 import org.apache.calcite.util.BuiltInMethod;
 
+import com.google.common.annotations.VisibleForTesting;
 import com.google.common.collect.Collections2;
 import com.google.common.collect.ImmutableList;
 import com.google.common.collect.Iterables;
@@ -464,7 +468,8 @@ public class EnumerableRelImplementor extends JavaRelImplementor {
   }
 
   /** Visitor that finds types in an {@link Expression} tree. */
-  private static class TypeFinder extends VisitorImpl<Void> {
+  @VisibleForTesting
+  static class TypeFinder extends VisitorImpl<Void> {
     private final Collection<Type> types;
 
     TypeFinder(Collection<Type> types) {
@@ -490,11 +495,36 @@ public class EnumerableRelImplementor extends JavaRelImplementor {
     }
 
     @Override public Void visit(ConstantExpression constantExpression) {
-      if (constantExpression.value instanceof Type) {
-        types.add((Type) constantExpression.value);
+      final Object value = constantExpression.value;
+      if (value instanceof Type) {
+        types.add((Type) value);
+      }
+      if (value == null) {
+        // null literal
+        Type type = constantExpression.getType();
+        types.add(type);
       }
       return super.visit(constantExpression);
     }
+
+    @Override public Void visit(FunctionExpression functionExpression) {
+      final List<ParameterExpression> list = functionExpression.parameterList;
+      for (ParameterExpression pe : list) {
+        types.add(pe.getType());
+      }
+      if (functionExpression.body == null) {
+        return super.visit(functionExpression);
+      }
+      types.add(functionExpression.body.getType());
+      return super.visit(functionExpression);
+    }
+
+    @Override public Void visit(UnaryExpression unaryExpression) {
+      if (unaryExpression.nodeType == ExpressionType.Convert) {
+        types.add(unaryExpression.getType());
+      }
+      return super.visit(unaryExpression);
+    }
   }
 
   /** Adds a declaration of each synthetic type found in a code block. */
diff --git a/core/src/test/java/org/apache/calcite/adapter/enumerable/TypeFinderTest.java b/core/src/test/java/org/apache/calcite/adapter/enumerable/TypeFinderTest.java
new file mode 100644
index 0000000..4866eb0
--- /dev/null
+++ b/core/src/test/java/org/apache/calcite/adapter/enumerable/TypeFinderTest.java
@@ -0,0 +1,132 @@
+/*
+ * 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.adapter.enumerable;
+
+import org.apache.calcite.linq4j.function.Function1;
+import org.apache.calcite.linq4j.tree.ConstantExpression;
+import org.apache.calcite.linq4j.tree.Expressions;
+import org.apache.calcite.linq4j.tree.FunctionExpression;
+import org.apache.calcite.linq4j.tree.Node;
+import org.apache.calcite.linq4j.tree.ParameterExpression;
+import org.apache.calcite.linq4j.tree.UnaryExpression;
+
+import org.hamcrest.BaseMatcher;
+import org.hamcrest.CoreMatchers;
+import org.hamcrest.Description;
+import org.junit.Assert;
+import org.junit.Test;
+
+import java.lang.reflect.Type;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Objects;
+import java.util.Set;
+
+/**
+ * Test for {@link org.apache.calcite.adapter.enumerable.EnumerableRelImplementor.TypeFinder}
+ */
+public class TypeFinderTest {
+
+  @Test public void testConstantExpression() {
+    ConstantExpression expr = Expressions.constant(null, Integer.class);
+    assertJavaCodeContains("(Integer) null\n", expr);
+    assertTypeContains(Integer.class, expr);
+  }
+
+  @Test public void testConvertExpression() {
+    UnaryExpression expr = Expressions.convert_(Expressions.new_(String.class), Object.class);
+    assertJavaCodeContains("(Object) new String()\n", expr);
+    assertTypeContains(Arrays.asList(String.class, Object.class), expr);
+  }
+
+  @Test public void testFunctionExpression1() {
+    ParameterExpression param = Expressions.parameter(String.class, "input");
+    FunctionExpression expr = Expressions.lambda(Function1.class,
+        Expressions.block(
+            Expressions.return_(null, param)),
+        param);
+    assertJavaCodeContains("new org.apache.calcite.linq4j.function.Function1() {\n"
+        + "  public String apply(String input) {\n"
+        + "    return input;\n"
+        + "  }\n"
+        + "  public Object apply(Object input) {\n"
+        + "    return apply(\n"
+        + "      (String) input);\n"
+        + "  }\n"
+        + "}\n", expr);
+    assertTypeContains(String.class, expr);
+  }
+
+  @Test public void testFunctionExpression2() {
+    FunctionExpression expr = Expressions.lambda(Function1.class,
+        Expressions.block(
+            Expressions.return_(null, Expressions.constant(1L, Long.class))),
+        Expressions.parameter(String.class, "input"));
+    assertJavaCodeContains("new org.apache.calcite.linq4j.function.Function1() {\n"
+        + "  public Long apply(String input) {\n"
+        + "    return Long.valueOf(1L);\n"
+        + "  }\n"
+        + "  public Object apply(Object input) {\n"
+        + "    return apply(\n"
+        + "      (String) input);\n"
+        + "  }\n"
+        + "}\n", expr);
+    assertTypeContains(Arrays.asList(String.class, Long.class), expr);
+  }
+  private void assertJavaCodeContains(String expected, Node node) {
+    assertJavaCodeContains(expected, Collections.singletonList(node));
+  }
+
+  private void assertJavaCodeContains(String expected, List<Node> nodes) {
+    final String javaCode = Expressions.toString(nodes, "\n", false);
+    Assert.assertThat(javaCode, CoreMatchers.containsString(expected));
+  }
+
+  private void assertTypeContains(Type expectedType, Node node) {
+    assertTypeContains(Collections.singletonList(expectedType),
+        Collections.singletonList(node));
+  }
+
+  private void assertTypeContains(List<Type> expectedType, Node node) {
+    assertTypeContains(expectedType,
+        Collections.singletonList(node));
+  }
+
+  private void assertTypeContains(List<Type> expectedTypes, List<Node> nodes) {
+    final HashSet<Type> types = new HashSet<>();
+    final EnumerableRelImplementor.TypeFinder typeFinder
+        = new EnumerableRelImplementor.TypeFinder(types);
+    for (Node node : nodes) {
+      node.accept(typeFinder);
+    }
+    Assert.assertThat(types, new BaseMatcher<HashSet<Type>>() {
+      @Override public boolean matches(Object o) {
+        final Set<Type> actual = (HashSet<Type>) o;
+        return actual.containsAll(expectedTypes);
+      }
+
+      @Override public void describeTo(Description description) {
+        description.appendText("Expected a set of types containing all of: ")
+            .appendText(Objects.toString(expectedTypes));
+      }
+    });
+  }
+}
+
+// End TypeFinderTest.java
diff --git a/core/src/test/java/org/apache/calcite/test/CalciteSuite.java b/core/src/test/java/org/apache/calcite/test/CalciteSuite.java
index 7c579ca..bced1bf 100644
--- a/core/src/test/java/org/apache/calcite/test/CalciteSuite.java
+++ b/core/src/test/java/org/apache/calcite/test/CalciteSuite.java
@@ -18,6 +18,7 @@ package org.apache.calcite.test;
 
 import org.apache.calcite.TestKtTest;
 import org.apache.calcite.adapter.clone.ArrayTableTest;
+import org.apache.calcite.adapter.enumerable.TypeFinderTest;
 import org.apache.calcite.jdbc.CalciteRemoteDriverTest;
 import org.apache.calcite.materialize.LatticeSuggesterTest;
 import org.apache.calcite.plan.RelOptPlanReaderTest;
@@ -131,6 +132,7 @@ import org.junit.runners.Suite;
     SqlTypeFactoryTest.class,
     SqlTypeUtilTest.class,
     SqlValidatorUtilTest.class,
+    TypeFinderTest.class,
 
     // medium tests (above 0.1s)
     SqlParserTest.class,
diff --git a/server/src/test/java/org/apache/calcite/test/ServerTest.java b/server/src/test/java/org/apache/calcite/test/ServerTest.java
index 2d8b241..3e4a71b 100644
--- a/server/src/test/java/org/apache/calcite/test/ServerTest.java
+++ b/server/src/test/java/org/apache/calcite/test/ServerTest.java
@@ -125,6 +125,10 @@ public class ServerTest {
     }
   }
 
+  /** Test case for
+   * <a href="https://issues.apache.org/jira/browse/CALCITE-3046">[CALCITE-3046]
+   * CompileException when inserting casted value of composited user defined type
+   * into table</a>. */
   @Test public void testCreateTable() throws Exception {
     try (Connection c = connect();
          Statement s = c.createStatement()) {
@@ -150,6 +154,19 @@ public class ServerTest {
     }
   }
 
+  @Test public void testInsertCastedValueOfCompositeUdt() throws Exception {
+    try (Connection c = connect();
+         Statement s = c.createStatement()) {
+      boolean b = s.execute("create type mytype as (i int, j int)");
+      assertThat(b, is(false));
+      b = s.execute("create table w (i int not null, j mytype)");
+      assertThat(b, is(false));
+      int x = s.executeUpdate("insert into w "
+          + "values (1, cast((select j from w limit 1) as mytype))");
+      assertThat(x, is(1));
+    }
+  }
+
   @Test public void testStoredGeneratedColumn() throws Exception {
     try (Connection c = connect();
          Statement s = c.createStatement()) {