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:37 UTC

[calcite] branch master updated (350802b -> f1309fa)

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

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


    from 350802b  [CALCITE-4510] RexLiteral can produce wrong digest for some user defined types
     add dfb934a  [CALCITE-4593] DiffRepository tests should fail if new XML resources are not in alphabetical order
     new e193b18  DiffRepository should write a test's resource file only when it is modified
     new f1309fa  [CALCITE-4594] Interpreter returns wrong result when Values has zero fields

The 2 revisions listed above as "new" are entirely new to this
repository and will be described in separate emails.  The revisions
listed as "add" were already present in the repository and have only
been added to this reference.


Summary of changes:
 .editorconfig                                      |     3 +
 .../org/apache/calcite/interpreter/ValuesNode.java |     4 +-
 .../main/java/org/apache/calcite/util/Util.java    |     3 +
 .../org/apache/calcite/test/DiffRepository.java    |   174 +-
 .../org/apache/calcite/test/InterpreterTest.java   |    67 +-
 .../java/org/apache/calcite/util/TestUtil.java     |    17 +
 .../java/org/apache/calcite/util/TestUtilTest.java |    39 +
 .../calcite/sql/test/SqlPrettyWriterTest.xml       |   286 +-
 .../org/apache/calcite/test/HepPlannerTest.xml     |   150 +-
 .../org/apache/calcite/test/RelOptRulesTest.xml    | 23458 +++++++++----------
 .../apache/calcite/test/SqlHintsConverterTest.xml  |   342 +-
 .../org/apache/calcite/test/SqlLimitsTest.xml      |    10 +-
 .../apache/calcite/test/SqlToRelConverterTest.xml  | 13440 +++++------
 .../org/apache/calcite/test/TopDownOptTest.xml     |  1866 +-
 .../calcite/test/TypeCoercionConverterTest.xml     |   136 +-
 15 files changed, 20090 insertions(+), 19905 deletions(-)

[calcite] 01/02: DiffRepository should write a test's resource file only when it is modified

Posted by jh...@apache.org.
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 e193b18ac23cde8bdac41e3864219ff1e9d6e0c5
Author: Julian Hyde <jh...@apache.org>
AuthorDate: Thu Apr 29 16:48:34 2021 -0700

    DiffRepository should write a test's resource file only when it is modified
    
    Before this change, DiffRepository writes the resource file
    (to xxx_actual.xml) for every test case, regardless of whether
    its resources (e.g. SQL or plan) have changed. For large tests
    with lots of test cases and large resource files, such as
    RelOptRulesTest, that is a considerable CPU and IO cost.
    
    After this change, DiffRepository only writes the resource
    file if the resources change. So if RelOptRulesTest has 3
    failures out of 500 tests, DiffRepository will write
    RelOptRulesTest_actual.xml 3 times. As a result, such tests
    run ~10x faster.
    
    There are a few test methods in SqlToRelConverter test that
    run 2 or 3 SQL statements that all produce the same plan.
    DiffRepository regards each SQL statement as a resource, and
    marks the resource file 'dirty' even though the final result
    is always the same. We should fix this, but it's only a minor
    impact on performance.
---
 .../main/java/org/apache/calcite/util/Util.java    |  3 +
 .../org/apache/calcite/test/DiffRepository.java    | 67 ++++++++++++++++++----
 2 files changed, 60 insertions(+), 10 deletions(-)

diff --git a/core/src/main/java/org/apache/calcite/util/Util.java b/core/src/main/java/org/apache/calcite/util/Util.java
index cd4fc0f..fe771cb 100644
--- a/core/src/main/java/org/apache/calcite/util/Util.java
+++ b/core/src/main/java/org/apache/calcite/util/Util.java
@@ -967,6 +967,9 @@ public class Util {
       throwable.addSuppressed(new Throwable(message));
       throw (Error) throwable;
     }
+    if (throwable instanceof IOException) {
+      return new UncheckedIOException(message, (IOException) throwable);
+    }
     throw new RuntimeException(message, throwable);
   }
 
diff --git a/core/src/test/java/org/apache/calcite/test/DiffRepository.java b/core/src/test/java/org/apache/calcite/test/DiffRepository.java
index 65c2529..b1c010b 100644
--- a/core/src/test/java/org/apache/calcite/test/DiffRepository.java
+++ b/core/src/test/java/org/apache/calcite/test/DiffRepository.java
@@ -25,6 +25,7 @@ import org.apache.calcite.util.XmlOutput;
 import com.google.common.cache.CacheBuilder;
 import com.google.common.cache.CacheLoader;
 import com.google.common.cache.LoadingCache;
+import com.google.common.collect.ImmutableList;
 import com.google.common.collect.ImmutableSortedSet;
 
 import org.junit.jupiter.api.Assertions;
@@ -179,6 +180,8 @@ public class DiffRepository {
   private final URL refFile;
   private final File logFile;
   private final Filter filter;
+  private int modCount;
+  private int modCountAtLastWrite;
 
   /**
    * Creates a DiffRepository.
@@ -194,11 +197,10 @@ public class DiffRepository {
     this.baseRepository = baseRepository;
     this.filter = filter;
     this.indent = indent;
-    if (refFile == null) {
-      throw new IllegalArgumentException("url must not be null");
-    }
-    this.refFile = refFile;
+    this.refFile = Objects.requireNonNull(refFile, "refFile");
     this.logFile = logFile;
+    this.modCountAtLastWrite = 0;
+    this.modCount = 0;
 
     // Load the document.
     DocumentBuilderFactory fac = DocumentBuilderFactory.newInstance();
@@ -375,6 +377,7 @@ public class DiffRepository {
                 + "not specify 'overrides=true'");
           }
           if (outOfOrderTests.contains(testCaseName)) {
+            ++modCount;
             flushDoc();
             throw new IllegalArgumentException("TestCase '" + testCaseName
                 + "' is out of order in the reference file: "
@@ -471,6 +474,7 @@ public class DiffRepository {
       testCaseElement.setAttribute(TEST_CASE_NAME_ATTR, testCaseName);
       Node refElement = ref(testCaseName, map);
       root.insertBefore(testCaseElement, refElement);
+      ++modCount;
     }
     Element resourceElement =
         getResourceElement(testCaseElement, resourceName, true);
@@ -478,11 +482,20 @@ public class DiffRepository {
       resourceElement = doc.createElement(RESOURCE_TAG);
       resourceElement.setAttribute(RESOURCE_NAME_ATTR, resourceName);
       testCaseElement.appendChild(resourceElement);
+      ++modCount;
+      if (!value.equals("")) {
+        resourceElement.appendChild(doc.createCDATASection(value));
+      }
     } else {
-      removeAllChildren(resourceElement);
-    }
-    if (!value.equals("")) {
-      resourceElement.appendChild(doc.createCDATASection(value));
+      final List<Node> newChildList;
+      if (value.equals("")) {
+        newChildList = ImmutableList.of();
+      } else {
+        newChildList = ImmutableList.of(doc.createCDATASection(value));
+      }
+      if (replaceChildren(resourceElement, newChildList)) {
+        ++modCount;
+      }
     }
 
     // Write out the document.
@@ -525,17 +538,23 @@ public class DiffRepository {
   /**
    * Flushes the reference document to the file system.
    */
-  private void flushDoc() {
+  private synchronized void flushDoc() {
+    if (modCount == modCountAtLastWrite) {
+      // Document has not been modified since last write.
+      return;
+    }
     try {
       boolean b = logFile.getParentFile().mkdirs();
       Util.discard(b);
       try (Writer w = Util.printWriter(logFile)) {
         write(doc, w, indent);
       }
+      System.out.println("write; modCount=" + modCount);
     } catch (IOException e) {
-      throw new RuntimeException("error while writing test reference log '"
+      throw Util.throwAsRuntime("error while writing test reference log '"
           + logFile + "'", e);
     }
+    modCountAtLastWrite = modCount;
   }
 
   /** Validates the root element.
@@ -636,6 +655,34 @@ public class DiffRepository {
     }
   }
 
+  private static boolean replaceChildren(Element element, List<Node> children) {
+    // Current children
+    final NodeList childNodes = element.getChildNodes();
+    final List<Node> list = new ArrayList<>();
+    for (Node item : iterate(childNodes)) {
+      if (item.getNodeType() != Node.TEXT_NODE) {
+        list.add(item);
+      }
+    }
+
+    // Are new children equal to old?
+    if (equalList(children, list)) {
+      return false;
+    }
+
+    // Replace old children with new children
+    removeAllChildren(element);
+    children.forEach(element::appendChild);
+    return true;
+  }
+
+  /** Returns whether two lists of nodes are equal. */
+  private static boolean equalList(List<Node> list0, List<Node> list1) {
+    return list1.size() == list0.size()
+        && Pair.zip(list1, list0).stream()
+        .allMatch(p -> p.left.isEqualNode(p.right));
+  }
+
   /**
    * Serializes an XML document as text.
    *

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

Posted by jh...@apache.org.
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("[]", "[]");
+  }
 }