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

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

commit 996692e36a0d1bdff2b3fd1e08fa99fcc755f2ab
Author: Mihai Budiu <mbu...@vmware.com>
AuthorDate: Tue Apr 11 16:44:36 2023 -0700

    [CALCITE-5615] Run SQL Logic Test suite using Calcite's HSQLDB JDBC adapter
    
    Signed-off-by: Mihai Budiu <mbu...@vmware.com>
    
    Close apache/calcite#3145
---
 bom/build.gradle.kts                               |   1 +
 gradle.properties                                  |   1 +
 plus/build.gradle.kts                              |   1 +
 .../java/org/apache/calcite/slt/SqlLogicTests.java | 314 +++++++++++
 .../calcite/slt/executors/CalciteExecutor.java     |  94 ++++
 .../calcite/slt/executors/package-info.java}       |  22 +-
 .../java/org/apache/calcite/slt/package-info.java  | 105 ++++
 plus/src/test/resources/slttestfailures.txt        | 615 +++++++++++++++++++++
 8 files changed, 1135 insertions(+), 18 deletions(-)

diff --git a/bom/build.gradle.kts b/bom/build.gradle.kts
index e4b3a82aa2..dd68f3036b 100644
--- a/bom/build.gradle.kts
+++ b/bom/build.gradle.kts
@@ -90,6 +90,7 @@ dependencies {
         apiv("net.hydromatic:foodmart-queries")
         apiv("net.hydromatic:quidem")
         apiv("net.hydromatic:scott-data-hsqldb")
+        apiv("net.hydromatic:sql-logic-test")
         apiv("net.hydromatic:tpcds", "hydromatic.tpcds")
         apiv("net.java.dev.jna:jna")
         apiv("net.sf.opencsv:opencsv")
diff --git a/gradle.properties b/gradle.properties
index e99adc707c..e59712d8c8 100644
--- a/gradle.properties
+++ b/gradle.properties
@@ -159,6 +159,7 @@ sketches-core.version=0.9.0
 slf4j.version=1.7.25
 spark.version=2.2.2
 sqlline.version=1.12.0
+sql-logic-test.version=0.3
 teradata.tpcds.version=1.2
 testcontainers.version=1.17.3
 tpch.version=1.0
diff --git a/plus/build.gradle.kts b/plus/build.gradle.kts
index 2aef9d28c5..1b58da0029 100644
--- a/plus/build.gradle.kts
+++ b/plus/build.gradle.kts
@@ -29,6 +29,7 @@ dependencies {
     implementation("org.apache.calcite.avatica:avatica-server")
     implementation("org.hsqldb:hsqldb")
 
+    testImplementation("net.hydromatic:sql-logic-test")
     testImplementation(project(":testkit"))
     testImplementation("org.incava:java-diff")
     testRuntimeOnly("org.apache.logging.log4j:log4j-slf4j-impl")
diff --git a/plus/src/test/java/org/apache/calcite/slt/SqlLogicTests.java 
b/plus/src/test/java/org/apache/calcite/slt/SqlLogicTests.java
new file mode 100644
index 0000000000..7e6ee0c36e
--- /dev/null
+++ b/plus/src/test/java/org/apache/calcite/slt/SqlLogicTests.java
@@ -0,0 +1,314 @@
+/*
+ * 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.slt;
+
+import org.apache.calcite.slt.executors.CalciteExecutor;
+import org.apache.calcite.util.trace.CalciteTrace;
+
+import com.google.common.collect.ImmutableSet;
+
+import net.hydromatic.sqllogictest.Main;
+import net.hydromatic.sqllogictest.OptionsParser;
+import net.hydromatic.sqllogictest.TestStatistics;
+
+import org.junit.jupiter.api.Assumptions;
+import org.junit.jupiter.api.Disabled;
+import org.junit.jupiter.api.DynamicTest;
+import org.junit.jupiter.api.Tag;
+import org.junit.jupiter.api.TestFactory;
+import org.slf4j.Logger;
+
+import java.io.BufferedReader;
+import java.io.BufferedWriter;
+import java.io.File;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.InputStreamReader;
+import java.io.OutputStream;
+import java.io.OutputStreamWriter;
+import java.io.PrintStream;
+import java.io.UnsupportedEncodingException;
+import java.nio.charset.StandardCharsets;
+import java.nio.file.Files;
+import java.util.ArrayList;
+import java.util.Comparator;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import java.util.Set;
+
+import static org.junit.jupiter.api.Assertions.assertFalse;
+
+/**
+ * Tests using sql-logic-test suite.
+ *
+ * <p>For each test file the number of failed tests is saved in a "golden" 
file.
+ * These results are checked in as part of the sltttestfailures.txt resource 
file.
+ * Currently, there are quite a few errors, so this tool does not track of the 
actual
+ * errors that were encountered; we expect that, as bugs are fixed in Calcite,
+ * the number of errors will shrink, and a more precise accounting method will 
be used.
+ *
+ * <p>The tests will fail if any test script generates
+ * more errors than the number from the golden file.
+ */
+public class SqlLogicTests {
+  private static final Logger LOGGER =
+      CalciteTrace.getTestTracer(SqlLogicTests.class);
+
+  /**
+   * Short summary of the results of a test execution.
+   */
+  public static class TestSummary {
+    /**
+     * File containing tests.
+     */
+    final String file;
+    /**
+     * Number of tests that have failed.
+     */
+    final int failed;
+
+    TestSummary(String file, int failed) {
+      this.file = file;
+      this.failed = failed;
+    }
+
+    /**
+     * Parses a TestSummary from a string.
+     * The inverse of 'toString'.
+     *
+     * @return The parsed TestSummary or null on failure.
+     */
+    public static TestSummary parse(String line) {
+      String[] parts = line.split(":");
+      if (parts.length != 2) {
+        return null;
+      }
+      try {
+        int failed = Integer.parseInt(parts[1]);
+        return new TestSummary(parts[0], failed);
+      } catch (NumberFormatException ex) {
+        return null;
+      }
+    }
+
+    @Override public String toString() {
+      return this.file + ":" + this.failed;
+    }
+
+    /**
+     * Check if the 'other' TestSummaries indicate a regressions
+     * when compared to 'this'.
+     *
+     * @param other TestSummary to compare against.
+     * @return 'true' if 'other' is a regression from 'this'.
+     */
+    public boolean isRegression(TestSummary other) {
+      return other.failed > this.failed;
+    }
+  }
+
+  /**
+   * Summary for all tests executed.
+   */
+  public static class AllTestSummaries {
+    /**
+     * Map test summary name to test summary.
+     */
+    final Map<String, TestSummary> testResults;
+
+    AllTestSummaries() {
+      this.testResults = new HashMap<>();
+    }
+
+    void add(TestSummary summary) {
+      this.testResults.put(summary.file, summary);
+    }
+
+    AllTestSummaries read(InputStream stream) {
+      try (BufferedReader reader =
+               new BufferedReader(new InputStreamReader(stream, 
StandardCharsets.UTF_8))) {
+        reader.lines().forEach(line -> {
+          TestSummary summary = TestSummary.parse(line);
+          if (summary != null) {
+            this.add(summary);
+          } else {
+            LOGGER.warn("Could not parse line " + line);
+          }
+        });
+        return this;
+      } catch (IOException ex) {
+        // Wrapping the IOException makes it easier to use this method in the
+        // initializer of a static variable.
+        throw new RuntimeException(ex);
+      }
+    }
+
+    boolean regression(TestSummary summary) {
+      TestSummary original = this.testResults.get(summary.file);
+      if (original == null) {
+        LOGGER.warn("No historical data for test " + summary.file);
+        return false;
+      }
+      if (original.isRegression(summary)) {
+        LOGGER.error("Regression: " + original.file
+            + " had " + original.failed + " failures, now has " + 
summary.failed);
+        return true;
+      }
+      return false;
+    }
+
+    @Override public String toString() {
+      List<TestSummary> results = new ArrayList<>(this.testResults.values());
+      results.sort(Comparator.comparing(left -> left.file));
+      StringBuilder result = new StringBuilder();
+      for (TestSummary summary: results) {
+        result.append(summary.toString());
+        result.append(System.lineSeparator());
+      }
+      return result.toString();
+    }
+
+    /**
+     * Write the test results to the specified file.
+     */
+    public void writeToFile(File file) throws IOException {
+      try (BufferedWriter writer =
+               new BufferedWriter(
+                   new OutputStreamWriter(
+                       Files.newOutputStream(file.toPath()), 
StandardCharsets.UTF_8))) {
+        writer.write(this.toString());
+      }
+    }
+  }
+
+  /**
+   * Summaries produced for the current run.
+   * Must be static since it is written by the `findRegressions`
+   * static method.
+   */
+  private static final AllTestSummaries SUMMARIES = new AllTestSummaries();
+
+  private static final String GOLDEN_FILE = "/slttestfailures.txt";
+  /**
+   * Summaries checked-in as resources that we compare against.
+   */
+  private static final AllTestSummaries GOLDEN_SUMMARIES =
+      new AllTestSummaries()
+          .read(SqlLogicTests.class.getResourceAsStream(GOLDEN_FILE));
+  /**
+   * The following tests currently timeout during execution.
+   * Technically these are Calcite bugs.
+   */
+  private static final Set<String> TIMEOUT =
+      ImmutableSet.of("test/select5.test",
+          "test/random/groupby/slt_good_10.test");
+
+  /**
+   * The following tests contain SQL statements that are not supported by 
HSQLDB.
+   */
+  private static final Set<String> UNSUPPORTED =
+      ImmutableSet.of("test/evidence/slt_lang_replace.test",
+          "test/evidence/slt_lang_createtrigger.test",
+          "test/evidence/slt_lang_droptrigger.test",
+          "test/evidence/slt_lang_update.test",
+          "test/evidence/slt_lang_reindex.test");
+
+  private static void runTestFile(String testFile) {
+    Assumptions.assumeFalse(TIMEOUT.contains(testFile), testFile + " currently 
timeouts");
+    Assumptions.assumeFalse(UNSUPPORTED.contains(testFile),
+        testFile + " contains unsupported statements");
+    OptionsParser options = new OptionsParser(false, nullStream(), 
nullStream());
+    CalciteExecutor.register(options);
+    TestStatistics res;
+    try {
+      res = Main.execute(options, "-e", "calcite", testFile);
+    } catch (IOException e) {
+      throw new RuntimeException(e);
+    }
+    checkStatsForSingleRun(res);
+    TestSummary summary = new TestSummary(testFile, res.getFailedTestCount());
+    boolean regression = GOLDEN_SUMMARIES.regression(summary);
+    assertFalse(regression, "Regression in " + summary.file);
+    // The following is only useful if a new golden file need to be created
+    SUMMARIES.add(summary);
+  }
+
+  private static PrintStream nullStream() {
+    try {
+      return new PrintStream(new OutputStream() {
+        @Override public void write(final int b) {
+          // Do nothing
+        }
+      }, false, "UTF-8");
+    } catch (UnsupportedEncodingException e) {
+      throw new RuntimeException(e);
+    }
+  }
+
+  private static void checkStatsForSingleRun(TestStatistics stats) {
+    Objects.requireNonNull(stats, "stats");
+    if (stats.getParseFailureCount() > 0) {
+      throw new IllegalStateException("Failed to parse file");
+    } else if (stats.getIgnoredTestCount() > 0) {
+      throw new IllegalStateException("File was ignored");
+    } else if (stats.getTestFileCount() > 1) {
+      throw new IllegalStateException("Running multiple files not supported");
+    }
+  }
+
+  @TestFactory @Tag("slow")
+  List<DynamicTest> testSlow() {
+    return generateTests(ImmutableSet.of("select1.test"));
+  }
+
+  @TestFactory @Disabled("This takes very long, should be run manually")
+  List<DynamicTest> testAll() {
+    // Run in parallel each test file.  There are 622 of these, each taking
+    // a few minutes.
+    return generateTests(Main.getTestList());
+  }
+
+  /**
+   * Generate a list of all the tests that can be executed.
+   *
+   * @param testFiles Names of files containing tests.
+   */
+  private List<DynamicTest> generateTests(Set<String> testFiles) {
+    List<DynamicTest> result = new ArrayList<>();
+    for (String test: testFiles) {
+      DynamicTest dynamicTest = DynamicTest.dynamicTest(test, () -> 
runTestFile(test));
+      result.add(dynamicTest);
+    }
+    return result;
+  }
+
+  /**
+   * Create the golden reference file with test results.
+   */
+  public static void createGoldenFile() throws IOException {
+    // Currently this method is not invoked.
+    // It can be used to create a new version of the GOLDEN_FILE
+    // when bugs in Calcite are fixed.  It should be called after
+    // all tests have executed and passed.
+    File file = new File(GOLDEN_FILE);
+    if (!file.exists()) {
+      SUMMARIES.writeToFile(file);
+    }
+  }
+}
diff --git 
a/plus/src/test/java/org/apache/calcite/slt/executors/CalciteExecutor.java 
b/plus/src/test/java/org/apache/calcite/slt/executors/CalciteExecutor.java
new file mode 100644
index 0000000000..6d82045230
--- /dev/null
+++ b/plus/src/test/java/org/apache/calcite/slt/executors/CalciteExecutor.java
@@ -0,0 +1,94 @@
+/*
+ * 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.slt.executors;
+
+import org.apache.calcite.adapter.jdbc.JdbcSchema;
+import org.apache.calcite.jdbc.CalciteConnection;
+import org.apache.calcite.schema.SchemaPlus;
+
+import net.hydromatic.sqllogictest.OptionsParser;
+import net.hydromatic.sqllogictest.SltSqlStatement;
+import net.hydromatic.sqllogictest.executors.HsqldbExecutor;
+import net.hydromatic.sqllogictest.executors.JdbcExecutor;
+
+import java.io.IOException;
+import java.sql.DriverManager;
+import java.sql.SQLException;
+import java.util.Set;
+import javax.sql.DataSource;
+
+/**
+ * Executor for SQL logic tests using Calcite's JDBC adapter.
+ */
+public class CalciteExecutor extends JdbcExecutor {
+  /**
+   * This executor is used for executing the statements (CREATE TABLE,
+   * CREATE VIEW, INSERT).  Queries are executed by Calcite.
+   */
+  private final JdbcExecutor statementExecutor;
+
+  public static void register(OptionsParser parser) {
+    parser.registerExecutor("calcite", () -> {
+      OptionsParser.SuppliedOptions options = parser.getOptions();
+      HsqldbExecutor statementExecutor = new HsqldbExecutor(options);
+      try {
+        CalciteExecutor result = new CalciteExecutor(options, 
statementExecutor);
+        Set<String> bugs = options.readBugsFile();
+        result.avoid(bugs);
+        return result;
+      } catch (IOException | SQLException e) {
+        throw new RuntimeException(e);
+      }
+    });
+  }
+
+  @Override public void establishConnection() throws SQLException {
+    this.statementExecutor.establishConnection();
+  }
+
+  @Override public void dropAllTables() throws SQLException {
+    this.statementExecutor.dropAllTables();
+  }
+
+  @Override public void dropAllViews() throws SQLException {
+    this.statementExecutor.dropAllViews();
+  }
+
+  @Override public void statement(SltSqlStatement statement) throws 
SQLException {
+    this.statementExecutor.statement(statement);
+  }
+
+  public CalciteExecutor(OptionsParser.SuppliedOptions options, JdbcExecutor 
statementExecutor)
+      throws SQLException {
+    super(options, "jdbc:calcite:lex=ORACLE", "", "");
+    this.statementExecutor = statementExecutor;
+    // Build our connection
+    this.connection =
+        DriverManager.getConnection("jdbc:calcite:lex=ORACLE");
+    CalciteConnection calciteConnection = 
this.connection.unwrap(CalciteConnection.class);
+    SchemaPlus rootSchema = calciteConnection.getRootSchema();
+    DataSource hsqldb =
+        JdbcSchema.dataSource(statementExecutor.dbUrl,
+        "org.hsqldb.jdbcDriver",
+        "",
+        "");
+    final String schemaName = "SLT";
+    JdbcSchema jdbcSchema = JdbcSchema.create(rootSchema, schemaName, hsqldb, 
null, null);
+    rootSchema.add(schemaName, jdbcSchema);
+    calciteConnection.setSchema(schemaName);
+  }
+}
diff --git a/plus/build.gradle.kts 
b/plus/src/test/java/org/apache/calcite/slt/executors/package-info.java
similarity index 53%
copy from plus/build.gradle.kts
copy to plus/src/test/java/org/apache/calcite/slt/executors/package-info.java
index 2aef9d28c5..21dba71684 100644
--- a/plus/build.gradle.kts
+++ b/plus/src/test/java/org/apache/calcite/slt/executors/package-info.java
@@ -14,22 +14,8 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-dependencies {
-    api(project(":core"))
-    api(project(":linq4j"))
-    api("net.hydromatic:quidem")
-    api("org.apache.calcite.avatica:avatica-core")
-    api("org.checkerframework:checker-qual")
 
-    implementation("com.google.guava:guava")
-    implementation("com.teradata.tpcds:tpcds")
-    implementation("io.prestosql.tpch:tpch")
-    implementation("net.hydromatic:chinook-data-hsqldb")
-    implementation("net.hydromatic:tpcds")
-    implementation("org.apache.calcite.avatica:avatica-server")
-    implementation("org.hsqldb:hsqldb")
-
-    testImplementation(project(":testkit"))
-    testImplementation("org.incava:java-diff")
-    testRuntimeOnly("org.apache.logging.log4j:log4j-slf4j-impl")
-}
+/**
+ * Executors for Sql Logic Test based on Calcite.
+ */
+package org.apache.calcite.slt.executors;
diff --git a/plus/src/test/java/org/apache/calcite/slt/package-info.java 
b/plus/src/test/java/org/apache/calcite/slt/package-info.java
new file mode 100644
index 0000000000..a059439464
--- /dev/null
+++ b/plus/src/test/java/org/apache/calcite/slt/package-info.java
@@ -0,0 +1,105 @@
+/*
+ * 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.
+ */
+
+/**
+ * <a href="https://www.sqlite.org/sqllogictest/doc/trunk/about.wiki";>SQL 
Logic Tests</a>
+ * is public-domain suite of several million SQL tests created as part of the 
sqlite engine.
+ * The project <a href="https://github.com/hydromatic/sql-logic-test/";>
+ * hydromatic/sql-logic-test</a>
+ * has packaged these tests into a Java framework which makes it easy to
+ * run them against a JDBC provider.
+ *
+ * <p>In this project we test Calcite as a JDBC provider, coupled with
+ * <a href="http://hsqldb.org/";>HSQLDB</a> for storage.
+ * Another storage layer, such as <a 
href="https://www.postgresql.org";>Postgres</a>,
+ * could be substituted for HSQLDB.
+ * (Please note that the hydromatic/sql-logic-test project is configured
+ * to use the Postgres dialect version of the queries; for other dialects
+ * you may need to tweak the source code.)
+ *
+ * <p>The test suite consists of 622 SQL scripts.  Each script contains SQL
+ * statements and queries.  The statements are executed using HSQLDB, while
+ * the queries are executed using Calcite.
+ *
+ * <p><strong>Errors</strong>
+ *
+ * <p>There are several classes of errors that these test have discovered in 
Calcite:
+ * <ul>
+ * <li>compilation or execution that does not terminate.  Currently, we have 2 
such tests,
+ *   appearing in the `SqlLogicTestsForCalciteTests.timeouts` set.</li>
+ * <li>crashes in Calcite compilation or execution.  These show up as 
`SqlException` exceptions
+ *   and usually produce several lines of logs each</li>
+ * <li>incorrect outputs produced by Calcite.  By increasing the verbosity
+ *   even further one can see the expected and produced output.  Note that
+ *   not all tests specify the output in full, some only specify an MD5 
checksum.</li>
+ * <li>incorrect outputs produced by Calcite detected through a different MD5 
checksum.</li>
+ * </ul>
+ *
+ * <p>Here is an example output fragment produced for one test script when 
using verbosity 1
+ * (there are 12 errors, but we only show 4 different ones here):
+ * <pre>
+ * 12 failures
+ * Total files processed: 1
+ * Files not parsed: 0
+ * Passed: 988
+ * Failed: 12
+ * Ignored: 0
+ * 12 failures:
+ * ERROR: Required columns {2, 3} not subset of left columns {0, 1, 2}
+ * test: test/select2.test:1947
+ * SELECT d-e, (SELECT count(*) FROM t1 AS x WHERE x.c>t1.c AND x.d &lt; t1.d) 
FROM t1 WHERE
+ * (e &gt; c OR e &lt; d)
+ * Litmus.java:31 org.apache.calcite.util.Litmus.lambda$static$0
+ * ERROR: Error while executing SQL "SELECT d, abs(a), (SELECT count(*) FROM 
t1 AS x
+ * WHERE x.b &lt; t1 .b) FROM t1 WHERE a IS NULL AND a &gt; b":
+ * Unable to implement EnumerableCalc(expr#0..3=[{inputs}],
+ * expr#4=[ABS($t0)], expr#5=[IS NULL($t3)], expr#6=[0:BIGINT], 
expr#7=[CASE($t5, $t6, $t3)],
+ * D=[$t2], EXPR$1=[$t4], EXPR$2=[$t7]): rowcount = 1.0, cumulative cost = 
{465.45 rows, 574.2
+ * cpu, 0.0 io}, id = 48555
+ *   EnumerableCorrelate(correlation=[$cor0], joinType=[left], 
requiredColumns=[{1}]): rowcount =
+ *   1.0, cumulative cost = {464.45 rows, 563.2 cpu, 0.0 io}, id = 48551
+ *     EnumerableValues(tuples=[[]]): rowcount = 1.0, cumulative cost = {1.0 
rows, 1.0 cpu, 0.0
+ *     io}, id = 48504
+ *     JdbcToEnumerableConverter: rowcount = 1.0, cumulative cost = {231.225 
rows, 281.1 cpu, 0.0
+ *     io}, id = 48549
+ *       JdbcAggregate(group=[{}], EXPR$0=[COUNT()]): rowcount = 1.0, 
cumulative cost = {231.125
+ *       rows, 281.0 cpu, 0.0 io}, id = 48547
+ *         JdbcFilter(condition=[<($0, $cor0.B)]): rowcount = 50.0, cumulative 
cost = {230.0
+ *         rows, 281.0 cpu, 0.0 io}, id = 48545
+ *           JdbcProject(B=[$1]): rowcount = 100.0, cumulative cost = {180.0 
rows, 181.0 cpu, 0.0
+ *           io}, id = 48543
+ *             JdbcTableScan(table=[[SLT, T1]]): rowcount = 100.0, cumulative 
cost = {100.0 rows,
+ *             101.0 cpu, 0.0 io}, id = 48416
+ * test: test/select2.test:2436
+ * SELECT d, abs(a), (SELECT count(*) FROM t1 AS x WHERE x.b &lt; t1.b)
+ * FROM t1 WHERE a IS NULL AND a &gt; b
+ * Helper.java:56 org.apache.calcite.avatica.Helper.createException
+ * ERROR: Hash of data does not match expected value
+ * test: test/select2.test:4852
+ * SELECT c-d, abs(b-c), b-c, (SELECT count(*) FROM t1 AS x WHERE x.b &lt; 
t1.b), d FROM t1
+ * WHERE b &gt; c OR c &gt; d OR d NOT BETWEEN 110 AND 150
+ * ERROR: Error while executing SQL "SELECT (SELECT count(*) FROM t1 AS x 
WHERE x.b &lt; t1.b),
+ * c-d, (SELECT count(*) FROM t1 AS x WHERE x.c &gt; t1.c AND x.d &lt; t1.d), 
abs(b-c), b
+ * FROM t1 WHERE c &gt; d AND EXISTS(SELECT 1 FROM t1 AS x WHERE x.b &lt; 
t1.b)": null
+ * test: test/select2.test:5141
+ * SELECT (SELECT count(*) FROM t1 AS x WHERE x.b &lt; t1.b), c-d,
+ * (SELECT count(*) FROM t1 AS x WHERE x.c &gt; t1.c AND x.d &lt; t1.d), 
abs(b-c), b FROM t1
+ * WHERE c &gt; d AND EXISTS(SELECT 1 FROM t1 AS x WHERE x .b &lt; t1.b)
+ * Helper.java:56 org.apache.calcite.avatica.Helper.createException
+ * </pre>
+ */
+package org.apache.calcite.slt;
diff --git a/plus/src/test/resources/slttestfailures.txt 
b/plus/src/test/resources/slttestfailures.txt
new file mode 100644
index 0000000000..d2747148c9
--- /dev/null
+++ b/plus/src/test/resources/slttestfailures.txt
@@ -0,0 +1,615 @@
+test/evidence/in1.test:8
+test/evidence/in2.test:8
+test/evidence/slt_lang_aggfunc.test:0
+test/evidence/slt_lang_createview.test:0
+test/evidence/slt_lang_dropindex.test:0
+test/evidence/slt_lang_droptable.test:0
+test/evidence/slt_lang_dropview.test:0
+test/index/between/1/slt_good_0.test:60
+test/index/between/10/slt_good_0.test:50
+test/index/between/10/slt_good_1.test:30
+test/index/between/10/slt_good_2.test:40
+test/index/between/10/slt_good_3.test:70
+test/index/between/10/slt_good_4.test:30
+test/index/between/10/slt_good_5.test:60
+test/index/between/100/slt_good_0.test:20
+test/index/between/100/slt_good_1.test:60
+test/index/between/100/slt_good_2.test:50
+test/index/between/100/slt_good_3.test:40
+test/index/between/100/slt_good_4.test:50
+test/index/between/1000/slt_good_0.test:10
+test/index/commute/10/slt_good_0.test:0
+test/index/commute/10/slt_good_1.test:20
+test/index/commute/10/slt_good_10.test:10
+test/index/commute/10/slt_good_11.test:10
+test/index/commute/10/slt_good_12.test:0
+test/index/commute/10/slt_good_13.test:10
+test/index/commute/10/slt_good_14.test:20
+test/index/commute/10/slt_good_15.test:10
+test/index/commute/10/slt_good_16.test:20
+test/index/commute/10/slt_good_17.test:40
+test/index/commute/10/slt_good_18.test:20
+test/index/commute/10/slt_good_19.test:10
+test/index/commute/10/slt_good_2.test:20
+test/index/commute/10/slt_good_20.test:0
+test/index/commute/10/slt_good_21.test:0
+test/index/commute/10/slt_good_22.test:10
+test/index/commute/10/slt_good_23.test:10
+test/index/commute/10/slt_good_24.test:0
+test/index/commute/10/slt_good_25.test:10
+test/index/commute/10/slt_good_26.test:0
+test/index/commute/10/slt_good_27.test:0
+test/index/commute/10/slt_good_28.test:0
+test/index/commute/10/slt_good_29.test:0
+test/index/commute/10/slt_good_3.test:0
+test/index/commute/10/slt_good_30.test:10
+test/index/commute/10/slt_good_31.test:10
+test/index/commute/10/slt_good_32.test:10
+test/index/commute/10/slt_good_33.test:20
+test/index/commute/10/slt_good_34.test:0
+test/index/commute/10/slt_good_4.test:0
+test/index/commute/10/slt_good_5.test:15
+test/index/commute/10/slt_good_6.test:0
+test/index/commute/10/slt_good_7.test:10
+test/index/commute/10/slt_good_8.test:10
+test/index/commute/10/slt_good_9.test:10
+test/index/commute/100/slt_good_0.test:10
+test/index/commute/100/slt_good_1.test:0
+test/index/commute/100/slt_good_10.test:20
+test/index/commute/100/slt_good_11.test:20
+test/index/commute/100/slt_good_12.test:0
+test/index/commute/100/slt_good_2.test:10
+test/index/commute/100/slt_good_3.test:10
+test/index/commute/100/slt_good_4.test:20
+test/index/commute/100/slt_good_5.test:20
+test/index/commute/100/slt_good_6.test:10
+test/index/commute/100/slt_good_7.test:10
+test/index/commute/100/slt_good_8.test:10
+test/index/commute/100/slt_good_9.test:10
+test/index/commute/1000/slt_good_0.test:0
+test/index/commute/1000/slt_good_1.test:10
+test/index/commute/1000/slt_good_2.test:10
+test/index/commute/1000/slt_good_3.test:0
+test/index/delete/1/slt_good_0.test:0
+test/index/delete/10/slt_good_0.test:0
+test/index/delete/10/slt_good_1.test:0
+test/index/delete/10/slt_good_2.test:0
+test/index/delete/10/slt_good_3.test:0
+test/index/delete/10/slt_good_4.test:0
+test/index/delete/10/slt_good_5.test:0
+test/index/delete/100/slt_good_0.test:0
+test/index/delete/100/slt_good_1.test:0
+test/index/delete/100/slt_good_2.test:0
+test/index/delete/100/slt_good_3.test:0
+test/index/delete/1000/slt_good_0.test:0
+test/index/delete/1000/slt_good_1.test:0
+test/index/delete/10000/slt_good_0.test:0
+test/index/in/10/slt_good_0.test:30
+test/index/in/10/slt_good_1.test:45
+test/index/in/10/slt_good_2.test:45
+test/index/in/10/slt_good_3.test:0
+test/index/in/10/slt_good_4.test:30
+test/index/in/10/slt_good_5.test:60
+test/index/in/100/slt_good_0.test:45
+test/index/in/100/slt_good_1.test:45
+test/index/in/100/slt_good_2.test:60
+test/index/in/100/slt_good_3.test:15
+test/index/in/100/slt_good_4.test:45
+test/index/in/1000/slt_good_0.test:75
+test/index/in/1000/slt_good_1.test:60
+test/index/orderby/10/slt_good_0.test:120
+test/index/orderby/10/slt_good_1.test:0
+test/index/orderby/10/slt_good_10.test:0
+test/index/orderby/10/slt_good_11.test:0
+test/index/orderby/10/slt_good_12.test:0
+test/index/orderby/10/slt_good_13.test:0
+test/index/orderby/10/slt_good_14.test:0
+test/index/orderby/10/slt_good_15.test:0
+test/index/orderby/10/slt_good_16.test:0
+test/index/orderby/10/slt_good_17.test:0
+test/index/orderby/10/slt_good_18.test:0
+test/index/orderby/10/slt_good_19.test:0
+test/index/orderby/10/slt_good_2.test:0
+test/index/orderby/10/slt_good_20.test:0
+test/index/orderby/10/slt_good_21.test:0
+test/index/orderby/10/slt_good_22.test:0
+test/index/orderby/10/slt_good_23.test:0
+test/index/orderby/10/slt_good_24.test:0
+test/index/orderby/10/slt_good_25.test:0
+test/index/orderby/10/slt_good_3.test:0
+test/index/orderby/10/slt_good_4.test:0
+test/index/orderby/10/slt_good_5.test:0
+test/index/orderby/10/slt_good_6.test:0
+test/index/orderby/10/slt_good_7.test:0
+test/index/orderby/10/slt_good_8.test:60
+test/index/orderby/10/slt_good_9.test:0
+test/index/orderby/100/slt_good_0.test:0
+test/index/orderby/100/slt_good_1.test:0
+test/index/orderby/100/slt_good_2.test:0
+test/index/orderby/100/slt_good_3.test:0
+test/index/orderby/1000/slt_good_0.test:0
+test/index/orderby_nosort/10/slt_good_0.test:0
+test/index/orderby_nosort/10/slt_good_1.test:0
+test/index/orderby_nosort/10/slt_good_10.test:0
+test/index/orderby_nosort/10/slt_good_11.test:60
+test/index/orderby_nosort/10/slt_good_12.test:0
+test/index/orderby_nosort/10/slt_good_13.test:0
+test/index/orderby_nosort/10/slt_good_14.test:0
+test/index/orderby_nosort/10/slt_good_15.test:0
+test/index/orderby_nosort/10/slt_good_16.test:0
+test/index/orderby_nosort/10/slt_good_17.test:0
+test/index/orderby_nosort/10/slt_good_18.test:0
+test/index/orderby_nosort/10/slt_good_19.test:0
+test/index/orderby_nosort/10/slt_good_2.test:0
+test/index/orderby_nosort/10/slt_good_20.test:0
+test/index/orderby_nosort/10/slt_good_21.test:0
+test/index/orderby_nosort/10/slt_good_22.test:0
+test/index/orderby_nosort/10/slt_good_23.test:0
+test/index/orderby_nosort/10/slt_good_24.test:0
+test/index/orderby_nosort/10/slt_good_25.test:0
+test/index/orderby_nosort/10/slt_good_26.test:0
+test/index/orderby_nosort/10/slt_good_27.test:0
+test/index/orderby_nosort/10/slt_good_28.test:0
+test/index/orderby_nosort/10/slt_good_29.test:0
+test/index/orderby_nosort/10/slt_good_3.test:0
+test/index/orderby_nosort/10/slt_good_30.test:0
+test/index/orderby_nosort/10/slt_good_31.test:0
+test/index/orderby_nosort/10/slt_good_32.test:60
+test/index/orderby_nosort/10/slt_good_33.test:0
+test/index/orderby_nosort/10/slt_good_34.test:0
+test/index/orderby_nosort/10/slt_good_35.test:0
+test/index/orderby_nosort/10/slt_good_36.test:0
+test/index/orderby_nosort/10/slt_good_37.test:0
+test/index/orderby_nosort/10/slt_good_38.test:60
+test/index/orderby_nosort/10/slt_good_39.test:0
+test/index/orderby_nosort/10/slt_good_4.test:60
+test/index/orderby_nosort/10/slt_good_5.test:0
+test/index/orderby_nosort/10/slt_good_6.test:0
+test/index/orderby_nosort/10/slt_good_7.test:0
+test/index/orderby_nosort/10/slt_good_8.test:0
+test/index/orderby_nosort/10/slt_good_9.test:0
+test/index/orderby_nosort/100/slt_good_0.test:60
+test/index/orderby_nosort/100/slt_good_1.test:0
+test/index/orderby_nosort/100/slt_good_2.test:0
+test/index/orderby_nosort/100/slt_good_3.test:0
+test/index/orderby_nosort/100/slt_good_4.test:0
+test/index/orderby_nosort/100/slt_good_5.test:0
+test/index/orderby_nosort/100/slt_good_6.test:0
+test/index/orderby_nosort/1000/slt_good_0.test:0
+test/index/orderby_nosort/1000/slt_good_1.test:0
+test/index/random/10/slt_good_0.test:145
+test/index/random/10/slt_good_1.test:150
+test/index/random/10/slt_good_10.test:190
+test/index/random/10/slt_good_11.test:205
+test/index/random/10/slt_good_12.test:190
+test/index/random/10/slt_good_13.test:150
+test/index/random/10/slt_good_14.test:180
+test/index/random/10/slt_good_2.test:195
+test/index/random/10/slt_good_3.test:170
+test/index/random/10/slt_good_4.test:140
+test/index/random/10/slt_good_5.test:205
+test/index/random/10/slt_good_6.test:205
+test/index/random/10/slt_good_7.test:195
+test/index/random/10/slt_good_8.test:150
+test/index/random/10/slt_good_9.test:210
+test/index/random/100/slt_good_0.test:195
+test/index/random/100/slt_good_1.test:205
+test/index/random/1000/slt_good_0.test:15
+test/index/random/1000/slt_good_1.test:0
+test/index/random/1000/slt_good_2.test:0
+test/index/random/1000/slt_good_3.test:0
+test/index/random/1000/slt_good_4.test:0
+test/index/random/1000/slt_good_5.test:58
+test/index/random/1000/slt_good_6.test:195
+test/index/random/1000/slt_good_7.test:215
+test/index/random/1000/slt_good_8.test:40
+test/index/view/10/slt_good_0.test:0
+test/index/view/10/slt_good_1.test:0
+test/index/view/10/slt_good_2.test:0
+test/index/view/10/slt_good_3.test:0
+test/index/view/10/slt_good_4.test:0
+test/index/view/10/slt_good_5.test:0
+test/index/view/10/slt_good_6.test:0
+test/index/view/10/slt_good_7.test:0
+test/index/view/100/slt_good_0.test:0
+test/index/view/100/slt_good_1.test:0
+test/index/view/100/slt_good_2.test:0
+test/index/view/100/slt_good_3.test:0
+test/index/view/100/slt_good_4.test:0
+test/index/view/100/slt_good_5.test:0
+test/index/view/1000/slt_good_0.test:0
+test/index/view/10000/slt_good_0.test:0
+test/random/aggregates/slt_good_0.test:14
+test/random/aggregates/slt_good_1.test:17
+test/random/aggregates/slt_good_10.test:16
+test/random/aggregates/slt_good_100.test:10
+test/random/aggregates/slt_good_101.test:22
+test/random/aggregates/slt_good_102.test:17
+test/random/aggregates/slt_good_103.test:13
+test/random/aggregates/slt_good_104.test:12
+test/random/aggregates/slt_good_105.test:14
+test/random/aggregates/slt_good_106.test:17
+test/random/aggregates/slt_good_107.test:6
+test/random/aggregates/slt_good_108.test:17
+test/random/aggregates/slt_good_109.test:14
+test/random/aggregates/slt_good_11.test:19
+test/random/aggregates/slt_good_110.test:23
+test/random/aggregates/slt_good_111.test:15
+test/random/aggregates/slt_good_112.test:19
+test/random/aggregates/slt_good_113.test:15
+test/random/aggregates/slt_good_114.test:21
+test/random/aggregates/slt_good_115.test:16
+test/random/aggregates/slt_good_116.test:14
+test/random/aggregates/slt_good_117.test:20
+test/random/aggregates/slt_good_118.test:19
+test/random/aggregates/slt_good_119.test:18
+test/random/aggregates/slt_good_12.test:21
+test/random/aggregates/slt_good_120.test:17
+test/random/aggregates/slt_good_121.test:13
+test/random/aggregates/slt_good_122.test:14
+test/random/aggregates/slt_good_123.test:13
+test/random/aggregates/slt_good_124.test:16
+test/random/aggregates/slt_good_125.test:23
+test/random/aggregates/slt_good_126.test:26
+test/random/aggregates/slt_good_127.test:20
+test/random/aggregates/slt_good_128.test:20
+test/random/aggregates/slt_good_129.test:1
+test/random/aggregates/slt_good_13.test:24
+test/random/aggregates/slt_good_14.test:14
+test/random/aggregates/slt_good_15.test:19
+test/random/aggregates/slt_good_16.test:17
+test/random/aggregates/slt_good_17.test:22
+test/random/aggregates/slt_good_18.test:21
+test/random/aggregates/slt_good_19.test:15
+test/random/aggregates/slt_good_2.test:15
+test/random/aggregates/slt_good_20.test:14
+test/random/aggregates/slt_good_21.test:26
+test/random/aggregates/slt_good_22.test:18
+test/random/aggregates/slt_good_23.test:24
+test/random/aggregates/slt_good_24.test:13
+test/random/aggregates/slt_good_25.test:15
+test/random/aggregates/slt_good_26.test:21
+test/random/aggregates/slt_good_27.test:16
+test/random/aggregates/slt_good_28.test:22
+test/random/aggregates/slt_good_29.test:20
+test/random/aggregates/slt_good_3.test:13
+test/random/aggregates/slt_good_30.test:15
+test/random/aggregates/slt_good_31.test:19
+test/random/aggregates/slt_good_32.test:20
+test/random/aggregates/slt_good_33.test:18
+test/random/aggregates/slt_good_34.test:17
+test/random/aggregates/slt_good_35.test:21
+test/random/aggregates/slt_good_36.test:9
+test/random/aggregates/slt_good_37.test:12
+test/random/aggregates/slt_good_38.test:15
+test/random/aggregates/slt_good_39.test:15
+test/random/aggregates/slt_good_4.test:13
+test/random/aggregates/slt_good_40.test:11
+test/random/aggregates/slt_good_41.test:15
+test/random/aggregates/slt_good_42.test:20
+test/random/aggregates/slt_good_43.test:18
+test/random/aggregates/slt_good_44.test:15
+test/random/aggregates/slt_good_45.test:14
+test/random/aggregates/slt_good_46.test:20
+test/random/aggregates/slt_good_47.test:7
+test/random/aggregates/slt_good_48.test:18
+test/random/aggregates/slt_good_49.test:18
+test/random/aggregates/slt_good_5.test:21
+test/random/aggregates/slt_good_50.test:18
+test/random/aggregates/slt_good_51.test:16
+test/random/aggregates/slt_good_52.test:22
+test/random/aggregates/slt_good_53.test:14
+test/random/aggregates/slt_good_54.test:14
+test/random/aggregates/slt_good_55.test:16
+test/random/aggregates/slt_good_56.test:20
+test/random/aggregates/slt_good_57.test:14
+test/random/aggregates/slt_good_58.test:16
+test/random/aggregates/slt_good_59.test:11
+test/random/aggregates/slt_good_6.test:14
+test/random/aggregates/slt_good_60.test:15
+test/random/aggregates/slt_good_61.test:15
+test/random/aggregates/slt_good_62.test:11
+test/random/aggregates/slt_good_63.test:16
+test/random/aggregates/slt_good_64.test:11
+test/random/aggregates/slt_good_65.test:22
+test/random/aggregates/slt_good_66.test:13
+test/random/aggregates/slt_good_67.test:14
+test/random/aggregates/slt_good_68.test:21
+test/random/aggregates/slt_good_69.test:14
+test/random/aggregates/slt_good_7.test:20
+test/random/aggregates/slt_good_70.test:17
+test/random/aggregates/slt_good_71.test:19
+test/random/aggregates/slt_good_72.test:15
+test/random/aggregates/slt_good_73.test:20
+test/random/aggregates/slt_good_74.test:10
+test/random/aggregates/slt_good_75.test:14
+test/random/aggregates/slt_good_76.test:24
+test/random/aggregates/slt_good_77.test:14
+test/random/aggregates/slt_good_78.test:20
+test/random/aggregates/slt_good_79.test:23
+test/random/aggregates/slt_good_8.test:18
+test/random/aggregates/slt_good_80.test:23
+test/random/aggregates/slt_good_81.test:21
+test/random/aggregates/slt_good_82.test:25
+test/random/aggregates/slt_good_83.test:21
+test/random/aggregates/slt_good_84.test:8
+test/random/aggregates/slt_good_85.test:26
+test/random/aggregates/slt_good_86.test:21
+test/random/aggregates/slt_good_87.test:14
+test/random/aggregates/slt_good_88.test:15
+test/random/aggregates/slt_good_89.test:24
+test/random/aggregates/slt_good_9.test:16
+test/random/aggregates/slt_good_90.test:27
+test/random/aggregates/slt_good_91.test:16
+test/random/aggregates/slt_good_92.test:15
+test/random/aggregates/slt_good_93.test:18
+test/random/aggregates/slt_good_94.test:26
+test/random/aggregates/slt_good_95.test:12
+test/random/aggregates/slt_good_96.test:11
+test/random/aggregates/slt_good_97.test:13
+test/random/aggregates/slt_good_98.test:14
+test/random/aggregates/slt_good_99.test:26
+test/random/expr/slt_good_0.test:0
+test/random/expr/slt_good_1.test:0
+test/random/expr/slt_good_10.test:0
+test/random/expr/slt_good_100.test:2
+test/random/expr/slt_good_101.test:3
+test/random/expr/slt_good_102.test:3
+test/random/expr/slt_good_103.test:3
+test/random/expr/slt_good_104.test:1
+test/random/expr/slt_good_105.test:1
+test/random/expr/slt_good_106.test:2
+test/random/expr/slt_good_107.test:3
+test/random/expr/slt_good_108.test:2
+test/random/expr/slt_good_109.test:1
+test/random/expr/slt_good_11.test:2
+test/random/expr/slt_good_110.test:1
+test/random/expr/slt_good_111.test:1
+test/random/expr/slt_good_112.test:3
+test/random/expr/slt_good_113.test:2
+test/random/expr/slt_good_114.test:2
+test/random/expr/slt_good_115.test:1
+test/random/expr/slt_good_116.test:2
+test/random/expr/slt_good_117.test:0
+test/random/expr/slt_good_118.test:2
+test/random/expr/slt_good_119.test:3
+test/random/expr/slt_good_12.test:2
+test/random/expr/slt_good_13.test:0
+test/random/expr/slt_good_14.test:0
+test/random/expr/slt_good_15.test:2
+test/random/expr/slt_good_16.test:1
+test/random/expr/slt_good_17.test:2
+test/random/expr/slt_good_18.test:5
+test/random/expr/slt_good_19.test:1
+test/random/expr/slt_good_2.test:1
+test/random/expr/slt_good_20.test:2
+test/random/expr/slt_good_21.test:0
+test/random/expr/slt_good_22.test:3
+test/random/expr/slt_good_23.test:1
+test/random/expr/slt_good_24.test:2
+test/random/expr/slt_good_25.test:1
+test/random/expr/slt_good_26.test:1
+test/random/expr/slt_good_27.test:0
+test/random/expr/slt_good_28.test:3
+test/random/expr/slt_good_29.test:0
+test/random/expr/slt_good_3.test:1
+test/random/expr/slt_good_30.test:2
+test/random/expr/slt_good_31.test:1
+test/random/expr/slt_good_32.test:0
+test/random/expr/slt_good_33.test:1
+test/random/expr/slt_good_34.test:3
+test/random/expr/slt_good_35.test:2
+test/random/expr/slt_good_36.test:3
+test/random/expr/slt_good_37.test:3
+test/random/expr/slt_good_38.test:1
+test/random/expr/slt_good_39.test:1
+test/random/expr/slt_good_4.test:3
+test/random/expr/slt_good_40.test:4
+test/random/expr/slt_good_41.test:4
+test/random/expr/slt_good_42.test:3
+test/random/expr/slt_good_43.test:0
+test/random/expr/slt_good_44.test:2
+test/random/expr/slt_good_45.test:0
+test/random/expr/slt_good_46.test:3
+test/random/expr/slt_good_47.test:3
+test/random/expr/slt_good_48.test:1
+test/random/expr/slt_good_49.test:0
+test/random/expr/slt_good_5.test:5
+test/random/expr/slt_good_50.test:2
+test/random/expr/slt_good_51.test:0
+test/random/expr/slt_good_52.test:0
+test/random/expr/slt_good_53.test:1
+test/random/expr/slt_good_54.test:0
+test/random/expr/slt_good_55.test:3
+test/random/expr/slt_good_56.test:1
+test/random/expr/slt_good_57.test:1
+test/random/expr/slt_good_58.test:4
+test/random/expr/slt_good_59.test:2
+test/random/expr/slt_good_6.test:2
+test/random/expr/slt_good_60.test:2
+test/random/expr/slt_good_61.test:0
+test/random/expr/slt_good_62.test:2
+test/random/expr/slt_good_63.test:2
+test/random/expr/slt_good_64.test:2
+test/random/expr/slt_good_65.test:0
+test/random/expr/slt_good_66.test:3
+test/random/expr/slt_good_67.test:3
+test/random/expr/slt_good_68.test:0
+test/random/expr/slt_good_69.test:3
+test/random/expr/slt_good_7.test:3
+test/random/expr/slt_good_70.test:1
+test/random/expr/slt_good_71.test:3
+test/random/expr/slt_good_72.test:1
+test/random/expr/slt_good_73.test:2
+test/random/expr/slt_good_74.test:1
+test/random/expr/slt_good_75.test:1
+test/random/expr/slt_good_76.test:0
+test/random/expr/slt_good_77.test:4
+test/random/expr/slt_good_78.test:0
+test/random/expr/slt_good_79.test:3
+test/random/expr/slt_good_8.test:1
+test/random/expr/slt_good_80.test:1
+test/random/expr/slt_good_81.test:2
+test/random/expr/slt_good_82.test:1
+test/random/expr/slt_good_83.test:4
+test/random/expr/slt_good_84.test:2
+test/random/expr/slt_good_85.test:1
+test/random/expr/slt_good_86.test:2
+test/random/expr/slt_good_87.test:0
+test/random/expr/slt_good_88.test:1
+test/random/expr/slt_good_89.test:5
+test/random/expr/slt_good_9.test:2
+test/random/expr/slt_good_90.test:1
+test/random/expr/slt_good_91.test:2
+test/random/expr/slt_good_92.test:1
+test/random/expr/slt_good_93.test:1
+test/random/expr/slt_good_94.test:3
+test/random/expr/slt_good_95.test:1
+test/random/expr/slt_good_96.test:2
+test/random/expr/slt_good_97.test:0
+test/random/expr/slt_good_98.test:1
+test/random/expr/slt_good_99.test:5
+test/random/groupby/slt_good_0.test:3
+test/random/groupby/slt_good_1.test:7
+test/random/groupby/slt_good_11.test:14
+test/random/groupby/slt_good_12.test:24
+test/random/groupby/slt_good_13.test:1
+test/random/groupby/slt_good_2.test:3
+test/random/groupby/slt_good_3.test:6
+test/random/groupby/slt_good_4.test:7
+test/random/groupby/slt_good_5.test:5
+test/random/groupby/slt_good_6.test:4
+test/random/groupby/slt_good_7.test:5
+test/random/groupby/slt_good_8.test:2
+test/random/groupby/slt_good_9.test:0
+test/random/select/slt_good_0.test:76
+test/random/select/slt_good_1.test:70
+test/random/select/slt_good_10.test:54
+test/random/select/slt_good_100.test:42
+test/random/select/slt_good_101.test:42
+test/random/select/slt_good_102.test:37
+test/random/select/slt_good_103.test:35
+test/random/select/slt_good_104.test:44
+test/random/select/slt_good_105.test:45
+test/random/select/slt_good_106.test:32
+test/random/select/slt_good_107.test:44
+test/random/select/slt_good_108.test:41
+test/random/select/slt_good_109.test:34
+test/random/select/slt_good_11.test:50
+test/random/select/slt_good_110.test:36
+test/random/select/slt_good_111.test:26
+test/random/select/slt_good_112.test:28
+test/random/select/slt_good_113.test:46
+test/random/select/slt_good_114.test:32
+test/random/select/slt_good_115.test:40
+test/random/select/slt_good_116.test:38
+test/random/select/slt_good_117.test:26
+test/random/select/slt_good_118.test:38
+test/random/select/slt_good_119.test:35
+test/random/select/slt_good_12.test:53
+test/random/select/slt_good_120.test:32
+test/random/select/slt_good_121.test:39
+test/random/select/slt_good_122.test:46
+test/random/select/slt_good_123.test:34
+test/random/select/slt_good_124.test:8
+test/random/select/slt_good_125.test:0
+test/random/select/slt_good_126.test:0
+test/random/select/slt_good_13.test:40
+test/random/select/slt_good_14.test:47
+test/random/select/slt_good_15.test:36
+test/random/select/slt_good_16.test:45
+test/random/select/slt_good_17.test:32
+test/random/select/slt_good_18.test:34
+test/random/select/slt_good_19.test:46
+test/random/select/slt_good_2.test:47
+test/random/select/slt_good_20.test:36
+test/random/select/slt_good_21.test:37
+test/random/select/slt_good_22.test:29
+test/random/select/slt_good_23.test:48
+test/random/select/slt_good_24.test:35
+test/random/select/slt_good_25.test:28
+test/random/select/slt_good_26.test:36
+test/random/select/slt_good_27.test:43
+test/random/select/slt_good_28.test:33
+test/random/select/slt_good_29.test:36
+test/random/select/slt_good_3.test:38
+test/random/select/slt_good_30.test:36
+test/random/select/slt_good_31.test:40
+test/random/select/slt_good_32.test:30
+test/random/select/slt_good_33.test:41
+test/random/select/slt_good_34.test:42
+test/random/select/slt_good_35.test:46
+test/random/select/slt_good_36.test:28
+test/random/select/slt_good_37.test:39
+test/random/select/slt_good_38.test:49
+test/random/select/slt_good_39.test:36
+test/random/select/slt_good_4.test:41
+test/random/select/slt_good_40.test:46
+test/random/select/slt_good_41.test:33
+test/random/select/slt_good_42.test:40
+test/random/select/slt_good_43.test:44
+test/random/select/slt_good_44.test:35
+test/random/select/slt_good_45.test:44
+test/random/select/slt_good_46.test:41
+test/random/select/slt_good_47.test:36
+test/random/select/slt_good_48.test:39
+test/random/select/slt_good_49.test:44
+test/random/select/slt_good_5.test:57
+test/random/select/slt_good_50.test:26
+test/random/select/slt_good_51.test:42
+test/random/select/slt_good_52.test:31
+test/random/select/slt_good_53.test:40
+test/random/select/slt_good_54.test:41
+test/random/select/slt_good_55.test:33
+test/random/select/slt_good_56.test:32
+test/random/select/slt_good_57.test:44
+test/random/select/slt_good_58.test:42
+test/random/select/slt_good_59.test:28
+test/random/select/slt_good_6.test:49
+test/random/select/slt_good_60.test:38
+test/random/select/slt_good_61.test:47
+test/random/select/slt_good_62.test:53
+test/random/select/slt_good_63.test:33
+test/random/select/slt_good_64.test:47
+test/random/select/slt_good_65.test:42
+test/random/select/slt_good_66.test:43
+test/random/select/slt_good_67.test:48
+test/random/select/slt_good_68.test:49
+test/random/select/slt_good_69.test:36
+test/random/select/slt_good_7.test:54
+test/random/select/slt_good_70.test:45
+test/random/select/slt_good_71.test:49
+test/random/select/slt_good_72.test:30
+test/random/select/slt_good_73.test:31
+test/random/select/slt_good_74.test:34
+test/random/select/slt_good_75.test:47
+test/random/select/slt_good_76.test:36
+test/random/select/slt_good_77.test:41
+test/random/select/slt_good_78.test:34
+test/random/select/slt_good_79.test:49
+test/random/select/slt_good_8.test:46
+test/random/select/slt_good_80.test:33
+test/random/select/slt_good_81.test:33
+test/random/select/slt_good_82.test:37
+test/random/select/slt_good_83.test:31
+test/random/select/slt_good_84.test:36
+test/random/select/slt_good_85.test:38
+test/random/select/slt_good_86.test:34
+test/random/select/slt_good_87.test:31
+test/random/select/slt_good_88.test:43
+test/random/select/slt_good_89.test:45
+test/random/select/slt_good_9.test:47
+test/random/select/slt_good_90.test:33
+test/random/select/slt_good_91.test:27
+test/random/select/slt_good_92.test:36
+test/random/select/slt_good_93.test:34
+test/random/select/slt_good_94.test:35
+test/random/select/slt_good_95.test:30
+test/random/select/slt_good_96.test:40
+test/random/select/slt_good_97.test:43
+test/random/select/slt_good_98.test:33
+test/random/select/slt_good_99.test:33
+test/select1.test:6
+test/select2.test:12
+test/select3.test:42
+test/select4.test:362

Reply via email to