You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@calcite.apache.org by jh...@apache.org on 2021/05/06 00:03:39 UTC

[calcite] 02/02: [CALCITE-4594] Interpreter returns wrong result when Values has zero fields

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

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

commit f1309fa3af6826e73377e54081160228eb7ab951
Author: Julian Hyde <jh...@apache.org>
AuthorDate: Mon May 3 19:18:33 2021 -0700

    [CALCITE-4594] Interpreter returns wrong result when Values has zero fields
    
    Currently, if a Values RelNode has a zero-field row type and
    N rows, the Interpreter returns zero rows; it should, of
    course, return N rows.
    
    As a project, we have not decided whether to allow zero-field
    row types. Unless and until we ban them (by throwing whenever
    we see them), the Interpreter should do its best when it sees
    them, not return wrong results.
---
 .../org/apache/calcite/interpreter/ValuesNode.java |  4 +-
 .../org/apache/calcite/test/InterpreterTest.java   | 67 +++++++++++++++++-----
 2 files changed, 55 insertions(+), 16 deletions(-)

diff --git a/core/src/main/java/org/apache/calcite/interpreter/ValuesNode.java b/core/src/main/java/org/apache/calcite/interpreter/ValuesNode.java
index 6800f94..b1a8ac4 100644
--- a/core/src/main/java/org/apache/calcite/interpreter/ValuesNode.java
+++ b/core/src/main/java/org/apache/calcite/interpreter/ValuesNode.java
@@ -53,8 +53,8 @@ public class ValuesNode implements Node {
     scalar.execute(context, values);
     final ImmutableList.Builder<Row> rows = ImmutableList.builder();
     Object[] subValues = new Object[fieldCount];
-    for (int i = 0; i < values.length; i += fieldCount) {
-      System.arraycopy(values, i, subValues, 0, fieldCount);
+    for (int r = 0, n = tuples.size(); r < n; ++r) {
+      System.arraycopy(values, r * fieldCount, subValues, 0, fieldCount);
       rows.add(Row.asCopy(subValues));
     }
     return rows.build();
diff --git a/core/src/test/java/org/apache/calcite/test/InterpreterTest.java b/core/src/test/java/org/apache/calcite/test/InterpreterTest.java
index 16d590e..b4a52d7 100644
--- a/core/src/test/java/org/apache/calcite/test/InterpreterTest.java
+++ b/core/src/test/java/org/apache/calcite/test/InterpreterTest.java
@@ -30,6 +30,7 @@ import org.apache.calcite.rel.RelRoot;
 import org.apache.calcite.rel.rules.CoreRules;
 import org.apache.calcite.rel.type.RelDataType;
 import org.apache.calcite.rel.type.RelDataTypeField;
+import org.apache.calcite.rex.RexLiteral;
 import org.apache.calcite.schema.ScalarFunction;
 import org.apache.calcite.schema.SchemaPlus;
 import org.apache.calcite.schema.TableFunction;
@@ -42,11 +43,14 @@ import org.apache.calcite.sql.parser.SqlParser;
 import org.apache.calcite.tools.FrameworkConfig;
 import org.apache.calcite.tools.Frameworks;
 import org.apache.calcite.tools.Planner;
+import org.apache.calcite.tools.RelBuilder;
 import org.apache.calcite.tools.RelConversionException;
 import org.apache.calcite.tools.ValidationException;
 import org.apache.calcite.util.Smalls;
 import org.apache.calcite.util.Util;
 
+import com.google.common.collect.ImmutableList;
+
 import org.checkerframework.checker.nullness.qual.Nullable;
 import org.junit.jupiter.api.AfterEach;
 import org.junit.jupiter.api.BeforeEach;
@@ -73,12 +77,12 @@ class InterpreterTest {
   /** Implementation of {@link DataContext} for executing queries without a
    * connection. */
   private static class MyDataContext implements DataContext {
-    private SchemaPlus rootSchema;
-    private final Planner planner;
+    private final SchemaPlus rootSchema;
+    private final JavaTypeFactory typeFactory;
 
-    MyDataContext(SchemaPlus rootSchema, Planner planner) {
+    MyDataContext(SchemaPlus rootSchema, RelNode rel) {
       this.rootSchema = rootSchema;
-      this.planner = planner;
+      this.typeFactory = (JavaTypeFactory) rel.getCluster().getTypeFactory();
     }
 
     public SchemaPlus getRootSchema() {
@@ -86,7 +90,7 @@ class InterpreterTest {
     }
 
     public JavaTypeFactory getTypeFactory() {
-      return (JavaTypeFactory) planner.getTypeFactory();
+      return typeFactory;
     }
 
     public @Nullable QueryProvider getQueryProvider() {
@@ -103,16 +107,23 @@ class InterpreterTest {
     private final String sql;
     private final SchemaPlus rootSchema;
     private final boolean project;
+    private final Function<RelBuilder, RelNode> relFn;
 
-    Sql(String sql, SchemaPlus rootSchema, boolean project) {
+    Sql(String sql, SchemaPlus rootSchema, boolean project,
+        @Nullable Function<RelBuilder, RelNode> relFn) {
       this.sql = sql;
       this.rootSchema = rootSchema;
       this.project = project;
+      this.relFn = relFn;
     }
 
     @SuppressWarnings("SameParameterValue")
     Sql withProject(boolean project) {
-      return new Sql(sql, rootSchema, project);
+      return new Sql(sql, rootSchema, project, relFn);
+    }
+
+    Sql withRel(Function<RelBuilder, RelNode> relFn) {
+      return new Sql(sql, rootSchema, project, relFn);
     }
 
     /** Interprets the sql and checks result with specified rows, ordered. */
@@ -138,14 +149,30 @@ class InterpreterTest {
       return Frameworks.getPlanner(config);
     }
 
+    /** Performs an action that requires a {@link RelBuilder}, and returns the
+     * result. */
+    private <T> T withRelBuilder(Function<RelBuilder, T> fn) {
+      final FrameworkConfig config = Frameworks.newConfigBuilder()
+          .defaultSchema(rootSchema)
+          .build();
+      final RelBuilder relBuilder = RelBuilder.create(config);
+      return fn.apply(relBuilder);
+    }
+
     /** Interprets the sql and checks result with specified rows. */
     private Sql returnsRows(boolean unordered, String[] rows) {
       try (Planner planner = createPlanner()) {
-        SqlNode parse = planner.parse(sql);
-        SqlNode validate = planner.validate(parse);
-        final RelRoot root = planner.rel(validate);
-        RelNode convert = project ? root.project() : root.rel;
-        final MyDataContext dataContext = new MyDataContext(rootSchema, planner);
+        final RelNode convert;
+        if (relFn != null) {
+          convert = withRelBuilder(relFn);
+        } else {
+          SqlNode parse = planner.parse(sql);
+          SqlNode validate = planner.validate(parse);
+          final RelRoot root = planner.rel(validate);
+          convert = project ? root.project() : root.rel;
+        }
+        final MyDataContext dataContext =
+            new MyDataContext(rootSchema, convert);
         assertInterpret(convert, dataContext, unordered, rows);
         return this;
       } catch (ValidationException
@@ -158,7 +185,7 @@ class InterpreterTest {
 
   /** Creates a {@link Sql}. */
   private Sql sql(String sql) {
-    return new Sql(sql, rootSchema, false);
+    return new Sql(sql, rootSchema, false, null);
   }
 
   private void reset() {
@@ -476,7 +503,8 @@ class InterpreterTest {
       final HepPlanner hepPlanner = new HepPlanner(program);
       hepPlanner.setRoot(convert);
       final RelNode relNode = hepPlanner.findBestExp();
-      final MyDataContext dataContext = new MyDataContext(rootSchema, planner);
+      final MyDataContext dataContext =
+          new MyDataContext(rootSchema, relNode);
       assertInterpret(relNode, dataContext, true, "[1, a]", "[3, c]");
     } catch (ValidationException
         | SqlParseException
@@ -674,4 +702,15 @@ class InterpreterTest {
         + "from table(\"s\".\"t\"('=100='))";
     sql(sql).returnsRows("[1]", "[3]", "[100]");
   }
+
+  /** Tests projecting zero fields. */
+  @Test void testZeroFields() {
+    final List<RexLiteral> row = ImmutableList.of();
+    sql("?")
+        .withRel(b ->
+            b.values(ImmutableList.of(row, row),
+                b.getTypeFactory().builder().build())
+                .build())
+        .returnsRows("[]", "[]");
+  }
 }