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 2016/06/13 19:10:06 UTC

calcite git commit: [CALCITE-1284] Move Quidem tests from JdbcTest into their own class

Repository: calcite
Updated Branches:
  refs/heads/master 660d083fe -> a2d7165f3


[CALCITE-1284] Move Quidem tests from JdbcTest into their own class


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

Branch: refs/heads/master
Commit: a2d7165f360e450889ba8b8d051810ddb45e3b62
Parents: 660d083
Author: Julian Hyde <jh...@apache.org>
Authored: Fri Jun 10 14:46:13 2016 -0700
Committer: Julian Hyde <jh...@apache.org>
Committed: Mon Jun 13 00:29:21 2016 -0700

----------------------------------------------------------------------
 .../org/apache/calcite/test/CalciteSuite.java   |   1 +
 .../java/org/apache/calcite/test/JdbcTest.java  | 211 +-------------
 .../org/apache/calcite/test/QuidemTest.java     | 287 +++++++++++++++++++
 3 files changed, 293 insertions(+), 206 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/calcite/blob/a2d7165f/core/src/test/java/org/apache/calcite/test/CalciteSuite.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 0132016..3dfb659 100644
--- a/core/src/test/java/org/apache/calcite/test/CalciteSuite.java
+++ b/core/src/test/java/org/apache/calcite/test/CalciteSuite.java
@@ -134,6 +134,7 @@ import org.junit.runners.Suite;
     LatticeTest.class,
     ReflectiveSchemaTest.class,
     JdbcTest.class,
+    QuidemTest.class,
     CalciteRemoteDriverTest.class,
     StreamTest.class,
 

http://git-wip-us.apache.org/repos/asf/calcite/blob/a2d7165f/core/src/test/java/org/apache/calcite/test/JdbcTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/calcite/test/JdbcTest.java b/core/src/test/java/org/apache/calcite/test/JdbcTest.java
index a200ebb..e8aec78 100644
--- a/core/src/test/java/org/apache/calcite/test/JdbcTest.java
+++ b/core/src/test/java/org/apache/calcite/test/JdbcTest.java
@@ -87,7 +87,6 @@ import org.apache.calcite.sql.parser.SqlParserImplFactory;
 import org.apache.calcite.sql.parser.SqlParserPos;
 import org.apache.calcite.sql.parser.SqlParserUtil;
 import org.apache.calcite.sql.parser.impl.SqlParserImpl;
-import org.apache.calcite.sql.type.SqlTypeName;
 import org.apache.calcite.util.Bug;
 import org.apache.calcite.util.JsonBuilder;
 import org.apache.calcite.util.Pair;
@@ -100,21 +99,14 @@ import com.google.common.base.Throwables;
 import com.google.common.collect.ImmutableList;
 import com.google.common.collect.ImmutableMap;
 
-import net.hydromatic.quidem.Quidem;
-
 import org.hsqldb.jdbcDriver;
 
 import org.junit.Ignore;
 import org.junit.Test;
 
-import java.io.BufferedReader;
-import java.io.File;
-import java.io.FileReader;
-import java.io.FileWriter;
 import java.io.IOException;
 import java.io.Reader;
 import java.math.BigDecimal;
-import java.net.URL;
 import java.sql.Array;
 import java.sql.Connection;
 import java.sql.DatabaseMetaData;
@@ -4411,12 +4403,11 @@ public class JdbcTest {
             "U=b; V=c");
   }
 
-  /** Tests CALCITE-980: different flavors of boolean logic */
-  @Test public void testBooleansInWhere() throws Exception {
-    checkRun("sql/conditions.iq");
-  }
-
-  /** Tests CALCITE-980: different flavors of boolean logic */
+  /** Test case for
+   * <a href="https://issues.apache.org/jira/browse/CALCITE-980">[CALCITE-980]
+   * different flavors of boolean logic</a>.
+   *
+   * @see QuidemTest sql/conditions.iq */
   @Ignore("Fails with org.codehaus.commons.compiler.CompileException: Line 16, Column 112:"
       + " Cannot compare types \"int\" and \"java.lang.String\"\n")
   @Test public void testComparingIntAndString() throws Exception {
@@ -4815,198 +4806,6 @@ public class JdbcTest {
             + sql);
   }
 
-  /** Runs the dummy script, which is checked in empty but which you may
-   * use as scratch space during development. */
-  // Do not add '@Ignore'; just remember not to commit changes to dummy.iq
-  @Test public void testRunDummy() throws Exception {
-    try (final TryThreadLocal.Memo ignored = Prepare.THREAD_EXPAND.push(true)) {
-      checkRun("sql/dummy.iq");
-    }
-  }
-
-  @Test public void testRunAgg() throws Exception {
-    checkRun("sql/agg.iq");
-  }
-
-  @Test public void testRunJoin() throws Exception {
-    checkRun("sql/join.iq");
-  }
-
-  @Test public void testRunOuter() throws Exception {
-    checkRun("sql/outer.iq");
-  }
-
-  @Test public void testRunWinAgg() throws Exception {
-    checkRun("sql/winagg.iq");
-  }
-
-  @Test public void testRunMisc() throws Exception {
-    switch (CalciteAssert.DB) {
-    case ORACLE:
-      // There are formatting differences (e.g. "4.000" vs "4") when using
-      // Oracle as the JDBC data source.
-      return;
-    }
-    try (final TryThreadLocal.Memo ignored = Prepare.THREAD_EXPAND.push(true)) {
-      checkRun("sql/misc.iq");
-    }
-  }
-
-  @Test public void testRunSequence() throws Exception {
-    checkRun("sql/sequence.iq");
-  }
-
-  @Test public void testRunSort() throws Exception {
-    checkRun("sql/sort.iq");
-  }
-
-  @Test public void testRunScalar() throws Exception {
-    try (final TryThreadLocal.Memo ignored = Prepare.THREAD_EXPAND.push(true)) {
-      checkRun("sql/scalar.iq");
-    }
-  }
-
-  @Test public void testRunSubquery() throws Exception {
-    checkRun("sql/subquery.iq");
-  }
-
-  private void checkRun(String path) throws Exception {
-    final File inFile;
-    final File outFile;
-    if (path.startsWith("/")) {
-      // e.g. path = "/tmp/foo.iq"
-      inFile = new File(path);
-      outFile = new File(path + ".out");
-    } else {
-      // e.g. path = "sql/outer.iq"
-      // inUrl = "file:/home/fred/calcite/core/target/test-classes/sql/outer.iq"
-      final URL inUrl = JdbcTest.class.getResource("/" + path);
-      String x = inUrl.getFile();
-      assert x.endsWith(path);
-      x = x.substring(0, x.length() - path.length());
-      assert x.endsWith("/test-classes/");
-      x = x.substring(0, x.length() - "/test-classes/".length());
-      final File base = new File(x);
-      inFile = new File(base, "/test-classes/" + path);
-      outFile = new File(base, "/surefire/" + path);
-    }
-    outFile.getParentFile().mkdirs();
-    final FileReader fileReader = new FileReader(inFile);
-    final BufferedReader bufferedReader = new BufferedReader(fileReader);
-    final FileWriter writer = new FileWriter(outFile);
-    final Function<String, Object> env =
-        new Function<String, Object>() {
-          public Object apply(String varName) {
-            switch (varName) {
-            case "jdk18":
-              return System.getProperty("java.version").startsWith("1.8");
-            case "fixed":
-              return new Function<String, Object>() {
-                public Object apply(String v) {
-                  switch (v) {
-                  case "calcite1045":
-                    return Bug.CALCITE_1045_FIXED;
-                  case "calcite1048":
-                    return Bug.CALCITE_1048_FIXED;
-                  }
-                  return null;
-                }
-              };
-            default:
-              return null;
-            }
-          }
-        };
-    final Quidem.NewConnectionFactory connectionFactory =
-        new Quidem.NewConnectionFactory() {
-          public Connection connect(String name) throws Exception {
-            return connect(name, false);
-          }
-
-          public Connection connect(String name, boolean reference)
-              throws Exception {
-            if (reference) {
-              if (name.equals("foodmart")) {
-                final ConnectionSpec db =
-                    CalciteAssert.DatabaseInstance.HSQLDB.foodmart;
-                final Connection connection = DriverManager.getConnection(db.url,
-                    db.username,
-                    db.password);
-                connection.setSchema("foodmart");
-                return connection;
-              }
-              return null;
-            }
-            if (name.equals("hr")) {
-              return CalciteAssert.hr()
-                  .connect();
-            }
-            if (name.equals("foodmart")) {
-              return CalciteAssert.that()
-                  .with(CalciteAssert.Config.FOODMART_CLONE)
-                  .connect();
-            }
-            if (name.equals("scott")) {
-              return CalciteAssert.that()
-                  .with(CalciteAssert.Config.SCOTT)
-                  .connect();
-            }
-            if (name.equals("jdbc_scott")) {
-              return CalciteAssert.that()
-                  .with(CalciteAssert.Config.JDBC_SCOTT)
-                  .connect();
-            }
-            if (name.equals("post")) {
-              return CalciteAssert.that()
-                  .with(CalciteAssert.Config.REGULAR)
-                  .with(CalciteAssert.SchemaSpec.POST)
-                  .withDefaultSchema("POST")
-                  .connect();
-            }
-            if (name.equals("catchall")) {
-              return CalciteAssert.that()
-                  .withSchema("s",
-                      new ReflectiveSchema(
-                          new ReflectiveSchemaTest.CatchallSchema()))
-                  .connect();
-            }
-            if (name.equals("orinoco")) {
-              return CalciteAssert.that()
-                  .with(CalciteAssert.SchemaSpec.ORINOCO)
-                  .withDefaultSchema("ORINOCO")
-                  .connect();
-            }
-            if (name.equals("seq")) {
-              final Connection connection = CalciteAssert.that()
-                  .withSchema("s", new AbstractSchema())
-                  .connect();
-              connection.unwrap(CalciteConnection.class).getRootSchema()
-                  .getSubSchema("s")
-                  .add("my_seq",
-                      new AbstractTable() {
-                        public RelDataType getRowType(
-                            RelDataTypeFactory typeFactory) {
-                          return typeFactory.builder()
-                              .add("$seq", SqlTypeName.BIGINT).build();
-                        }
-
-                        @Override public Schema.TableType getJdbcTableType() {
-                          return Schema.TableType.SEQUENCE;
-                        }
-                      });
-              return connection;
-            }
-            throw new RuntimeException("unknown connection '" + name + "'");
-          }
-        };
-    new Quidem(bufferedReader, writer, env, connectionFactory).execute();
-    final String diff = DiffTestCase.diff(inFile, outFile);
-    if (!diff.isEmpty()) {
-      fail("Files differ: " + outFile + " " + inFile + "\n"
-          + diff);
-    }
-  }
-
   @Test public void testScalarSubQueryUncorrelated() {
     CalciteAssert.hr()
         .query("select \"empid\", \"deptno\",\n"

http://git-wip-us.apache.org/repos/asf/calcite/blob/a2d7165f/core/src/test/java/org/apache/calcite/test/QuidemTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/calcite/test/QuidemTest.java b/core/src/test/java/org/apache/calcite/test/QuidemTest.java
new file mode 100644
index 0000000..d0d1960
--- /dev/null
+++ b/core/src/test/java/org/apache/calcite/test/QuidemTest.java
@@ -0,0 +1,287 @@
+/*
+ * 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.test;
+
+import org.apache.calcite.adapter.java.ReflectiveSchema;
+import org.apache.calcite.avatica.AvaticaUtils;
+import org.apache.calcite.jdbc.CalciteConnection;
+import org.apache.calcite.prepare.Prepare;
+import org.apache.calcite.rel.type.RelDataType;
+import org.apache.calcite.rel.type.RelDataTypeFactory;
+import org.apache.calcite.schema.Schema;
+import org.apache.calcite.schema.impl.AbstractSchema;
+import org.apache.calcite.schema.impl.AbstractTable;
+import org.apache.calcite.sql.type.SqlTypeName;
+import org.apache.calcite.util.Bug;
+import org.apache.calcite.util.TryThreadLocal;
+import org.apache.calcite.util.Util;
+
+import com.google.common.base.Function;
+import com.google.common.collect.Lists;
+import com.google.common.io.PatternFilenameFilter;
+
+import net.hydromatic.quidem.Quidem;
+
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
+
+import java.io.BufferedReader;
+import java.io.File;
+import java.io.FileReader;
+import java.io.FileWriter;
+import java.lang.reflect.Method;
+import java.net.URL;
+import java.sql.Connection;
+import java.sql.DriverManager;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.List;
+
+import static org.junit.Assert.fail;
+
+/**
+ * Test that runs every Quidem file as a test.
+ */
+@RunWith(Parameterized.class)
+public class QuidemTest {
+  private final String path;
+  private final Method method;
+
+  public QuidemTest(String path) {
+    this.path = path;
+    this.method = findMethod(path);
+  }
+
+  private Method findMethod(String path) {
+    // E.g. path "sql/agg.iq" gives method "testSqlAgg"
+    String methodName =
+        AvaticaUtils.toCamelCase("test_" + path.replace('/', '_').replaceAll("\\.iq$", ""));
+    Method m;
+    try {
+      m = getClass().getMethod(methodName);
+    } catch (NoSuchMethodException e) {
+      m = null;
+    }
+    return m;
+  }
+
+  /** For {@link org.junit.runners.Parameterized} runner. */
+  @Parameterized.Parameters(name = "{index}: quidem({0})")
+  public static Collection<Object[]> data() {
+    // Start with a test file we know exists, then find the directory and list
+    // its files.
+    final String first = "sql/agg.iq";
+    // inUrl = "file:/home/fred/calcite/core/target/test-classes/sql/agg.iq"
+    final URL inUrl = JdbcTest.class.getResource("/" + first);
+    String x = inUrl.getFile();
+    assert x.endsWith(first);
+    final String base = x.substring(0, x.length() - first.length());
+    final File firstFile = new File(x);
+    final File dir = firstFile.getParentFile();
+    final List<String> paths = new ArrayList<>();
+    for (File f : dir.listFiles(new PatternFilenameFilter(".*\\.iq$"))) {
+      assert f.getAbsolutePath().startsWith(base);
+      paths.add(f.getAbsolutePath().substring(base.length()));
+    }
+    return Lists.transform(paths, new Function<String, Object[]>() {
+      public Object[] apply(String path) {
+        return new Object[] {path};
+      }
+    });
+  }
+
+  private void checkRun(String path) throws Exception {
+    final File inFile;
+    final File outFile;
+    if (path.startsWith("/")) {
+      // e.g. path = "/tmp/foo.iq"
+      inFile = new File(path);
+      outFile = new File(path + ".out");
+    } else {
+      // e.g. path = "sql/outer.iq"
+      // inUrl = "file:/home/fred/calcite/core/target/test-classes/sql/outer.iq"
+      final URL inUrl = JdbcTest.class.getResource("/" + path);
+      String x = inUrl.getFile();
+      assert x.endsWith(path);
+      x = x.substring(0, x.length() - path.length());
+      assert x.endsWith("/test-classes/");
+      x = x.substring(0, x.length() - "/test-classes/".length());
+      final File base = new File(x);
+      inFile = new File(base, "/test-classes/" + path);
+      outFile = new File(base, "/surefire/" + path);
+    }
+    Util.discard(outFile.getParentFile().mkdirs());
+    final FileReader fileReader = new FileReader(inFile);
+    final BufferedReader bufferedReader = new BufferedReader(fileReader);
+    final FileWriter writer = new FileWriter(outFile);
+    new Quidem(bufferedReader, writer, env(), new QuidemConnectionFactory())
+        .execute();
+    final String diff = DiffTestCase.diff(inFile, outFile);
+    if (!diff.isEmpty()) {
+      fail("Files differ: " + outFile + " " + inFile + "\n"
+          + diff);
+    }
+  }
+
+  private Function<String, Object> env() {
+    return new Function<String, Object>() {
+      public Object apply(String varName) {
+        switch (varName) {
+        case "jdk18":
+          return System.getProperty("java.version").startsWith("1.8");
+        case "fixed":
+          return new Function<String, Object>() {
+            public Object apply(String v) {
+              switch (v) {
+              case "calcite1045":
+                return Bug.CALCITE_1045_FIXED;
+              case "calcite1048":
+                return Bug.CALCITE_1048_FIXED;
+              }
+              return null;
+            }
+          };
+        default:
+          return null;
+        }
+      }
+    };
+  }
+
+  @Test public void test() throws Exception {
+    if (method != null) {
+      method.invoke(this);
+    } else {
+      checkRun(path);
+    }
+  }
+
+  /** Override settings for "sql/misc.iq". */
+  public void testSqlMisc() throws Exception {
+    switch (CalciteAssert.DB) {
+    case ORACLE:
+      // There are formatting differences (e.g. "4.000" vs "4") when using
+      // Oracle as the JDBC data source.
+      return;
+    }
+    try (final TryThreadLocal.Memo ignored = Prepare.THREAD_EXPAND.push(true)) {
+      checkRun(path);
+    }
+  }
+
+  /** Override settings for "sql/scalar.iq". */
+  public void testSqlScalar() throws Exception {
+    try (final TryThreadLocal.Memo ignored = Prepare.THREAD_EXPAND.push(true)) {
+      checkRun(path);
+    }
+  }
+
+  /** Runs the dummy script "sql/dummy.iq", which is checked in empty but
+   * which you may use as scratch space during development. */
+  // Do not add disable this test; just remember not to commit changes to dummy.iq
+  public void testSqlDummy() throws Exception {
+    try (final TryThreadLocal.Memo ignored = Prepare.THREAD_EXPAND.push(true)) {
+      checkRun(path);
+    }
+  }
+
+  /** Quidem connection factory for Calcite's built-in test schemas. */
+  private static class QuidemConnectionFactory implements Quidem.NewConnectionFactory {
+    public Connection connect(String name) throws Exception {
+      return connect(name, false);
+    }
+
+    public Connection connect(String name, boolean reference)
+        throws Exception {
+      if (reference) {
+        if (name.equals("foodmart")) {
+          final ConnectionSpec db =
+              CalciteAssert.DatabaseInstance.HSQLDB.foodmart;
+          final Connection connection =
+              DriverManager.getConnection(db.url, db.username,
+                  db.password);
+          connection.setSchema("foodmart");
+          return connection;
+        }
+        return null;
+      }
+      if (name.equals("hr")) {
+        return CalciteAssert.hr()
+            .connect();
+      }
+      if (name.equals("foodmart")) {
+        return CalciteAssert.that()
+            .with(CalciteAssert.Config.FOODMART_CLONE)
+            .connect();
+      }
+      if (name.equals("scott")) {
+        return CalciteAssert.that()
+            .with(CalciteAssert.Config.SCOTT)
+            .connect();
+      }
+      if (name.equals("jdbc_scott")) {
+        return CalciteAssert.that()
+            .with(CalciteAssert.Config.JDBC_SCOTT)
+            .connect();
+      }
+      if (name.equals("post")) {
+        return CalciteAssert.that()
+            .with(CalciteAssert.Config.REGULAR)
+            .with(CalciteAssert.SchemaSpec.POST)
+            .withDefaultSchema("POST")
+            .connect();
+      }
+      if (name.equals("catchall")) {
+        return CalciteAssert.that()
+            .withSchema("s",
+                new ReflectiveSchema(
+                    new ReflectiveSchemaTest.CatchallSchema()))
+            .connect();
+      }
+      if (name.equals("orinoco")) {
+        return CalciteAssert.that()
+            .with(CalciteAssert.SchemaSpec.ORINOCO)
+            .withDefaultSchema("ORINOCO")
+            .connect();
+      }
+      if (name.equals("seq")) {
+        final Connection connection = CalciteAssert.that()
+            .withSchema("s", new AbstractSchema())
+            .connect();
+        connection.unwrap(CalciteConnection.class).getRootSchema()
+            .getSubSchema("s")
+            .add("my_seq",
+                new AbstractTable() {
+                  public RelDataType getRowType(
+                      RelDataTypeFactory typeFactory) {
+                    return typeFactory.builder()
+                        .add("$seq", SqlTypeName.BIGINT).build();
+                  }
+
+                  @Override public Schema.TableType getJdbcTableType() {
+                    return Schema.TableType.SEQUENCE;
+                  }
+                });
+        return connection;
+      }
+      throw new RuntimeException("unknown connection '" + name + "'");
+    }
+  }
+}
+
+// End QuidemTest.java