You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@calcite.apache.org by "mbudiu-vmw (via GitHub)" <gi...@apache.org> on 2023/04/12 00:03:21 UTC

[GitHub] [calcite] mbudiu-vmw opened a new pull request, #3145: Program to run SQL Logic Tests for Calcite

mbudiu-vmw opened a new pull request, #3145:
URL: https://github.com/apache/calcite/pull/3145

   This PR essentially adds more than 7 million SQL tests to Calcite, from the SQL Logic Test Suite built by sqlite.
   
   This addresses https://issues.apache.org/jira/browse/CALCITE-5615
   
   This is a larger PR but it is for a completely self-contained program with a main entry point.
   The README.md contained in this PR documents the program structure and its intended usage.
   Using this program will require users to install separately the tests from the SQL Logic Test Suite: https://www.sqlite.org/sqllogictest/doc/trunk/about.wiki  
   
   The testing program finds multiple bugs in the default configuration of the Calcite compiler, both crashes and incorrect outputs. The plan is to file separate issues for each of these bugs as they are found.
   However, this program is also intended to be extended to test other Calcite-based compilers (or it could be used for other SQL query engines; there is nothing special about Calcite in the implementation.)


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@calcite.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [calcite] mihaibudiu commented on a diff in pull request #3145: [CALCITE-5615] Program to run SQL Logic Tests for Calcite

Posted by "mihaibudiu (via GitHub)" <gi...@apache.org>.
mihaibudiu commented on code in PR #3145:
URL: https://github.com/apache/calcite/pull/3145#discussion_r1186546089


##########
slt/src/main/java/org/apache/calcite/slt/Main1.java:
##########
@@ -0,0 +1,77 @@
+/*
+ * 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 java.io.IOException;
+import java.io.PrintStream;
+import java.net.URL;
+import java.nio.file.Files;
+import java.nio.file.Path;
+import java.nio.file.Paths;
+
+import net.hydromatic.sqllogictest.ExecutionOptions;
+import net.hydromatic.sqllogictest.executors.HsqldbExecutor;
+import net.hydromatic.sqllogictest.executors.NoExecutor;
+import net.hydromatic.sqllogictest.TestLoader;
+
+import org.apache.calcite.slt.executors.CalciteExecutor;
+
+/**
+ * Execute all SqlLogicTest tests.
+ */
+public class Main1 {
+  private Main1() {
+  }
+
+  public static void main(String[] argv) throws IOException {
+    execute(true, System.out, System.err, argv);
+  }
+
+  /** As {@link #main} but does not call {@link System#exit} if {@code exit}
+   * is false. */
+  public static int execute(boolean exit, PrintStream out, PrintStream err,
+      String... argv) throws IOException {
+    ExecutionOptions options = new ExecutionOptions(exit, out, err);
+    options.setBinaryName("slt");
+    NoExecutor.register(options);
+    HsqldbExecutor.register(options);
+    CalciteExecutor.register(options);
+    int parse = options.parse(argv);
+    if (parse != 0) {
+      return parse;
+    }
+
+    /*
+    URL r = Thread.currentThread().getContextClassLoader().getResource("test");
+    if (r == null) {
+      out.println("Cannot find resources");
+      return 1;
+    }
+     */
+    URL r = new URL("/home/mbudiu/git/sqllogictest/test");

Review Comment:
   Sorry, that's a bug left over from testing, the commented-out code should be used instead.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@calcite.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [calcite] julianhyde commented on pull request #3145: [CALCITE-5615] Program to run SQL Logic Tests for Calcite

Posted by "julianhyde (via GitHub)" <gi...@apache.org>.
julianhyde commented on PR #3145:
URL: https://github.com/apache/calcite/pull/3145#issuecomment-1536561051

   I think this code could go into the existing `plus` module, not create a new `slt` module. Calcite has too many modules already.


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@calcite.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [calcite] mihaibudiu commented on pull request #3145: [CALCITE-5615] Program to run SQL Logic Tests for Calcite

Posted by "mihaibudiu (via GitHub)" <gi...@apache.org>.
mihaibudiu commented on PR #3145:
URL: https://github.com/apache/calcite/pull/3145#issuecomment-1614887197

   "Very slow" sounds fine to me. This could be run just prior to a release or maybe once a month.
   The tool is really designed for human-in-the-loop: someone has to find the bugs and file issues.
   I want to add some capabilities to the sql-logic-test project to make this task easier (https://github.com/hydromatic/sql-logic-test/issues/22). I hope to do it in the next few days.
   The main question is what to do when this tool finds regressions. Probably we need to bisect the commits to find the source of regressions.


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@calcite.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [calcite] mbudiu-vmw commented on pull request #3145: [CALCITE-5615] Program to run SQL Logic Tests for Calcite

Posted by "mbudiu-vmw (via GitHub)" <gi...@apache.org>.
mbudiu-vmw commented on PR #3145:
URL: https://github.com/apache/calcite/pull/3145#issuecomment-1509045584

   I will only address @zabetak 's comments once we understand how this should be packaged; I have sent an email to the dev mailing list asking for suggestions. One thing I would encourage people to do in the meantime is to actually run the code (first on one or two test files, these are specified on the command-line, e.g., `select1.test`) and look at the bugs found. 


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@calcite.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [calcite] zabetak commented on pull request #3145: [CALCITE-5615] Program to run SQL Logic Tests for Calcite

Posted by "zabetak (via GitHub)" <gi...@apache.org>.
zabetak commented on PR #3145:
URL: https://github.com/apache/calcite/pull/3145#issuecomment-1537089353

   > But I would appreciate help to decide how to package this tool.
   
   @mihaibudiu From my perspective (and Julian's I think) this PR should contain something equivalent to what is now `TestCalcite` inside the `plus` module and allow all SQL logic tests that pass to run. It should probably contain `CalciteExecutor` but nothing more. We just want to run to be able to run the tests as part of the Calcite suite but nothing more for now. If you want I can push the relevant commits following this idea in this PR.


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@calcite.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [calcite] mihaibudiu commented on pull request #3145: [CALCITE-5615] Program to run SQL Logic Tests for Calcite

Posted by "mihaibudiu (via GitHub)" <gi...@apache.org>.
mihaibudiu commented on PR #3145:
URL: https://github.com/apache/calcite/pull/3145#issuecomment-1603442065

   I hope we're close to a working solution for this PR.
   You will notice that the CalciteExecutor has been reduced to just a few lines of code. 
   Most of the complexity is in the test code - packaging a very large number of tests and failures into a compact representation. The README.md has details about how this tool can be used not only for regression testing but also for diagnosing bugs in Calcite.


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@calcite.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [calcite] julianhyde commented on pull request #3145: [CALCITE-5615] Program to run SQL Logic Tests for Calcite

Posted by "julianhyde (via GitHub)" <gi...@apache.org>.
julianhyde commented on PR #3145:
URL: https://github.com/apache/calcite/pull/3145#issuecomment-1581663993

   Can you get anything useful committed to Calcite without a new release of sql-logic-test? I have put a lot of effort into this project so far and I want to see some returns before I do another release.


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@calcite.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [calcite] mihaibudiu commented on pull request #3145: [CALCITE-5615] Program to run SQL Logic Tests for Calcite

Posted by "mihaibudiu (via GitHub)" <gi...@apache.org>.
mihaibudiu commented on PR #3145:
URL: https://github.com/apache/calcite/pull/3145#issuecomment-1613354307

   On my machine using 8 parallel threads the tests take about 5-6 hours.
   I am not familiar with the tiers of tests in Calcite, so I don't know what tag should be used., but if there is a slower one it should be even easier.


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@calcite.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [calcite] zabetak commented on a diff in pull request #3145: [CALCITE-5615] Program to run SQL Logic Tests for Calcite

Posted by "zabetak (via GitHub)" <gi...@apache.org>.
zabetak commented on code in PR #3145:
URL: https://github.com/apache/calcite/pull/3145#discussion_r1186452908


##########
slt/README.md:
##########
@@ -0,0 +1,72 @@
+<!--
+{% comment %}
+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.
+{% endcomment %}
+-->
+
+# SQL logic tests

Review Comment:
   We are not going to distribute any `slt` binaries as part of Calcite so we don't need this README.



##########
slt/src/main/java/org/apache/calcite/slt/Main1.java:
##########
@@ -0,0 +1,77 @@
+/*
+ * 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 java.io.IOException;
+import java.io.PrintStream;
+import java.net.URL;
+import java.nio.file.Files;
+import java.nio.file.Path;
+import java.nio.file.Paths;
+
+import net.hydromatic.sqllogictest.ExecutionOptions;
+import net.hydromatic.sqllogictest.executors.HsqldbExecutor;
+import net.hydromatic.sqllogictest.executors.NoExecutor;
+import net.hydromatic.sqllogictest.TestLoader;
+
+import org.apache.calcite.slt.executors.CalciteExecutor;
+
+/**
+ * Execute all SqlLogicTest tests.
+ */
+public class Main1 {

Review Comment:
   What's the purpose of this class? I don't see it used anywhere.



##########
slt/src/test/java/org/apache/calcite/slt/TestCalcite.java:
##########
@@ -0,0 +1,87 @@
+/*
+ * 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 java.io.ByteArrayOutputStream;
+import java.io.IOException;
+import java.io.PrintStream;
+import java.nio.charset.StandardCharsets;
+
+import org.junit.jupiter.api.Test;
+
+import static org.hamcrest.MatcherAssert.assertThat;
+import static org.hamcrest.core.Is.is;
+
+public class TestCalcite {

Review Comment:
   Probably we need to pick a name that reflects that we are running `sql-logic-test`; `TestCalcite` is quite general.



##########
slt/src/test/java/org/apache/calcite/slt/TestCalcite.java:
##########
@@ -0,0 +1,87 @@
+/*
+ * 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 java.io.ByteArrayOutputStream;
+import java.io.IOException;
+import java.io.PrintStream;
+import java.nio.charset.StandardCharsets;
+
+import org.junit.jupiter.api.Test;
+
+import static org.hamcrest.MatcherAssert.assertThat;
+import static org.hamcrest.core.Is.is;
+
+public class TestCalcite {
+  private static final String USAGE = ""

Review Comment:
   Is this used? Do we need this?



##########
settings.gradle.kts:
##########
@@ -83,7 +83,8 @@ include(
     "spark",
     "splunk",
     "testkit",
-    "ubenchmark"
+    "ubenchmark",
+    "slt"

Review Comment:
   I agree with Julian that we don't need a new module; putting the tests in plus makes sense.



##########
slt/src/main/java/org/apache/calcite/slt/Main1.java:
##########
@@ -0,0 +1,77 @@
+/*
+ * 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 java.io.IOException;
+import java.io.PrintStream;
+import java.net.URL;
+import java.nio.file.Files;
+import java.nio.file.Path;
+import java.nio.file.Paths;
+
+import net.hydromatic.sqllogictest.ExecutionOptions;
+import net.hydromatic.sqllogictest.executors.HsqldbExecutor;
+import net.hydromatic.sqllogictest.executors.NoExecutor;
+import net.hydromatic.sqllogictest.TestLoader;
+
+import org.apache.calcite.slt.executors.CalciteExecutor;
+
+/**
+ * Execute all SqlLogicTest tests.
+ */
+public class Main1 {
+  private Main1() {
+  }
+
+  public static void main(String[] argv) throws IOException {
+    execute(true, System.out, System.err, argv);
+  }
+
+  /** As {@link #main} but does not call {@link System#exit} if {@code exit}
+   * is false. */
+  public static int execute(boolean exit, PrintStream out, PrintStream err,
+      String... argv) throws IOException {
+    ExecutionOptions options = new ExecutionOptions(exit, out, err);
+    options.setBinaryName("slt");
+    NoExecutor.register(options);
+    HsqldbExecutor.register(options);
+    CalciteExecutor.register(options);
+    int parse = options.parse(argv);
+    if (parse != 0) {
+      return parse;
+    }
+
+    /*
+    URL r = Thread.currentThread().getContextClassLoader().getResource("test");
+    if (r == null) {
+      out.println("Cannot find resources");
+      return 1;
+    }
+     */
+    URL r = new URL("/home/mbudiu/git/sqllogictest/test");

Review Comment:
   This path to a local directory seems really suspicious. I am not sure what's the purpose of this class but if we really need it we will have to figure a way to make it work without assuming that test files exist in a local directory.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@calcite.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [calcite] mihaibudiu commented on a diff in pull request #3145: [CALCITE-5615] Program to run SQL Logic Tests for Calcite

Posted by "mihaibudiu (via GitHub)" <gi...@apache.org>.
mihaibudiu commented on code in PR #3145:
URL: https://github.com/apache/calcite/pull/3145#discussion_r1186546893


##########
slt/src/test/java/org/apache/calcite/slt/TestCalcite.java:
##########
@@ -0,0 +1,87 @@
+/*
+ * 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 java.io.ByteArrayOutputStream;
+import java.io.IOException;
+import java.io.PrintStream;
+import java.nio.charset.StandardCharsets;
+
+import org.junit.jupiter.api.Test;
+
+import static org.hamcrest.MatcherAssert.assertThat;
+import static org.hamcrest.core.Is.is;
+
+public class TestCalcite {

Review Comment:
   This class was created only to make sure that this project correctly extends sql-logic-test. 
   If we plan to integrate this tool into the Calcite testsuite we have to devise a better name and design for this class.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@calcite.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [calcite] zabetak commented on a diff in pull request #3145: [CALCITE-5615] Program to run SQL Logic Tests for Calcite

Posted by "zabetak (via GitHub)" <gi...@apache.org>.
zabetak commented on code in PR #3145:
URL: https://github.com/apache/calcite/pull/3145#discussion_r1201939308


##########
plus/build.gradle.kts:
##########
@@ -25,9 +25,11 @@ dependencies {
     implementation("com.teradata.tpcds:tpcds")
     implementation("io.prestosql.tpch:tpch")
     implementation("net.hydromatic:chinook-data-hsqldb")
+    implementation("net.hydromatic:sql-logic-test:0.2")
     implementation("net.hydromatic:tpcds")
     implementation("org.apache.calcite.avatica:avatica-server")
     implementation("org.hsqldb:hsqldb")
+    implementation("org.reflections:reflections:0.10.2")

Review Comment:
   Why do we need to declare this? Usually transitive dependencies should be discovered automatically?



##########
plus/src/test/java/org/apache/calcite/slt/TestCalcite.java:
##########
@@ -0,0 +1,86 @@
+/*
+ * 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 net.hydromatic.sqllogictest.OptionsParser;
+
+import org.junit.jupiter.api.Test;
+
+import java.io.ByteArrayOutputStream;
+import java.io.IOException;
+import java.io.PrintStream;
+import java.nio.charset.StandardCharsets;
+
+import static org.hamcrest.MatcherAssert.assertThat;
+import static org.hamcrest.core.Is.is;
+
+/**
+ * Tests using sql-logic-test suite.
+ */
+public class TestCalcite {

Review Comment:
   The class should be reworked. 
   
   1. We want to run all sql-logic-test that pass and not only `select1.test`. 
   2. When there is a regression we want to have a clear way to know which test and which file failed. (optional but useful)
   3. We only care about tests using the Calcite executor; anything else is out of context of this pr.



##########
plus/src/test/java/org/apache/calcite/slt/executors/CalciteExecutor.java:
##########
@@ -0,0 +1,170 @@
+/*
+ * 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.ISqlTestOperation;
+import net.hydromatic.sqllogictest.OptionsParser;
+import net.hydromatic.sqllogictest.SltSqlStatement;
+import net.hydromatic.sqllogictest.SltTestFile;
+import net.hydromatic.sqllogictest.SqlTestQuery;
+import net.hydromatic.sqllogictest.TestStatistics;
+import net.hydromatic.sqllogictest.executors.HsqldbExecutor;
+import net.hydromatic.sqllogictest.executors.JdbcExecutor;
+import net.hydromatic.sqllogictest.executors.SqlSltTestExecutor;
+
+import java.io.IOException;
+import java.security.NoSuchAlgorithmException;
+import java.sql.Connection;
+import java.sql.DriverManager;
+import java.sql.PreparedStatement;
+import java.sql.ResultSet;
+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 SqlSltTestExecutor {

Review Comment:
   I think I left some comments regarding this class in a previous review.



##########
plus/build.gradle.kts:
##########
@@ -25,9 +25,11 @@ dependencies {
     implementation("com.teradata.tpcds:tpcds")
     implementation("io.prestosql.tpch:tpch")
     implementation("net.hydromatic:chinook-data-hsqldb")
+    implementation("net.hydromatic:sql-logic-test:0.2")

Review Comment:
   Also probably the version ("0.2") should not be controlled here. That's why we have the gradle.properties file.



##########
plus/build.gradle.kts:
##########
@@ -25,9 +25,11 @@ dependencies {
     implementation("com.teradata.tpcds:tpcds")
     implementation("io.prestosql.tpch:tpch")
     implementation("net.hydromatic:chinook-data-hsqldb")
+    implementation("net.hydromatic:sql-logic-test:0.2")

Review Comment:
   Should this rather be a `testImplementation` dependency?



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@calcite.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [calcite] mihaibudiu commented on a diff in pull request #3145: [CALCITE-5615] Program to run SQL Logic Tests for Calcite

Posted by "mihaibudiu (via GitHub)" <gi...@apache.org>.
mihaibudiu commented on code in PR #3145:
URL: https://github.com/apache/calcite/pull/3145#discussion_r1266049814


##########
plus/src/test/java/org/apache/calcite/slt/SqlLogicTestsForCalciteTests.java:
##########
@@ -0,0 +1,312 @@
+/*
+ * 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 net.hydromatic.sqllogictest.OptionsParser;
+import net.hydromatic.sqllogictest.TestStatistics;
+
+import org.junit.jupiter.api.AfterAll;
+import org.junit.jupiter.api.Assertions;
+import org.junit.jupiter.api.BeforeAll;
+import org.junit.jupiter.api.Disabled;
+import org.junit.jupiter.api.DynamicTest;
+import org.junit.jupiter.api.Tag;
+import org.junit.jupiter.api.Test;
+import org.junit.jupiter.api.TestFactory;
+import org.junit.jupiter.api.function.Executable;
+
+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.OutputStreamWriter;
+import java.nio.charset.StandardCharsets;
+import java.nio.file.Files;
+import java.time.Duration;
+import java.util.ArrayList;
+import java.util.Comparator;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+
+import static org.hamcrest.CoreMatchers.is;
+import static org.hamcrest.CoreMatchers.notNullValue;
+import static org.hamcrest.MatcherAssert.assertThat;
+import static org.junit.jupiter.api.Assertions.assertTimeoutPreemptively;
+
+/**
+ * Tests using sql-logic-test suite.
+ */
+public class SqlLogicTestsForCalciteTests {
+  /**
+   * Short summary of the results of a test execution.
+   */
+  static class TestSummary {
+    /**
+     * File containing tests.
+     */
+    final String file;
+    /**
+     * Number of tests that have passed.
+     */
+    final int passed;
+    /**
+     * Number of tests that have failed.
+     */
+    final int failed;
+
+    TestSummary(String file, int passed, int failed) {
+      this.file = file;
+      this.passed = passed;
+      this.failed = failed;
+    }
+
+    /**
+     * Parse a TestSummary from a string.
+     * The inverse of 'toString'.
+     *
+     * @return The parsed TestSummary or null on failure.
+     */
+    static TestSummary parse(String line) {
+      String[] parts = line.split(":");
+      if (parts.length != 3) {
+        return null;
+      }
+      try {
+        int passed = Integer.parseInt(parts[1]);
+        int failed = Integer.parseInt(parts[2]);
+        return new TestSummary(parts[0], passed, failed);
+      } catch (NumberFormatException ex) {
+        return null;
+      }
+    }
+
+    @Override public String toString() {
+      return this.file + ":" + this.passed + ":" + this.failed;
+    }
+
+    /**
+     * Check if the 'other' TestSummaries are a regressions
+     * when compared to 'this'.
+     *
+     * @param other TestSummary to compare against.
+     * @return 'true' if 'other' is a regression from 'this'.
+     */
+    boolean regression(TestSummary other) {
+      return other.failed > this.failed;
+    }
+  }
+
+  /**
+   * Summary for all tests executed.
+   */
+  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);
+    }
+
+    void read(InputStream stream) throws IOException {
+      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 {
+            System.err.println("Could not parse line " + line);
+          }
+        });
+      }
+    }
+
+    /**
+     * Check if 'other' summaries have regressions compared to `this`.
+     *
+     * @return 'true' if other contains regressions.
+     * @param other  Test results to compare with.
+     *               'other' can contain only a subset of the tests.
+     */
+    boolean regression(AllTestSummaries other) {
+      boolean regression = false;
+      for (TestSummary summary: other.testResults.values()) {
+        TestSummary original = this.testResults.get(summary.file);
+        if (original == null) {
+          System.err.println("No historical data for test " + summary.file);

Review Comment:
   I will remove the prints, but the hydromatic project also prints stuff, and we currently pass System.out and System.err for the PrintStream values it expects in `launchSqlLogicTest`.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@calcite.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [calcite] julianhyde commented on pull request #3145: [CALCITE-5615] Program to run SQL Logic Tests for Calcite

Posted by "julianhyde (via GitHub)" <gi...@apache.org>.
julianhyde commented on PR #3145:
URL: https://github.com/apache/calcite/pull/3145#issuecomment-1581669286

   It takes about an hour of my weekend to make each release.


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@calcite.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [calcite] sonarcloud[bot] commented on pull request #3145: [CALCITE-5615] Program to run SQL Logic Tests for Calcite

Posted by "sonarcloud[bot] (via GitHub)" <gi...@apache.org>.
sonarcloud[bot] commented on PR #3145:
URL: https://github.com/apache/calcite/pull/3145#issuecomment-1560061072

   Kudos, SonarCloud Quality Gate passed!&nbsp; &nbsp; [![Quality Gate passed](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/QualityGateBadge/passed-16px.png 'Quality Gate passed')](https://sonarcloud.io/dashboard?id=apache_calcite&pullRequest=3145)
   
   [![Bug](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/bug-16px.png 'Bug')](https://sonarcloud.io/project/issues?id=apache_calcite&pullRequest=3145&resolved=false&types=BUG) [![A](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A-16px.png 'A')](https://sonarcloud.io/project/issues?id=apache_calcite&pullRequest=3145&resolved=false&types=BUG) [0 Bugs](https://sonarcloud.io/project/issues?id=apache_calcite&pullRequest=3145&resolved=false&types=BUG)  
   [![Vulnerability](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/vulnerability-16px.png 'Vulnerability')](https://sonarcloud.io/project/issues?id=apache_calcite&pullRequest=3145&resolved=false&types=VULNERABILITY) [![A](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A-16px.png 'A')](https://sonarcloud.io/project/issues?id=apache_calcite&pullRequest=3145&resolved=false&types=VULNERABILITY) [0 Vulnerabilities](https://sonarcloud.io/project/issues?id=apache_calcite&pullRequest=3145&resolved=false&types=VULNERABILITY)  
   [![Security Hotspot](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/security_hotspot-16px.png 'Security Hotspot')](https://sonarcloud.io/project/security_hotspots?id=apache_calcite&pullRequest=3145&resolved=false&types=SECURITY_HOTSPOT) [![A](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A-16px.png 'A')](https://sonarcloud.io/project/security_hotspots?id=apache_calcite&pullRequest=3145&resolved=false&types=SECURITY_HOTSPOT) [0 Security Hotspots](https://sonarcloud.io/project/security_hotspots?id=apache_calcite&pullRequest=3145&resolved=false&types=SECURITY_HOTSPOT)  
   [![Code Smell](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/code_smell-16px.png 'Code Smell')](https://sonarcloud.io/project/issues?id=apache_calcite&pullRequest=3145&resolved=false&types=CODE_SMELL) [![A](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A-16px.png 'A')](https://sonarcloud.io/project/issues?id=apache_calcite&pullRequest=3145&resolved=false&types=CODE_SMELL) [0 Code Smells](https://sonarcloud.io/project/issues?id=apache_calcite&pullRequest=3145&resolved=false&types=CODE_SMELL)
   
   [![No Coverage information](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/CoverageChart/NoCoverageInfo-16px.png 'No Coverage information')](https://sonarcloud.io/component_measures?id=apache_calcite&pullRequest=3145&metric=coverage&view=list) No Coverage information  
   [![No Duplication information](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/Duplications/NoDuplicationInfo-16px.png 'No Duplication information')](https://sonarcloud.io/component_measures?id=apache_calcite&pullRequest=3145&metric=duplicated_lines_density&view=list) No Duplication information
   
   


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@calcite.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [calcite] sonarcloud[bot] commented on pull request #3145: [CALCITE-5615] Program to run SQL Logic Tests for Calcite

Posted by "sonarcloud[bot] (via GitHub)" <gi...@apache.org>.
sonarcloud[bot] commented on PR #3145:
URL: https://github.com/apache/calcite/pull/3145#issuecomment-1624404155

   Kudos, SonarCloud Quality Gate passed!&nbsp; &nbsp; [![Quality Gate passed](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/QualityGateBadge/passed-16px.png 'Quality Gate passed')](https://sonarcloud.io/dashboard?id=apache_calcite&pullRequest=3145)
   
   [![Bug](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/bug-16px.png 'Bug')](https://sonarcloud.io/project/issues?id=apache_calcite&pullRequest=3145&resolved=false&types=BUG) [![A](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A-16px.png 'A')](https://sonarcloud.io/project/issues?id=apache_calcite&pullRequest=3145&resolved=false&types=BUG) [0 Bugs](https://sonarcloud.io/project/issues?id=apache_calcite&pullRequest=3145&resolved=false&types=BUG)  
   [![Vulnerability](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/vulnerability-16px.png 'Vulnerability')](https://sonarcloud.io/project/issues?id=apache_calcite&pullRequest=3145&resolved=false&types=VULNERABILITY) [![A](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A-16px.png 'A')](https://sonarcloud.io/project/issues?id=apache_calcite&pullRequest=3145&resolved=false&types=VULNERABILITY) [0 Vulnerabilities](https://sonarcloud.io/project/issues?id=apache_calcite&pullRequest=3145&resolved=false&types=VULNERABILITY)  
   [![Security Hotspot](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/security_hotspot-16px.png 'Security Hotspot')](https://sonarcloud.io/project/security_hotspots?id=apache_calcite&pullRequest=3145&resolved=false&types=SECURITY_HOTSPOT) [![A](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A-16px.png 'A')](https://sonarcloud.io/project/security_hotspots?id=apache_calcite&pullRequest=3145&resolved=false&types=SECURITY_HOTSPOT) [0 Security Hotspots](https://sonarcloud.io/project/security_hotspots?id=apache_calcite&pullRequest=3145&resolved=false&types=SECURITY_HOTSPOT)  
   [![Code Smell](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/code_smell-16px.png 'Code Smell')](https://sonarcloud.io/project/issues?id=apache_calcite&pullRequest=3145&resolved=false&types=CODE_SMELL) [![A](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A-16px.png 'A')](https://sonarcloud.io/project/issues?id=apache_calcite&pullRequest=3145&resolved=false&types=CODE_SMELL) [0 Code Smells](https://sonarcloud.io/project/issues?id=apache_calcite&pullRequest=3145&resolved=false&types=CODE_SMELL)
   
   [![No Coverage information](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/CoverageChart/NoCoverageInfo-16px.png 'No Coverage information')](https://sonarcloud.io/component_measures?id=apache_calcite&pullRequest=3145&metric=coverage&view=list) No Coverage information  
   [![No Duplication information](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/Duplications/NoDuplicationInfo-16px.png 'No Duplication information')](https://sonarcloud.io/component_measures?id=apache_calcite&pullRequest=3145&metric=duplicated_lines_density&view=list) No Duplication information
   
   


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@calcite.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [calcite] zabetak commented on a diff in pull request #3145: [CALCITE-5615] Program to run SQL Logic Tests for Calcite

Posted by "zabetak (via GitHub)" <gi...@apache.org>.
zabetak commented on code in PR #3145:
URL: https://github.com/apache/calcite/pull/3145#discussion_r1296921605


##########
plus/src/test/java/org/apache/calcite/slt/SqlLogicTests.java:
##########
@@ -0,0 +1,337 @@
+/*
+ * 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.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.junit.jupiter.api.function.Executable;
+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.OutputStreamWriter;
+import java.nio.charset.StandardCharsets;
+import java.nio.file.Files;
+import java.time.Duration;
+import java.util.ArrayList;
+import java.util.Comparator;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+
+import static org.hamcrest.CoreMatchers.is;
+import static org.hamcrest.CoreMatchers.notNullValue;
+import static org.hamcrest.MatcherAssert.assertThat;
+import static org.junit.jupiter.api.Assertions.assertTimeoutPreemptively;
+
+/**
+ * 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;
+    }
+
+    /**
+     * Check if 'other' summaries have regressions compared to `this`.
+     *
+     * @return 'true' if other contains regressions.
+     * @param other  Test results to compare with.
+     *               'other' can contain only a subset of the tests.
+     */
+    boolean regression(AllTestSummaries other) {
+      boolean regression = false;
+      for (TestSummary summary: other.testResults.values()) {
+        regression = regression || this.regression(summary);
+      }
+      return regression;
+    }
+
+    @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());
+      }
+    }
+
+    /**
+     * True if there is an entry for the specified test file.
+     *
+     * @param test Test file name.
+     */
+    public boolean contains(String test) {
+      return this.testResults.containsKey(test);
+    }
+  }
+
+  /**
+   * Summaries produced for the current run.
+   * Must be static since it is written by the `findRegressions`
+   * static method.
+   */
+  static AllTestSummaries testSummaries = new AllTestSummaries();
+
+  static final String GOLDEN_FILE = "/slttestfailures.txt";
+  /**
+   * Summaries checked-in as resources that we compare against.
+   */
+  static AllTestSummaries goldenTestSummaries =
+      new AllTestSummaries()
+          .read(SqlLogicTests.class.getResourceAsStream(GOLDEN_FILE));
+
+  private static TestStatistics launchSqlLogicTest(String... args) throws IOException {
+    OptionsParser options = new OptionsParser(false, System.out, System.err);
+    CalciteExecutor.register(options);
+    return net.hydromatic.sqllogictest.Main.execute(options, args);
+  }
+
+  TestSummary shortSummary(String file, TestStatistics statistics) {
+    return new TestSummary(file, statistics.getFailedTestCount());
+  }
+
+  /**
+   * The following tests currently timeout during execution.
+   * Technically these are Calcite bugs.
+   */
+  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.
+   */
+  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");
+
+  void runOneTestFile(String testFile) throws IOException {
+    if (timeout.contains(testFile)) {
+      return;
+    }
+    if (unsupported.contains(testFile)) {
+      return;
+    }
+
+    // The arguments below are command-line arguments for the sql-logic-test
+    // executable from the hydromatic project.  The verbosity of the
+    // output can be increased by adding more "-v" flags to the command-line.
+    // By increasing verbosity even more you can get in the output a complete stack trace
+    // for each error caused by an exception.
+    TestStatistics res = launchSqlLogicTest("-v", "-e", "calcite", testFile);
+    assertThat(res, notNullValue());
+    assertThat(res.getParseFailureCount(), is(0));
+    assertThat(res.getIgnoredTestCount(), is(0));
+    assertThat(res.getTestFileCount(), is(1));
+    res.printStatistics(System.err);  // Print errors found
+    TestSummary summary = this.shortSummary(testFile, res);
+    boolean regression = goldenTestSummaries.regression(summary);
+    Assumptions.assumeFalse(regression, "Regression in " + summary.file);

Review Comment:
   OK I understood, the assertThat are basically to ensure that the file was processes correctly.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@calcite.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [calcite] zabetak commented on pull request #3145: [CALCITE-5615] Program to run SQL Logic Tests for Calcite

Posted by "zabetak (via GitHub)" <gi...@apache.org>.
zabetak commented on PR #3145:
URL: https://github.com/apache/calcite/pull/3145#issuecomment-1612614757

   @mihaibudiu Thanks for pushing this forward. Indeed I think we are pretty close. 
   
   One basic point that we need to agree on before I do full review of the new part is the duration & frequency of running the new tests. I see that the new tests are tagged as `Slow`. Currently we run `Slow` tests on every commit that lands in master and without the changes under this PR they take ~30minutes.
   
   How much time do the new test take at the moment and how often do we want to run them? We should take into account that things are usually much faster locally than in CI.
   
   As I mentioned earlier (https://github.com/apache/calcite/pull/3145#issuecomment-1580300860) we don't need to address everything as part of this PR. However, I would like to have some of the tests enabled and running as part of the unit tests. 


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@calcite.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [calcite] mihaibudiu commented on pull request #3145: [CALCITE-5615] Program to run SQL Logic Tests for Calcite

Posted by "mihaibudiu (via GitHub)" <gi...@apache.org>.
mihaibudiu commented on PR #3145:
URL: https://github.com/apache/calcite/pull/3145#issuecomment-1563493613

   Currently the "execute" method from Main only returns an integer and writes all diagnostic to the supplied streams.
   We have to make it return the actual TestStatistics object.
   So this will require another release.


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@calcite.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [calcite] sonarcloud[bot] commented on pull request #3145: [CALCITE-5615] Program to run SQL Logic Tests for Calcite

Posted by "sonarcloud[bot] (via GitHub)" <gi...@apache.org>.
sonarcloud[bot] commented on PR #3145:
URL: https://github.com/apache/calcite/pull/3145#issuecomment-1507444453

   Kudos, SonarCloud Quality Gate passed!&nbsp; &nbsp; [![Quality Gate passed](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/QualityGateBadge/passed-16px.png 'Quality Gate passed')](https://sonarcloud.io/dashboard?id=apache_calcite&pullRequest=3145)
   
   [![Bug](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/bug-16px.png 'Bug')](https://sonarcloud.io/project/issues?id=apache_calcite&pullRequest=3145&resolved=false&types=BUG) [![A](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A-16px.png 'A')](https://sonarcloud.io/project/issues?id=apache_calcite&pullRequest=3145&resolved=false&types=BUG) [0 Bugs](https://sonarcloud.io/project/issues?id=apache_calcite&pullRequest=3145&resolved=false&types=BUG)  
   [![Vulnerability](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/vulnerability-16px.png 'Vulnerability')](https://sonarcloud.io/project/issues?id=apache_calcite&pullRequest=3145&resolved=false&types=VULNERABILITY) [![A](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A-16px.png 'A')](https://sonarcloud.io/project/issues?id=apache_calcite&pullRequest=3145&resolved=false&types=VULNERABILITY) [0 Vulnerabilities](https://sonarcloud.io/project/issues?id=apache_calcite&pullRequest=3145&resolved=false&types=VULNERABILITY)  
   [![Security Hotspot](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/security_hotspot-16px.png 'Security Hotspot')](https://sonarcloud.io/project/security_hotspots?id=apache_calcite&pullRequest=3145&resolved=false&types=SECURITY_HOTSPOT) [![A](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A-16px.png 'A')](https://sonarcloud.io/project/security_hotspots?id=apache_calcite&pullRequest=3145&resolved=false&types=SECURITY_HOTSPOT) [0 Security Hotspots](https://sonarcloud.io/project/security_hotspots?id=apache_calcite&pullRequest=3145&resolved=false&types=SECURITY_HOTSPOT)  
   [![Code Smell](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/code_smell-16px.png 'Code Smell')](https://sonarcloud.io/project/issues?id=apache_calcite&pullRequest=3145&resolved=false&types=CODE_SMELL) [![A](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A-16px.png 'A')](https://sonarcloud.io/project/issues?id=apache_calcite&pullRequest=3145&resolved=false&types=CODE_SMELL) [35 Code Smells](https://sonarcloud.io/project/issues?id=apache_calcite&pullRequest=3145&resolved=false&types=CODE_SMELL)
   
   [![0.0%](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/CoverageChart/0-16px.png '0.0%')](https://sonarcloud.io/component_measures?id=apache_calcite&pullRequest=3145&metric=new_coverage&view=list) [0.0% Coverage](https://sonarcloud.io/component_measures?id=apache_calcite&pullRequest=3145&metric=new_coverage&view=list)  
   [![0.0%](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/Duplications/3-16px.png '0.0%')](https://sonarcloud.io/component_measures?id=apache_calcite&pullRequest=3145&metric=new_duplicated_lines_density&view=list) [0.0% Duplication](https://sonarcloud.io/component_measures?id=apache_calcite&pullRequest=3145&metric=new_duplicated_lines_density&view=list)
   
   


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@calcite.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [calcite] mihaibudiu commented on pull request #3145: [CALCITE-5615] Program to run SQL Logic Tests for Calcite

Posted by "mihaibudiu (via GitHub)" <gi...@apache.org>.
mihaibudiu commented on PR #3145:
URL: https://github.com/apache/calcite/pull/3145#issuecomment-1581208831

   I think having many small tests is probably better. There are 622 test files in the project.
   We could write a small script to generate a test for each one (or a group of files).
   This could also be beneficial if the tests can run in parallel.
   
   However, JUnit5 has a capability called DynamicTests using the `@TestFactory` annotation, but Calcite does not currently use it: https://www.baeldung.com/junit5-dynamic-tests.
   I will try to prototype this solution to see what happens. With DynamicTests we could even choose how many tests to run by sampling the test file list.


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@calcite.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [calcite] mihaibudiu commented on pull request #3145: [CALCITE-5615] Program to run SQL Logic Tests for Calcite

Posted by "mihaibudiu (via GitHub)" <gi...@apache.org>.
mihaibudiu commented on PR #3145:
URL: https://github.com/apache/calcite/pull/3145#issuecomment-1624391721

   I wasn't exactly sure how to handle this, so what I have done is to:
   - run just one test file as a slow test, takes a couple of minutes
   - disable the full testsuite using the `@Disabled` annotation
   The idea is to run the testsuite by hand as desired, but not part of regular testing.
   Is this an acceptable solution?


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@calcite.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [calcite] zabetak commented on a diff in pull request #3145: [CALCITE-5615] Program to run SQL Logic Tests for Calcite

Posted by "zabetak (via GitHub)" <gi...@apache.org>.
zabetak commented on code in PR #3145:
URL: https://github.com/apache/calcite/pull/3145#discussion_r1296908112


##########
plus/src/test/java/org/apache/calcite/slt/SqlLogicTests.java:
##########
@@ -0,0 +1,337 @@
+/*
+ * 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.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.junit.jupiter.api.function.Executable;
+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.OutputStreamWriter;
+import java.nio.charset.StandardCharsets;
+import java.nio.file.Files;
+import java.time.Duration;
+import java.util.ArrayList;
+import java.util.Comparator;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+
+import static org.hamcrest.CoreMatchers.is;
+import static org.hamcrest.CoreMatchers.notNullValue;
+import static org.hamcrest.MatcherAssert.assertThat;
+import static org.junit.jupiter.api.Assertions.assertTimeoutPreemptively;
+
+/**
+ * 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

Review Comment:
   Please ignore, I am applying small fixes myself.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@calcite.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [calcite] mihaibudiu commented on a diff in pull request #3145: [CALCITE-5615] Program to run SQL Logic Tests for Calcite

Posted by "mihaibudiu (via GitHub)" <gi...@apache.org>.
mihaibudiu commented on code in PR #3145:
URL: https://github.com/apache/calcite/pull/3145#discussion_r1202916789


##########
plus/src/test/java/org/apache/calcite/slt/executors/CalciteExecutor.java:
##########
@@ -0,0 +1,170 @@
+/*
+ * 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.ISqlTestOperation;
+import net.hydromatic.sqllogictest.OptionsParser;
+import net.hydromatic.sqllogictest.SltSqlStatement;
+import net.hydromatic.sqllogictest.SltTestFile;
+import net.hydromatic.sqllogictest.SqlTestQuery;
+import net.hydromatic.sqllogictest.TestStatistics;
+import net.hydromatic.sqllogictest.executors.HsqldbExecutor;
+import net.hydromatic.sqllogictest.executors.JdbcExecutor;
+import net.hydromatic.sqllogictest.executors.SqlSltTestExecutor;
+
+import java.io.IOException;
+import java.security.NoSuchAlgorithmException;
+import java.sql.Connection;
+import java.sql.DriverManager;
+import java.sql.PreparedStatement;
+import java.sql.ResultSet;
+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 SqlSltTestExecutor {

Review Comment:
   I looked everywhere and I could not find other comments about this class. I apologize if I didn't address them. 



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@calcite.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [calcite] mihaibudiu commented on pull request #3145: [CALCITE-5615] Program to run SQL Logic Tests for Calcite

Posted by "mihaibudiu (via GitHub)" <gi...@apache.org>.
mihaibudiu commented on PR #3145:
URL: https://github.com/apache/calcite/pull/3145#issuecomment-1581668067

   Well, you said "release often". Is there something that prevents a new release of sql-logic-test?
   Unfortunately the code was missing some getters for non-public members and I don't know how to work around that.
   Only by writing the CalciteExecutor code I figured out the missing features.
   (Our own compiler also could use the new features in sql-logic-test; I have converted our implementation to use it.)
   I am more worried about this PR: is this way of running the tests useful? These tests take a fairly long time and only print diagnostic on stderr about the encountered bugs, but they can't really fail.


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@calcite.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [calcite] mihaibudiu commented on pull request #3145: [CALCITE-5615] Program to run SQL Logic Tests for Calcite

Posted by "mihaibudiu (via GitHub)" <gi...@apache.org>.
mihaibudiu commented on PR #3145:
URL: https://github.com/apache/calcite/pull/3145#issuecomment-1581672584

   I apologize, didn't realize it's that involved. Any way I can help?
   I hope this will be the last one for a while.
   


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@calcite.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [calcite] sonarcloud[bot] commented on pull request #3145: [CALCITE-5615] Program to run SQL Logic Tests for Calcite

Posted by "sonarcloud[bot] (via GitHub)" <gi...@apache.org>.
sonarcloud[bot] commented on PR #3145:
URL: https://github.com/apache/calcite/pull/3145#issuecomment-1507396474

   Kudos, SonarCloud Quality Gate passed!&nbsp; &nbsp; [![Quality Gate passed](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/QualityGateBadge/passed-16px.png 'Quality Gate passed')](https://sonarcloud.io/dashboard?id=apache_calcite&pullRequest=3145)
   
   [![Bug](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/bug-16px.png 'Bug')](https://sonarcloud.io/project/issues?id=apache_calcite&pullRequest=3145&resolved=false&types=BUG) [![E](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/E-16px.png 'E')](https://sonarcloud.io/project/issues?id=apache_calcite&pullRequest=3145&resolved=false&types=BUG) [2 Bugs](https://sonarcloud.io/project/issues?id=apache_calcite&pullRequest=3145&resolved=false&types=BUG)  
   [![Vulnerability](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/vulnerability-16px.png 'Vulnerability')](https://sonarcloud.io/project/issues?id=apache_calcite&pullRequest=3145&resolved=false&types=VULNERABILITY) [![E](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/E-16px.png 'E')](https://sonarcloud.io/project/issues?id=apache_calcite&pullRequest=3145&resolved=false&types=VULNERABILITY) [1 Vulnerability](https://sonarcloud.io/project/issues?id=apache_calcite&pullRequest=3145&resolved=false&types=VULNERABILITY)  
   [![Security Hotspot](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/security_hotspot-16px.png 'Security Hotspot')](https://sonarcloud.io/project/security_hotspots?id=apache_calcite&pullRequest=3145&resolved=false&types=SECURITY_HOTSPOT) [![A](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A-16px.png 'A')](https://sonarcloud.io/project/security_hotspots?id=apache_calcite&pullRequest=3145&resolved=false&types=SECURITY_HOTSPOT) [0 Security Hotspots](https://sonarcloud.io/project/security_hotspots?id=apache_calcite&pullRequest=3145&resolved=false&types=SECURITY_HOTSPOT)  
   [![Code Smell](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/code_smell-16px.png 'Code Smell')](https://sonarcloud.io/project/issues?id=apache_calcite&pullRequest=3145&resolved=false&types=CODE_SMELL) [![A](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A-16px.png 'A')](https://sonarcloud.io/project/issues?id=apache_calcite&pullRequest=3145&resolved=false&types=CODE_SMELL) [41 Code Smells](https://sonarcloud.io/project/issues?id=apache_calcite&pullRequest=3145&resolved=false&types=CODE_SMELL)
   
   [![0.0%](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/CoverageChart/0-16px.png '0.0%')](https://sonarcloud.io/component_measures?id=apache_calcite&pullRequest=3145&metric=new_coverage&view=list) [0.0% Coverage](https://sonarcloud.io/component_measures?id=apache_calcite&pullRequest=3145&metric=new_coverage&view=list)  
   [![0.0%](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/Duplications/3-16px.png '0.0%')](https://sonarcloud.io/component_measures?id=apache_calcite&pullRequest=3145&metric=new_duplicated_lines_density&view=list) [0.0% Duplication](https://sonarcloud.io/component_measures?id=apache_calcite&pullRequest=3145&metric=new_duplicated_lines_density&view=list)
   
   


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@calcite.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [calcite] zabetak commented on pull request #3145: [CALCITE-5615] Program to run SQL Logic Tests for Calcite

Posted by "zabetak (via GitHub)" <gi...@apache.org>.
zabetak commented on PR #3145:
URL: https://github.com/apache/calcite/pull/3145#issuecomment-1623394466

   The only blocking thing for getting this PR merged is putting in place the "Fast" and "Slow" bucket. The "Very Slow" bucket as well as finding and reporting regressions can be tackled in a follow-up discussion/JIRA ticket.


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@calcite.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [calcite] sonarcloud[bot] commented on pull request #3145: [CALCITE-5615] Program to run SQL Logic Tests for Calcite

Posted by "sonarcloud[bot] (via GitHub)" <gi...@apache.org>.
sonarcloud[bot] commented on PR #3145:
URL: https://github.com/apache/calcite/pull/3145#issuecomment-1639378840

   Kudos, SonarCloud Quality Gate passed!&nbsp; &nbsp; [![Quality Gate passed](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/QualityGateBadge/passed-16px.png 'Quality Gate passed')](https://sonarcloud.io/dashboard?id=apache_calcite&pullRequest=3145)
   
   [![Bug](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/bug-16px.png 'Bug')](https://sonarcloud.io/project/issues?id=apache_calcite&pullRequest=3145&resolved=false&types=BUG) [![A](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A-16px.png 'A')](https://sonarcloud.io/project/issues?id=apache_calcite&pullRequest=3145&resolved=false&types=BUG) [0 Bugs](https://sonarcloud.io/project/issues?id=apache_calcite&pullRequest=3145&resolved=false&types=BUG)  
   [![Vulnerability](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/vulnerability-16px.png 'Vulnerability')](https://sonarcloud.io/project/issues?id=apache_calcite&pullRequest=3145&resolved=false&types=VULNERABILITY) [![A](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A-16px.png 'A')](https://sonarcloud.io/project/issues?id=apache_calcite&pullRequest=3145&resolved=false&types=VULNERABILITY) [0 Vulnerabilities](https://sonarcloud.io/project/issues?id=apache_calcite&pullRequest=3145&resolved=false&types=VULNERABILITY)  
   [![Security Hotspot](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/security_hotspot-16px.png 'Security Hotspot')](https://sonarcloud.io/project/security_hotspots?id=apache_calcite&pullRequest=3145&resolved=false&types=SECURITY_HOTSPOT) [![A](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A-16px.png 'A')](https://sonarcloud.io/project/security_hotspots?id=apache_calcite&pullRequest=3145&resolved=false&types=SECURITY_HOTSPOT) [0 Security Hotspots](https://sonarcloud.io/project/security_hotspots?id=apache_calcite&pullRequest=3145&resolved=false&types=SECURITY_HOTSPOT)  
   [![Code Smell](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/code_smell-16px.png 'Code Smell')](https://sonarcloud.io/project/issues?id=apache_calcite&pullRequest=3145&resolved=false&types=CODE_SMELL) [![A](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A-16px.png 'A')](https://sonarcloud.io/project/issues?id=apache_calcite&pullRequest=3145&resolved=false&types=CODE_SMELL) [0 Code Smells](https://sonarcloud.io/project/issues?id=apache_calcite&pullRequest=3145&resolved=false&types=CODE_SMELL)
   
   [![No Coverage information](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/CoverageChart/NoCoverageInfo-16px.png 'No Coverage information')](https://sonarcloud.io/component_measures?id=apache_calcite&pullRequest=3145&metric=coverage&view=list) No Coverage information  
   [![No Duplication information](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/Duplications/NoDuplicationInfo-16px.png 'No Duplication information')](https://sonarcloud.io/component_measures?id=apache_calcite&pullRequest=3145&metric=duplicated_lines_density&view=list) No Duplication information
   
   


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@calcite.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [calcite] mihaibudiu commented on pull request #3145: [CALCITE-5615] Program to run SQL Logic Tests for Calcite

Posted by "mihaibudiu (via GitHub)" <gi...@apache.org>.
mihaibudiu commented on PR #3145:
URL: https://github.com/apache/calcite/pull/3145#issuecomment-1572795640

   It took about 6 minutes to run 10K tests using CalciteExecutor, so by extrapolation it would take about 2 days to run all tests.
   Memory consumption increases slightly with time. The failures are saved in a list which keeps growing. I am not sure whether there could be leaks somewhere else.


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@calcite.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [calcite] zabetak commented on a diff in pull request #3145: [CALCITE-5615] Program to run SQL Logic Tests for Calcite

Posted by "zabetak (via GitHub)" <gi...@apache.org>.
zabetak commented on code in PR #3145:
URL: https://github.com/apache/calcite/pull/3145#discussion_r1265184045


##########
plus/src/test/java/org/apache/calcite/slt/README.md:
##########
@@ -0,0 +1,130 @@
+<!--
+{% comment %}
+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.
+{% endcomment %}
+-->
+
+# Testing Calcite using SQL Logic Tests
+
+[SQL Logic Tests](https://www.sqlite.org/sqllogictest/doc/trunk/about.wiki) is public domain suite of several million SQL tests
+created as part of the sqlite engine.
+The project [hydromatic/sql-logic-test](https://github.com/hydromatic/sql-logic-test/)
+has packaged these tests into a Java framework which makes it easy to
+run them against a JDBC provider.
+
+## How does it work
+
+In this project we test Calcite as a JDBC provider, coupled with [HSQLDB](http://hsqldb.org/) for storage.
+Another storage layer, such as [Postgres](https://www.postgresql.org), 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.)
+
+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.
+
+A small number of tests fail because they use statements currently not supported by
+HSQLDB (these are listed in the `SqlLogicTestsForCalciteTests.unsupported` set).
+
+For each test file we have summarized the number of passed and failed tests 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 we do not keep tab of the actual

Review Comment:
   's/tab/track/'



##########
plus/src/test/java/org/apache/calcite/slt/SqlLogicTestsForCalciteTests.java:
##########
@@ -0,0 +1,312 @@
+/*
+ * 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 net.hydromatic.sqllogictest.OptionsParser;
+import net.hydromatic.sqllogictest.TestStatistics;
+
+import org.junit.jupiter.api.AfterAll;
+import org.junit.jupiter.api.Assertions;
+import org.junit.jupiter.api.BeforeAll;
+import org.junit.jupiter.api.Disabled;
+import org.junit.jupiter.api.DynamicTest;
+import org.junit.jupiter.api.Tag;
+import org.junit.jupiter.api.Test;
+import org.junit.jupiter.api.TestFactory;
+import org.junit.jupiter.api.function.Executable;
+
+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.OutputStreamWriter;
+import java.nio.charset.StandardCharsets;
+import java.nio.file.Files;
+import java.time.Duration;
+import java.util.ArrayList;
+import java.util.Comparator;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+
+import static org.hamcrest.CoreMatchers.is;
+import static org.hamcrest.CoreMatchers.notNullValue;
+import static org.hamcrest.MatcherAssert.assertThat;
+import static org.junit.jupiter.api.Assertions.assertTimeoutPreemptively;
+
+/**
+ * Tests using sql-logic-test suite.
+ */
+public class SqlLogicTestsForCalciteTests {
+  /**
+   * Short summary of the results of a test execution.
+   */
+  static class TestSummary {
+    /**
+     * File containing tests.
+     */
+    final String file;
+    /**
+     * Number of tests that have passed.
+     */
+    final int passed;
+    /**
+     * Number of tests that have failed.
+     */
+    final int failed;
+
+    TestSummary(String file, int passed, int failed) {
+      this.file = file;
+      this.passed = passed;
+      this.failed = failed;
+    }
+
+    /**
+     * Parse a TestSummary from a string.
+     * The inverse of 'toString'.
+     *
+     * @return The parsed TestSummary or null on failure.
+     */
+    static TestSummary parse(String line) {
+      String[] parts = line.split(":");
+      if (parts.length != 3) {
+        return null;
+      }
+      try {
+        int passed = Integer.parseInt(parts[1]);
+        int failed = Integer.parseInt(parts[2]);
+        return new TestSummary(parts[0], passed, failed);
+      } catch (NumberFormatException ex) {
+        return null;
+      }
+    }
+
+    @Override public String toString() {
+      return this.file + ":" + this.passed + ":" + this.failed;
+    }
+
+    /**
+     * Check if the 'other' TestSummaries are a regressions
+     * when compared to 'this'.
+     *
+     * @param other TestSummary to compare against.
+     * @return 'true' if 'other' is a regression from 'this'.
+     */
+    boolean regression(TestSummary other) {
+      return other.failed > this.failed;

Review Comment:
   If we don't care about the number of passed tests then why keep it and carry it around ?



##########
plus/src/test/java/org/apache/calcite/slt/SqlLogicTestsForCalciteTests.java:
##########
@@ -0,0 +1,312 @@
+/*
+ * 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 net.hydromatic.sqllogictest.OptionsParser;
+import net.hydromatic.sqllogictest.TestStatistics;
+
+import org.junit.jupiter.api.AfterAll;
+import org.junit.jupiter.api.Assertions;
+import org.junit.jupiter.api.BeforeAll;
+import org.junit.jupiter.api.Disabled;
+import org.junit.jupiter.api.DynamicTest;
+import org.junit.jupiter.api.Tag;
+import org.junit.jupiter.api.Test;
+import org.junit.jupiter.api.TestFactory;
+import org.junit.jupiter.api.function.Executable;
+
+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.OutputStreamWriter;
+import java.nio.charset.StandardCharsets;
+import java.nio.file.Files;
+import java.time.Duration;
+import java.util.ArrayList;
+import java.util.Comparator;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+
+import static org.hamcrest.CoreMatchers.is;
+import static org.hamcrest.CoreMatchers.notNullValue;
+import static org.hamcrest.MatcherAssert.assertThat;
+import static org.junit.jupiter.api.Assertions.assertTimeoutPreemptively;
+
+/**
+ * Tests using sql-logic-test suite.
+ */
+public class SqlLogicTestsForCalciteTests {
+  /**
+   * Short summary of the results of a test execution.
+   */
+  static class TestSummary {
+    /**
+     * File containing tests.
+     */
+    final String file;
+    /**
+     * Number of tests that have passed.
+     */
+    final int passed;
+    /**
+     * Number of tests that have failed.
+     */
+    final int failed;
+
+    TestSummary(String file, int passed, int failed) {
+      this.file = file;
+      this.passed = passed;
+      this.failed = failed;
+    }
+
+    /**
+     * Parse a TestSummary from a string.
+     * The inverse of 'toString'.
+     *
+     * @return The parsed TestSummary or null on failure.
+     */
+    static TestSummary parse(String line) {
+      String[] parts = line.split(":");
+      if (parts.length != 3) {
+        return null;
+      }
+      try {
+        int passed = Integer.parseInt(parts[1]);
+        int failed = Integer.parseInt(parts[2]);
+        return new TestSummary(parts[0], passed, failed);
+      } catch (NumberFormatException ex) {
+        return null;
+      }
+    }
+
+    @Override public String toString() {
+      return this.file + ":" + this.passed + ":" + this.failed;
+    }
+
+    /**
+     * Check if the 'other' TestSummaries are a regressions
+     * when compared to 'this'.
+     *
+     * @param other TestSummary to compare against.
+     * @return 'true' if 'other' is a regression from 'this'.
+     */
+    boolean regression(TestSummary other) {
+      return other.failed > this.failed;
+    }
+  }
+
+  /**
+   * Summary for all tests executed.
+   */
+  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);
+    }
+
+    void read(InputStream stream) throws IOException {
+      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 {
+            System.err.println("Could not parse line " + line);
+          }
+        });
+      }
+    }
+
+    /**
+     * Check if 'other' summaries have regressions compared to `this`.
+     *
+     * @return 'true' if other contains regressions.
+     * @param other  Test results to compare with.
+     *               'other' can contain only a subset of the tests.
+     */
+    boolean regression(AllTestSummaries other) {
+      boolean regression = false;
+      for (TestSummary summary: other.testResults.values()) {
+        TestSummary original = this.testResults.get(summary.file);
+        if (original == null) {
+          System.err.println("No historical data for test " + summary.file);
+          continue;
+        }
+        if (original.regression(summary)) {
+          System.err.println("Regression: " + original.file
+              + " had " + original.failed + " failures, now has " + summary.failed);
+          regression = true;
+        }
+      }
+      return regression;
+    }
+
+    @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());
+      }
+    }
+
+    /**
+     * True if there is an entry for the specified test file.
+     *
+     * @param test Test file name.
+     */
+    public boolean contains(String test) {
+      return this.testResults.containsKey(test);
+    }
+  }
+
+  /**
+   * Summaries produced for the current run.
+   * Must be static since it is written by the `findRegressions`
+   * static method.
+   */
+  static AllTestSummaries testSummaries = new AllTestSummaries();
+  /**
+   * Summaries checked-in as resources that we compare against.
+   */
+  static AllTestSummaries goldenTestSummaries = new AllTestSummaries();
+
+  static final String GOLDENFILE = "/slttestfailures.txt";
+
+  private static TestStatistics launchSqlLogicTest(String... args) throws IOException {
+    OptionsParser options = new OptionsParser(false, System.out, System.err);
+    CalciteExecutor.register(options);
+    return net.hydromatic.sqllogictest.Main.execute(options, args);
+  }
+
+  TestSummary shortSummary(String file, TestStatistics statistics) {
+    return new TestSummary(file, statistics.getPassedTestCount(), statistics.getFailedTestCount());
+  }
+
+  // The following tests currently timeout during execution.
+  // Technically these are Calcite bugs.
+  Set<String> timeout = new HashSet<String>() {
+    {
+      add("test/select5.test");
+      add("test/random/groupby/slt_good_10.test");
+    }
+  };
+
+  // The following tests contain SQL statements that are not supported by HSQLDB

Review Comment:
   Idem with above.



##########
plus/src/test/java/org/apache/calcite/slt/SqlLogicTestsForCalciteTests.java:
##########
@@ -0,0 +1,312 @@
+/*
+ * 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 net.hydromatic.sqllogictest.OptionsParser;
+import net.hydromatic.sqllogictest.TestStatistics;
+
+import org.junit.jupiter.api.AfterAll;
+import org.junit.jupiter.api.Assertions;
+import org.junit.jupiter.api.BeforeAll;
+import org.junit.jupiter.api.Disabled;
+import org.junit.jupiter.api.DynamicTest;
+import org.junit.jupiter.api.Tag;
+import org.junit.jupiter.api.Test;
+import org.junit.jupiter.api.TestFactory;
+import org.junit.jupiter.api.function.Executable;
+
+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.OutputStreamWriter;
+import java.nio.charset.StandardCharsets;
+import java.nio.file.Files;
+import java.time.Duration;
+import java.util.ArrayList;
+import java.util.Comparator;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+
+import static org.hamcrest.CoreMatchers.is;
+import static org.hamcrest.CoreMatchers.notNullValue;
+import static org.hamcrest.MatcherAssert.assertThat;
+import static org.junit.jupiter.api.Assertions.assertTimeoutPreemptively;
+
+/**
+ * Tests using sql-logic-test suite.
+ */
+public class SqlLogicTestsForCalciteTests {
+  /**
+   * Short summary of the results of a test execution.
+   */
+  static class TestSummary {
+    /**
+     * File containing tests.
+     */
+    final String file;
+    /**
+     * Number of tests that have passed.
+     */
+    final int passed;
+    /**
+     * Number of tests that have failed.
+     */
+    final int failed;
+
+    TestSummary(String file, int passed, int failed) {
+      this.file = file;
+      this.passed = passed;
+      this.failed = failed;
+    }
+
+    /**
+     * Parse a TestSummary from a string.
+     * The inverse of 'toString'.
+     *
+     * @return The parsed TestSummary or null on failure.
+     */
+    static TestSummary parse(String line) {
+      String[] parts = line.split(":");
+      if (parts.length != 3) {
+        return null;
+      }
+      try {
+        int passed = Integer.parseInt(parts[1]);
+        int failed = Integer.parseInt(parts[2]);
+        return new TestSummary(parts[0], passed, failed);
+      } catch (NumberFormatException ex) {
+        return null;
+      }
+    }
+
+    @Override public String toString() {
+      return this.file + ":" + this.passed + ":" + this.failed;
+    }
+
+    /**
+     * Check if the 'other' TestSummaries are a regressions
+     * when compared to 'this'.
+     *
+     * @param other TestSummary to compare against.
+     * @return 'true' if 'other' is a regression from 'this'.
+     */
+    boolean regression(TestSummary other) {
+      return other.failed > this.failed;
+    }
+  }
+
+  /**
+   * Summary for all tests executed.
+   */
+  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);
+    }
+
+    void read(InputStream stream) throws IOException {
+      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 {
+            System.err.println("Could not parse line " + line);
+          }
+        });
+      }
+    }
+
+    /**
+     * Check if 'other' summaries have regressions compared to `this`.
+     *
+     * @return 'true' if other contains regressions.
+     * @param other  Test results to compare with.
+     *               'other' can contain only a subset of the tests.
+     */
+    boolean regression(AllTestSummaries other) {
+      boolean regression = false;
+      for (TestSummary summary: other.testResults.values()) {
+        TestSummary original = this.testResults.get(summary.file);
+        if (original == null) {
+          System.err.println("No historical data for test " + summary.file);
+          continue;
+        }
+        if (original.regression(summary)) {
+          System.err.println("Regression: " + original.file
+              + " had " + original.failed + " failures, now has " + summary.failed);
+          regression = true;
+        }
+      }
+      return regression;
+    }
+
+    @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());
+      }
+    }
+
+    /**
+     * True if there is an entry for the specified test file.
+     *
+     * @param test Test file name.
+     */
+    public boolean contains(String test) {
+      return this.testResults.containsKey(test);
+    }
+  }
+
+  /**
+   * Summaries produced for the current run.
+   * Must be static since it is written by the `findRegressions`
+   * static method.
+   */
+  static AllTestSummaries testSummaries = new AllTestSummaries();
+  /**
+   * Summaries checked-in as resources that we compare against.
+   */
+  static AllTestSummaries goldenTestSummaries = new AllTestSummaries();
+
+  static final String GOLDENFILE = "/slttestfailures.txt";
+
+  private static TestStatistics launchSqlLogicTest(String... args) throws IOException {
+    OptionsParser options = new OptionsParser(false, System.out, System.err);
+    CalciteExecutor.register(options);
+    return net.hydromatic.sqllogictest.Main.execute(options, args);
+  }
+
+  TestSummary shortSummary(String file, TestStatistics statistics) {
+    return new TestSummary(file, statistics.getPassedTestCount(), statistics.getFailedTestCount());
+  }
+
+  // The following tests currently timeout during execution.
+  // Technically these are Calcite bugs.
+  Set<String> timeout = new HashSet<String>() {
+    {
+      add("test/select5.test");
+      add("test/random/groupby/slt_good_10.test");
+    }
+  };
+
+  // The following tests contain SQL statements that are not supported by HSQLDB
+  Set<String> unsupported = new HashSet<String>() {
+    {
+      add("test/evidence/slt_lang_replace.test");
+      add("test/evidence/slt_lang_createtrigger.test");
+      add("test/evidence/slt_lang_droptrigger.test");
+      add("test/evidence/slt_lang_update.test");
+      add("test/evidence/slt_lang_reindex.test");
+    }
+  };
+
+  void runOneTestFile(String testFile) throws IOException {
+    if (timeout.contains(testFile)) {
+      return;
+    }
+    if (unsupported.contains(testFile)) {
+      return;
+    }
+
+    TestStatistics res = launchSqlLogicTest("-v", "-e", "calcite", testFile);
+    assertThat(res, notNullValue());
+    assertThat(res.getParseFailureCount(), is(0));
+    assertThat(res.getIgnoredTestCount(), is(0));
+    assertThat(res.getTestFileCount(), is(1));
+    res.printStatistics(System.err);  // Print errors found
+    TestSummary summary = this.shortSummary(testFile, res);
+    testSummaries.add(summary);
+  }
+
+  @Test @Tag("slow")
+  public void runOneTestFile() throws IOException {
+    runOneTestFile("select1.test");
+  }
+
+  @TestFactory @Disabled("This takes very long, should be run manually")
+  List<DynamicTest> runAllTests() {
+    // Run in parallel each test file.
+    Set<String> tests = net.hydromatic.sqllogictest.Main.getTestList();
+    List<DynamicTest> result = new ArrayList<>();
+    for (String test: tests) {
+      Executable executable = new Executable() {
+        @Override public void execute() {
+          assertTimeoutPreemptively(Duration.ofMinutes(10), () -> runOneTestFile(test));
+        }
+      };
+      DynamicTest dynamicTest = DynamicTest.dynamicTest(test, executable);
+      result.add(dynamicTest);
+    }
+    return result;
+  }
+
+  @BeforeAll
+  public static void readGoldenFile() throws IOException {
+    // Read the statistics of the previously-failing tests
+    try (InputStream stream = SqlLogicTestsForCalciteTests.class.getResourceAsStream(GOLDENFILE)) {
+      goldenTestSummaries.read(stream);
+    }
+  }
+
+  @AfterAll
+  public static void findRegressions() throws IOException {
+    // Compare with failures produced by a previous execution
+
+    // Code used to create the golden file originally
+    // File file = new File(goldenFile);
+    // if (!file.exists()) {
+    //   testSummaries.writeToFile(file);
+    //   return;
+    // }

Review Comment:
   In general we avoid leaving commented out code in the repo. It is not that important to know how the first file was generated but it is important to document somewhere the process/steps to update the golden file when that is necessary (e.g., someone fixes a bug and wants to update the file).



##########
plus/src/test/java/org/apache/calcite/slt/SqlLogicTestsForCalciteTests.java:
##########
@@ -0,0 +1,312 @@
+/*
+ * 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 net.hydromatic.sqllogictest.OptionsParser;
+import net.hydromatic.sqllogictest.TestStatistics;
+
+import org.junit.jupiter.api.AfterAll;
+import org.junit.jupiter.api.Assertions;
+import org.junit.jupiter.api.BeforeAll;
+import org.junit.jupiter.api.Disabled;
+import org.junit.jupiter.api.DynamicTest;
+import org.junit.jupiter.api.Tag;
+import org.junit.jupiter.api.Test;
+import org.junit.jupiter.api.TestFactory;
+import org.junit.jupiter.api.function.Executable;
+
+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.OutputStreamWriter;
+import java.nio.charset.StandardCharsets;
+import java.nio.file.Files;
+import java.time.Duration;
+import java.util.ArrayList;
+import java.util.Comparator;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+
+import static org.hamcrest.CoreMatchers.is;
+import static org.hamcrest.CoreMatchers.notNullValue;
+import static org.hamcrest.MatcherAssert.assertThat;
+import static org.junit.jupiter.api.Assertions.assertTimeoutPreemptively;
+
+/**
+ * Tests using sql-logic-test suite.
+ */
+public class SqlLogicTestsForCalciteTests {

Review Comment:
   We are already in Calcite so `ForCalciteTests` is a bit redundant; let's keet it super simple and just use `SqlLogicTests`.



##########
plus/src/test/java/org/apache/calcite/slt/SqlLogicTestsForCalciteTests.java:
##########
@@ -0,0 +1,312 @@
+/*
+ * 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 net.hydromatic.sqllogictest.OptionsParser;
+import net.hydromatic.sqllogictest.TestStatistics;
+
+import org.junit.jupiter.api.AfterAll;
+import org.junit.jupiter.api.Assertions;
+import org.junit.jupiter.api.BeforeAll;
+import org.junit.jupiter.api.Disabled;
+import org.junit.jupiter.api.DynamicTest;
+import org.junit.jupiter.api.Tag;
+import org.junit.jupiter.api.Test;
+import org.junit.jupiter.api.TestFactory;
+import org.junit.jupiter.api.function.Executable;
+
+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.OutputStreamWriter;
+import java.nio.charset.StandardCharsets;
+import java.nio.file.Files;
+import java.time.Duration;
+import java.util.ArrayList;
+import java.util.Comparator;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+
+import static org.hamcrest.CoreMatchers.is;
+import static org.hamcrest.CoreMatchers.notNullValue;
+import static org.hamcrest.MatcherAssert.assertThat;
+import static org.junit.jupiter.api.Assertions.assertTimeoutPreemptively;
+
+/**
+ * Tests using sql-logic-test suite.
+ */
+public class SqlLogicTestsForCalciteTests {
+  /**
+   * Short summary of the results of a test execution.
+   */
+  static class TestSummary {
+    /**
+     * File containing tests.
+     */
+    final String file;
+    /**
+     * Number of tests that have passed.
+     */
+    final int passed;
+    /**
+     * Number of tests that have failed.
+     */
+    final int failed;
+
+    TestSummary(String file, int passed, int failed) {
+      this.file = file;
+      this.passed = passed;
+      this.failed = failed;
+    }
+
+    /**
+     * Parse a TestSummary from a string.
+     * The inverse of 'toString'.
+     *
+     * @return The parsed TestSummary or null on failure.
+     */
+    static TestSummary parse(String line) {
+      String[] parts = line.split(":");
+      if (parts.length != 3) {
+        return null;
+      }
+      try {
+        int passed = Integer.parseInt(parts[1]);
+        int failed = Integer.parseInt(parts[2]);
+        return new TestSummary(parts[0], passed, failed);
+      } catch (NumberFormatException ex) {
+        return null;
+      }
+    }
+
+    @Override public String toString() {
+      return this.file + ":" + this.passed + ":" + this.failed;
+    }
+
+    /**
+     * Check if the 'other' TestSummaries are a regressions
+     * when compared to 'this'.
+     *
+     * @param other TestSummary to compare against.
+     * @return 'true' if 'other' is a regression from 'this'.
+     */
+    boolean regression(TestSummary other) {
+      return other.failed > this.failed;
+    }
+  }
+
+  /**
+   * Summary for all tests executed.
+   */
+  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);
+    }
+
+    void read(InputStream stream) throws IOException {
+      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 {
+            System.err.println("Could not parse line " + line);
+          }
+        });
+      }
+    }
+
+    /**
+     * Check if 'other' summaries have regressions compared to `this`.
+     *
+     * @return 'true' if other contains regressions.
+     * @param other  Test results to compare with.
+     *               'other' can contain only a subset of the tests.
+     */
+    boolean regression(AllTestSummaries other) {
+      boolean regression = false;
+      for (TestSummary summary: other.testResults.values()) {
+        TestSummary original = this.testResults.get(summary.file);
+        if (original == null) {
+          System.err.println("No historical data for test " + summary.file);
+          continue;
+        }
+        if (original.regression(summary)) {
+          System.err.println("Regression: " + original.file
+              + " had " + original.failed + " failures, now has " + summary.failed);
+          regression = true;
+        }
+      }
+      return regression;
+    }
+
+    @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());
+      }
+    }
+
+    /**
+     * True if there is an entry for the specified test file.
+     *
+     * @param test Test file name.
+     */
+    public boolean contains(String test) {
+      return this.testResults.containsKey(test);
+    }
+  }
+
+  /**
+   * Summaries produced for the current run.
+   * Must be static since it is written by the `findRegressions`
+   * static method.
+   */
+  static AllTestSummaries testSummaries = new AllTestSummaries();
+  /**
+   * Summaries checked-in as resources that we compare against.
+   */
+  static AllTestSummaries goldenTestSummaries = new AllTestSummaries();
+
+  static final String GOLDENFILE = "/slttestfailures.txt";
+
+  private static TestStatistics launchSqlLogicTest(String... args) throws IOException {
+    OptionsParser options = new OptionsParser(false, System.out, System.err);
+    CalciteExecutor.register(options);
+    return net.hydromatic.sqllogictest.Main.execute(options, args);
+  }
+
+  TestSummary shortSummary(String file, TestStatistics statistics) {
+    return new TestSummary(file, statistics.getPassedTestCount(), statistics.getFailedTestCount());
+  }
+
+  // The following tests currently timeout during execution.
+  // Technically these are Calcite bugs.
+  Set<String> timeout = new HashSet<String>() {
+    {
+      add("test/select5.test");
+      add("test/random/groupby/slt_good_10.test");
+    }
+  };
+
+  // The following tests contain SQL statements that are not supported by HSQLDB
+  Set<String> unsupported = new HashSet<String>() {
+    {
+      add("test/evidence/slt_lang_replace.test");
+      add("test/evidence/slt_lang_createtrigger.test");
+      add("test/evidence/slt_lang_droptrigger.test");
+      add("test/evidence/slt_lang_update.test");
+      add("test/evidence/slt_lang_reindex.test");
+    }
+  };
+
+  void runOneTestFile(String testFile) throws IOException {
+    if (timeout.contains(testFile)) {

Review Comment:
   Better use JUnit assumptions (https://junit.org/junit5/docs/5.0.3/api/org/junit/jupiter/api/Assumptions.html) so that we have a complete picture of what is run and what is skipped and when the latter happens to know also the reason.



##########
plus/src/test/java/org/apache/calcite/slt/SqlLogicTestsForCalciteTests.java:
##########
@@ -0,0 +1,312 @@
+/*
+ * 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 net.hydromatic.sqllogictest.OptionsParser;
+import net.hydromatic.sqllogictest.TestStatistics;
+
+import org.junit.jupiter.api.AfterAll;
+import org.junit.jupiter.api.Assertions;
+import org.junit.jupiter.api.BeforeAll;
+import org.junit.jupiter.api.Disabled;
+import org.junit.jupiter.api.DynamicTest;
+import org.junit.jupiter.api.Tag;
+import org.junit.jupiter.api.Test;
+import org.junit.jupiter.api.TestFactory;
+import org.junit.jupiter.api.function.Executable;
+
+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.OutputStreamWriter;
+import java.nio.charset.StandardCharsets;
+import java.nio.file.Files;
+import java.time.Duration;
+import java.util.ArrayList;
+import java.util.Comparator;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+
+import static org.hamcrest.CoreMatchers.is;
+import static org.hamcrest.CoreMatchers.notNullValue;
+import static org.hamcrest.MatcherAssert.assertThat;
+import static org.junit.jupiter.api.Assertions.assertTimeoutPreemptively;
+
+/**
+ * Tests using sql-logic-test suite.
+ */
+public class SqlLogicTestsForCalciteTests {
+  /**
+   * Short summary of the results of a test execution.
+   */
+  static class TestSummary {
+    /**
+     * File containing tests.
+     */
+    final String file;
+    /**
+     * Number of tests that have passed.
+     */
+    final int passed;
+    /**
+     * Number of tests that have failed.
+     */
+    final int failed;
+
+    TestSummary(String file, int passed, int failed) {
+      this.file = file;
+      this.passed = passed;
+      this.failed = failed;
+    }
+
+    /**
+     * Parse a TestSummary from a string.
+     * The inverse of 'toString'.
+     *
+     * @return The parsed TestSummary or null on failure.
+     */
+    static TestSummary parse(String line) {
+      String[] parts = line.split(":");
+      if (parts.length != 3) {
+        return null;
+      }
+      try {
+        int passed = Integer.parseInt(parts[1]);
+        int failed = Integer.parseInt(parts[2]);
+        return new TestSummary(parts[0], passed, failed);
+      } catch (NumberFormatException ex) {
+        return null;
+      }
+    }
+
+    @Override public String toString() {
+      return this.file + ":" + this.passed + ":" + this.failed;
+    }
+
+    /**
+     * Check if the 'other' TestSummaries are a regressions
+     * when compared to 'this'.
+     *
+     * @param other TestSummary to compare against.
+     * @return 'true' if 'other' is a regression from 'this'.
+     */
+    boolean regression(TestSummary other) {
+      return other.failed > this.failed;
+    }
+  }
+
+  /**
+   * Summary for all tests executed.
+   */
+  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);
+    }
+
+    void read(InputStream stream) throws IOException {
+      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 {
+            System.err.println("Could not parse line " + line);
+          }
+        });
+      }
+    }
+
+    /**
+     * Check if 'other' summaries have regressions compared to `this`.
+     *
+     * @return 'true' if other contains regressions.
+     * @param other  Test results to compare with.
+     *               'other' can contain only a subset of the tests.
+     */
+    boolean regression(AllTestSummaries other) {
+      boolean regression = false;
+      for (TestSummary summary: other.testResults.values()) {
+        TestSummary original = this.testResults.get(summary.file);
+        if (original == null) {
+          System.err.println("No historical data for test " + summary.file);
+          continue;
+        }
+        if (original.regression(summary)) {
+          System.err.println("Regression: " + original.file
+              + " had " + original.failed + " failures, now has " + summary.failed);
+          regression = true;
+        }
+      }
+      return regression;
+    }
+
+    @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());
+      }
+    }
+
+    /**
+     * True if there is an entry for the specified test file.
+     *
+     * @param test Test file name.
+     */
+    public boolean contains(String test) {
+      return this.testResults.containsKey(test);
+    }
+  }
+
+  /**
+   * Summaries produced for the current run.
+   * Must be static since it is written by the `findRegressions`
+   * static method.
+   */
+  static AllTestSummaries testSummaries = new AllTestSummaries();
+  /**
+   * Summaries checked-in as resources that we compare against.
+   */
+  static AllTestSummaries goldenTestSummaries = new AllTestSummaries();
+
+  static final String GOLDENFILE = "/slttestfailures.txt";
+
+  private static TestStatistics launchSqlLogicTest(String... args) throws IOException {
+    OptionsParser options = new OptionsParser(false, System.out, System.err);
+    CalciteExecutor.register(options);
+    return net.hydromatic.sqllogictest.Main.execute(options, args);
+  }
+
+  TestSummary shortSummary(String file, TestStatistics statistics) {
+    return new TestSummary(file, statistics.getPassedTestCount(), statistics.getFailedTestCount());
+  }
+
+  // The following tests currently timeout during execution.
+  // Technically these are Calcite bugs.
+  Set<String> timeout = new HashSet<String>() {
+    {
+      add("test/select5.test");
+      add("test/random/groupby/slt_good_10.test");
+    }
+  };
+
+  // The following tests contain SQL statements that are not supported by HSQLDB
+  Set<String> unsupported = new HashSet<String>() {
+    {
+      add("test/evidence/slt_lang_replace.test");
+      add("test/evidence/slt_lang_createtrigger.test");
+      add("test/evidence/slt_lang_droptrigger.test");
+      add("test/evidence/slt_lang_update.test");
+      add("test/evidence/slt_lang_reindex.test");
+    }
+  };
+
+  void runOneTestFile(String testFile) throws IOException {
+    if (timeout.contains(testFile)) {
+      return;
+    }
+    if (unsupported.contains(testFile)) {
+      return;
+    }
+
+    TestStatistics res = launchSqlLogicTest("-v", "-e", "calcite", testFile);
+    assertThat(res, notNullValue());
+    assertThat(res.getParseFailureCount(), is(0));
+    assertThat(res.getIgnoredTestCount(), is(0));
+    assertThat(res.getTestFileCount(), is(1));
+    res.printStatistics(System.err);  // Print errors found
+    TestSummary summary = this.shortSummary(testFile, res);
+    testSummaries.add(summary);
+  }
+
+  @Test @Tag("slow")
+  public void runOneTestFile() throws IOException {
+    runOneTestFile("select1.test");
+  }
+
+  @TestFactory @Disabled("This takes very long, should be run manually")
+  List<DynamicTest> runAllTests() {
+    // Run in parallel each test file.
+    Set<String> tests = net.hydromatic.sqllogictest.Main.getTestList();
+    List<DynamicTest> result = new ArrayList<>();
+    for (String test: tests) {
+      Executable executable = new Executable() {
+        @Override public void execute() {
+          assertTimeoutPreemptively(Duration.ofMinutes(10), () -> runOneTestFile(test));
+        }
+      };
+      DynamicTest dynamicTest = DynamicTest.dynamicTest(test, executable);
+      result.add(dynamicTest);
+    }
+    return result;
+  }
+
+  @BeforeAll

Review Comment:
   We don't really need to use a junit annotation since we simply have a static constant initializer that does not change. I don't think we really need a separate method to be honest.
   
   With a tiny bit of refactoring in we could drop this method and just use have the one in AllTestSumaries and have something like below:
   ```
   private static final AllTestSummaries GOLDEN_SUMMARIES = new AllTestSummaries().read("/slttestfailures.txt");
   private static final AllTestSummaries GOLDEN_SUMMARIES = AllTestSummaries.read("/slttestfailures.txt");



##########
plus/src/test/java/org/apache/calcite/slt/SqlLogicTestsForCalciteTests.java:
##########
@@ -0,0 +1,312 @@
+/*
+ * 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 net.hydromatic.sqllogictest.OptionsParser;
+import net.hydromatic.sqllogictest.TestStatistics;
+
+import org.junit.jupiter.api.AfterAll;
+import org.junit.jupiter.api.Assertions;
+import org.junit.jupiter.api.BeforeAll;
+import org.junit.jupiter.api.Disabled;
+import org.junit.jupiter.api.DynamicTest;
+import org.junit.jupiter.api.Tag;
+import org.junit.jupiter.api.Test;
+import org.junit.jupiter.api.TestFactory;
+import org.junit.jupiter.api.function.Executable;
+
+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.OutputStreamWriter;
+import java.nio.charset.StandardCharsets;
+import java.nio.file.Files;
+import java.time.Duration;
+import java.util.ArrayList;
+import java.util.Comparator;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+
+import static org.hamcrest.CoreMatchers.is;
+import static org.hamcrest.CoreMatchers.notNullValue;
+import static org.hamcrest.MatcherAssert.assertThat;
+import static org.junit.jupiter.api.Assertions.assertTimeoutPreemptively;
+
+/**
+ * Tests using sql-logic-test suite.
+ */
+public class SqlLogicTestsForCalciteTests {
+  /**
+   * Short summary of the results of a test execution.
+   */
+  static class TestSummary {
+    /**
+     * File containing tests.
+     */
+    final String file;
+    /**
+     * Number of tests that have passed.
+     */
+    final int passed;
+    /**
+     * Number of tests that have failed.
+     */
+    final int failed;
+
+    TestSummary(String file, int passed, int failed) {
+      this.file = file;
+      this.passed = passed;
+      this.failed = failed;
+    }
+
+    /**
+     * Parse a TestSummary from a string.
+     * The inverse of 'toString'.
+     *
+     * @return The parsed TestSummary or null on failure.
+     */
+    static TestSummary parse(String line) {
+      String[] parts = line.split(":");
+      if (parts.length != 3) {
+        return null;
+      }
+      try {
+        int passed = Integer.parseInt(parts[1]);
+        int failed = Integer.parseInt(parts[2]);
+        return new TestSummary(parts[0], passed, failed);
+      } catch (NumberFormatException ex) {
+        return null;
+      }
+    }
+
+    @Override public String toString() {
+      return this.file + ":" + this.passed + ":" + this.failed;
+    }
+
+    /**
+     * Check if the 'other' TestSummaries are a regressions
+     * when compared to 'this'.
+     *
+     * @param other TestSummary to compare against.
+     * @return 'true' if 'other' is a regression from 'this'.
+     */
+    boolean regression(TestSummary other) {
+      return other.failed > this.failed;
+    }
+  }
+
+  /**
+   * Summary for all tests executed.
+   */
+  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);
+    }
+
+    void read(InputStream stream) throws IOException {
+      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 {
+            System.err.println("Could not parse line " + line);
+          }
+        });
+      }
+    }
+
+    /**
+     * Check if 'other' summaries have regressions compared to `this`.
+     *
+     * @return 'true' if other contains regressions.
+     * @param other  Test results to compare with.
+     *               'other' can contain only a subset of the tests.
+     */
+    boolean regression(AllTestSummaries other) {
+      boolean regression = false;
+      for (TestSummary summary: other.testResults.values()) {
+        TestSummary original = this.testResults.get(summary.file);
+        if (original == null) {
+          System.err.println("No historical data for test " + summary.file);
+          continue;
+        }
+        if (original.regression(summary)) {
+          System.err.println("Regression: " + original.file
+              + " had " + original.failed + " failures, now has " + summary.failed);
+          regression = true;
+        }
+      }
+      return regression;
+    }
+
+    @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());
+      }
+    }
+
+    /**
+     * True if there is an entry for the specified test file.
+     *
+     * @param test Test file name.
+     */
+    public boolean contains(String test) {
+      return this.testResults.containsKey(test);
+    }
+  }
+
+  /**
+   * Summaries produced for the current run.
+   * Must be static since it is written by the `findRegressions`
+   * static method.
+   */
+  static AllTestSummaries testSummaries = new AllTestSummaries();
+  /**
+   * Summaries checked-in as resources that we compare against.
+   */
+  static AllTestSummaries goldenTestSummaries = new AllTestSummaries();
+
+  static final String GOLDENFILE = "/slttestfailures.txt";
+
+  private static TestStatistics launchSqlLogicTest(String... args) throws IOException {
+    OptionsParser options = new OptionsParser(false, System.out, System.err);
+    CalciteExecutor.register(options);
+    return net.hydromatic.sqllogictest.Main.execute(options, args);
+  }
+
+  TestSummary shortSummary(String file, TestStatistics statistics) {
+    return new TestSummary(file, statistics.getPassedTestCount(), statistics.getFailedTestCount());
+  }
+
+  // The following tests currently timeout during execution.
+  // Technically these are Calcite bugs.
+  Set<String> timeout = new HashSet<String>() {

Review Comment:
   Not worth sub-classing `HashSet` for this. Use `ImmutableSet.of` or `ImmutableSet.builder`. Same in other places where this pattern is used.



##########
plus/src/test/java/org/apache/calcite/slt/SqlLogicTestsForCalciteTests.java:
##########
@@ -0,0 +1,312 @@
+/*
+ * 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 net.hydromatic.sqllogictest.OptionsParser;
+import net.hydromatic.sqllogictest.TestStatistics;
+
+import org.junit.jupiter.api.AfterAll;
+import org.junit.jupiter.api.Assertions;
+import org.junit.jupiter.api.BeforeAll;
+import org.junit.jupiter.api.Disabled;
+import org.junit.jupiter.api.DynamicTest;
+import org.junit.jupiter.api.Tag;
+import org.junit.jupiter.api.Test;
+import org.junit.jupiter.api.TestFactory;
+import org.junit.jupiter.api.function.Executable;
+
+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.OutputStreamWriter;
+import java.nio.charset.StandardCharsets;
+import java.nio.file.Files;
+import java.time.Duration;
+import java.util.ArrayList;
+import java.util.Comparator;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+
+import static org.hamcrest.CoreMatchers.is;
+import static org.hamcrest.CoreMatchers.notNullValue;
+import static org.hamcrest.MatcherAssert.assertThat;
+import static org.junit.jupiter.api.Assertions.assertTimeoutPreemptively;
+
+/**
+ * Tests using sql-logic-test suite.
+ */
+public class SqlLogicTestsForCalciteTests {
+  /**
+   * Short summary of the results of a test execution.
+   */
+  static class TestSummary {
+    /**
+     * File containing tests.
+     */
+    final String file;
+    /**
+     * Number of tests that have passed.
+     */
+    final int passed;
+    /**
+     * Number of tests that have failed.
+     */
+    final int failed;
+
+    TestSummary(String file, int passed, int failed) {
+      this.file = file;
+      this.passed = passed;
+      this.failed = failed;
+    }
+
+    /**
+     * Parse a TestSummary from a string.
+     * The inverse of 'toString'.
+     *
+     * @return The parsed TestSummary or null on failure.
+     */
+    static TestSummary parse(String line) {
+      String[] parts = line.split(":");
+      if (parts.length != 3) {
+        return null;
+      }
+      try {
+        int passed = Integer.parseInt(parts[1]);
+        int failed = Integer.parseInt(parts[2]);
+        return new TestSummary(parts[0], passed, failed);
+      } catch (NumberFormatException ex) {
+        return null;
+      }
+    }
+
+    @Override public String toString() {
+      return this.file + ":" + this.passed + ":" + this.failed;
+    }
+
+    /**
+     * Check if the 'other' TestSummaries are a regressions
+     * when compared to 'this'.
+     *
+     * @param other TestSummary to compare against.
+     * @return 'true' if 'other' is a regression from 'this'.
+     */
+    boolean regression(TestSummary other) {
+      return other.failed > this.failed;

Review Comment:
   I don't know if regression is the only thing that we should care about. If a commit fixes a bug and we have more tests passing than before then it would be nice to capture that information as well.
   
   In an optimistic scenario where we keep fixing bugs and the number of `failed` tests keeps dropping we will never need to update the golden file hindering the progress that is made.
   
   I am in favor of making the test fail (and requiring update the golden file to pass) whenever there is a change in the passed/failed tests, which is inline with what happens in general in the Calcite tests.



##########
plus/src/test/java/org/apache/calcite/slt/README.md:
##########
@@ -0,0 +1,130 @@
+<!--
+{% comment %}
+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.
+{% endcomment %}
+-->
+
+# Testing Calcite using SQL Logic Tests
+
+[SQL Logic Tests](https://www.sqlite.org/sqllogictest/doc/trunk/about.wiki) is public domain suite of several million SQL tests
+created as part of the sqlite engine.
+The project [hydromatic/sql-logic-test](https://github.com/hydromatic/sql-logic-test/)
+has packaged these tests into a Java framework which makes it easy to
+run them against a JDBC provider.
+
+## How does it work

Review Comment:
   This and the following sections could be the actual documentation (javadoc) of `SqlLogicTestsForCalciteTests`. Not sure what's the benefit of having a separate README file.



##########
plus/src/test/java/org/apache/calcite/slt/SqlLogicTestsForCalciteTests.java:
##########
@@ -0,0 +1,312 @@
+/*
+ * 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 net.hydromatic.sqllogictest.OptionsParser;
+import net.hydromatic.sqllogictest.TestStatistics;
+
+import org.junit.jupiter.api.AfterAll;
+import org.junit.jupiter.api.Assertions;
+import org.junit.jupiter.api.BeforeAll;
+import org.junit.jupiter.api.Disabled;
+import org.junit.jupiter.api.DynamicTest;
+import org.junit.jupiter.api.Tag;
+import org.junit.jupiter.api.Test;
+import org.junit.jupiter.api.TestFactory;
+import org.junit.jupiter.api.function.Executable;
+
+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.OutputStreamWriter;
+import java.nio.charset.StandardCharsets;
+import java.nio.file.Files;
+import java.time.Duration;
+import java.util.ArrayList;
+import java.util.Comparator;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+
+import static org.hamcrest.CoreMatchers.is;
+import static org.hamcrest.CoreMatchers.notNullValue;
+import static org.hamcrest.MatcherAssert.assertThat;
+import static org.junit.jupiter.api.Assertions.assertTimeoutPreemptively;
+
+/**
+ * Tests using sql-logic-test suite.
+ */
+public class SqlLogicTestsForCalciteTests {
+  /**
+   * Short summary of the results of a test execution.
+   */
+  static class TestSummary {
+    /**
+     * File containing tests.
+     */
+    final String file;
+    /**
+     * Number of tests that have passed.
+     */
+    final int passed;
+    /**
+     * Number of tests that have failed.
+     */
+    final int failed;
+
+    TestSummary(String file, int passed, int failed) {
+      this.file = file;
+      this.passed = passed;
+      this.failed = failed;
+    }
+
+    /**
+     * Parse a TestSummary from a string.
+     * The inverse of 'toString'.
+     *
+     * @return The parsed TestSummary or null on failure.
+     */
+    static TestSummary parse(String line) {
+      String[] parts = line.split(":");
+      if (parts.length != 3) {
+        return null;
+      }
+      try {
+        int passed = Integer.parseInt(parts[1]);
+        int failed = Integer.parseInt(parts[2]);
+        return new TestSummary(parts[0], passed, failed);
+      } catch (NumberFormatException ex) {
+        return null;
+      }
+    }
+
+    @Override public String toString() {
+      return this.file + ":" + this.passed + ":" + this.failed;
+    }
+
+    /**
+     * Check if the 'other' TestSummaries are a regressions
+     * when compared to 'this'.
+     *
+     * @param other TestSummary to compare against.
+     * @return 'true' if 'other' is a regression from 'this'.
+     */
+    boolean regression(TestSummary other) {
+      return other.failed > this.failed;
+    }
+  }
+
+  /**
+   * Summary for all tests executed.
+   */
+  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);
+    }
+
+    void read(InputStream stream) throws IOException {
+      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 {
+            System.err.println("Could not parse line " + line);
+          }
+        });
+      }
+    }
+
+    /**
+     * Check if 'other' summaries have regressions compared to `this`.
+     *
+     * @return 'true' if other contains regressions.
+     * @param other  Test results to compare with.
+     *               'other' can contain only a subset of the tests.
+     */
+    boolean regression(AllTestSummaries other) {
+      boolean regression = false;
+      for (TestSummary summary: other.testResults.values()) {
+        TestSummary original = this.testResults.get(summary.file);
+        if (original == null) {
+          System.err.println("No historical data for test " + summary.file);

Review Comment:
   This is not fatal so it should rather be `LOG.warn`



##########
plus/src/test/java/org/apache/calcite/slt/SqlLogicTestsForCalciteTests.java:
##########
@@ -0,0 +1,312 @@
+/*
+ * 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 net.hydromatic.sqllogictest.OptionsParser;
+import net.hydromatic.sqllogictest.TestStatistics;
+
+import org.junit.jupiter.api.AfterAll;
+import org.junit.jupiter.api.Assertions;
+import org.junit.jupiter.api.BeforeAll;
+import org.junit.jupiter.api.Disabled;
+import org.junit.jupiter.api.DynamicTest;
+import org.junit.jupiter.api.Tag;
+import org.junit.jupiter.api.Test;
+import org.junit.jupiter.api.TestFactory;
+import org.junit.jupiter.api.function.Executable;
+
+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.OutputStreamWriter;
+import java.nio.charset.StandardCharsets;
+import java.nio.file.Files;
+import java.time.Duration;
+import java.util.ArrayList;
+import java.util.Comparator;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+
+import static org.hamcrest.CoreMatchers.is;
+import static org.hamcrest.CoreMatchers.notNullValue;
+import static org.hamcrest.MatcherAssert.assertThat;
+import static org.junit.jupiter.api.Assertions.assertTimeoutPreemptively;
+
+/**
+ * Tests using sql-logic-test suite.
+ */
+public class SqlLogicTestsForCalciteTests {
+  /**
+   * Short summary of the results of a test execution.
+   */
+  static class TestSummary {
+    /**
+     * File containing tests.
+     */
+    final String file;
+    /**
+     * Number of tests that have passed.
+     */
+    final int passed;
+    /**
+     * Number of tests that have failed.
+     */
+    final int failed;
+
+    TestSummary(String file, int passed, int failed) {
+      this.file = file;
+      this.passed = passed;
+      this.failed = failed;
+    }
+
+    /**
+     * Parse a TestSummary from a string.
+     * The inverse of 'toString'.
+     *
+     * @return The parsed TestSummary or null on failure.
+     */
+    static TestSummary parse(String line) {
+      String[] parts = line.split(":");
+      if (parts.length != 3) {
+        return null;
+      }
+      try {
+        int passed = Integer.parseInt(parts[1]);
+        int failed = Integer.parseInt(parts[2]);
+        return new TestSummary(parts[0], passed, failed);
+      } catch (NumberFormatException ex) {
+        return null;
+      }
+    }
+
+    @Override public String toString() {
+      return this.file + ":" + this.passed + ":" + this.failed;
+    }
+
+    /**
+     * Check if the 'other' TestSummaries are a regressions
+     * when compared to 'this'.
+     *
+     * @param other TestSummary to compare against.
+     * @return 'true' if 'other' is a regression from 'this'.
+     */
+    boolean regression(TestSummary other) {
+      return other.failed > this.failed;
+    }
+  }
+
+  /**
+   * Summary for all tests executed.
+   */
+  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);
+    }
+
+    void read(InputStream stream) throws IOException {
+      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 {
+            System.err.println("Could not parse line " + line);

Review Comment:
   This is not a Java app so we should not use `System.err` or `System.out` but use loggers instead so that we can tune via config where the output goes.



##########
plus/src/test/java/org/apache/calcite/slt/README.md:
##########
@@ -0,0 +1,130 @@
+<!--
+{% comment %}
+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.
+{% endcomment %}
+-->
+
+# Testing Calcite using SQL Logic Tests
+
+[SQL Logic Tests](https://www.sqlite.org/sqllogictest/doc/trunk/about.wiki) is public domain suite of several million SQL tests
+created as part of the sqlite engine.
+The project [hydromatic/sql-logic-test](https://github.com/hydromatic/sql-logic-test/)
+has packaged these tests into a Java framework which makes it easy to
+run them against a JDBC provider.

Review Comment:
   This part could be moved to javadoc of package-info.java or removed altogether. People who need more information about the sql-logic-test project can visit the respective repo.



##########
plus/src/test/java/org/apache/calcite/slt/SqlLogicTestsForCalciteTests.java:
##########
@@ -0,0 +1,312 @@
+/*
+ * 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 net.hydromatic.sqllogictest.OptionsParser;
+import net.hydromatic.sqllogictest.TestStatistics;
+
+import org.junit.jupiter.api.AfterAll;
+import org.junit.jupiter.api.Assertions;
+import org.junit.jupiter.api.BeforeAll;
+import org.junit.jupiter.api.Disabled;
+import org.junit.jupiter.api.DynamicTest;
+import org.junit.jupiter.api.Tag;
+import org.junit.jupiter.api.Test;
+import org.junit.jupiter.api.TestFactory;
+import org.junit.jupiter.api.function.Executable;
+
+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.OutputStreamWriter;
+import java.nio.charset.StandardCharsets;
+import java.nio.file.Files;
+import java.time.Duration;
+import java.util.ArrayList;
+import java.util.Comparator;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+
+import static org.hamcrest.CoreMatchers.is;
+import static org.hamcrest.CoreMatchers.notNullValue;
+import static org.hamcrest.MatcherAssert.assertThat;
+import static org.junit.jupiter.api.Assertions.assertTimeoutPreemptively;
+
+/**
+ * Tests using sql-logic-test suite.
+ */
+public class SqlLogicTestsForCalciteTests {
+  /**
+   * Short summary of the results of a test execution.
+   */
+  static class TestSummary {
+    /**
+     * File containing tests.
+     */
+    final String file;
+    /**
+     * Number of tests that have passed.
+     */
+    final int passed;
+    /**
+     * Number of tests that have failed.
+     */
+    final int failed;
+
+    TestSummary(String file, int passed, int failed) {
+      this.file = file;
+      this.passed = passed;
+      this.failed = failed;
+    }
+
+    /**
+     * Parse a TestSummary from a string.
+     * The inverse of 'toString'.
+     *
+     * @return The parsed TestSummary or null on failure.
+     */
+    static TestSummary parse(String line) {
+      String[] parts = line.split(":");
+      if (parts.length != 3) {
+        return null;
+      }
+      try {
+        int passed = Integer.parseInt(parts[1]);
+        int failed = Integer.parseInt(parts[2]);
+        return new TestSummary(parts[0], passed, failed);
+      } catch (NumberFormatException ex) {
+        return null;
+      }
+    }
+
+    @Override public String toString() {
+      return this.file + ":" + this.passed + ":" + this.failed;
+    }
+
+    /**
+     * Check if the 'other' TestSummaries are a regressions
+     * when compared to 'this'.
+     *
+     * @param other TestSummary to compare against.
+     * @return 'true' if 'other' is a regression from 'this'.
+     */
+    boolean regression(TestSummary other) {
+      return other.failed > this.failed;
+    }
+  }
+
+  /**
+   * Summary for all tests executed.
+   */
+  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);
+    }
+
+    void read(InputStream stream) throws IOException {
+      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 {
+            System.err.println("Could not parse line " + line);
+          }
+        });
+      }
+    }
+
+    /**
+     * Check if 'other' summaries have regressions compared to `this`.
+     *
+     * @return 'true' if other contains regressions.
+     * @param other  Test results to compare with.
+     *               'other' can contain only a subset of the tests.
+     */
+    boolean regression(AllTestSummaries other) {
+      boolean regression = false;
+      for (TestSummary summary: other.testResults.values()) {
+        TestSummary original = this.testResults.get(summary.file);
+        if (original == null) {
+          System.err.println("No historical data for test " + summary.file);
+          continue;
+        }
+        if (original.regression(summary)) {
+          System.err.println("Regression: " + original.file
+              + " had " + original.failed + " failures, now has " + summary.failed);
+          regression = true;
+        }
+      }
+      return regression;
+    }
+
+    @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());
+      }
+    }
+
+    /**
+     * True if there is an entry for the specified test file.
+     *
+     * @param test Test file name.
+     */
+    public boolean contains(String test) {
+      return this.testResults.containsKey(test);
+    }
+  }
+
+  /**
+   * Summaries produced for the current run.
+   * Must be static since it is written by the `findRegressions`
+   * static method.
+   */
+  static AllTestSummaries testSummaries = new AllTestSummaries();
+  /**
+   * Summaries checked-in as resources that we compare against.
+   */
+  static AllTestSummaries goldenTestSummaries = new AllTestSummaries();
+
+  static final String GOLDENFILE = "/slttestfailures.txt";
+
+  private static TestStatistics launchSqlLogicTest(String... args) throws IOException {
+    OptionsParser options = new OptionsParser(false, System.out, System.err);
+    CalciteExecutor.register(options);
+    return net.hydromatic.sqllogictest.Main.execute(options, args);
+  }
+
+  TestSummary shortSummary(String file, TestStatistics statistics) {
+    return new TestSummary(file, statistics.getPassedTestCount(), statistics.getFailedTestCount());
+  }
+
+  // The following tests currently timeout during execution.
+  // Technically these are Calcite bugs.

Review Comment:
   Turn it into Javadoc comment and move description from README.md here. It is easier to keep things up-to-date if they appear in a single place.



##########
plus/src/test/java/org/apache/calcite/slt/SqlLogicTestsForCalciteTests.java:
##########
@@ -0,0 +1,312 @@
+/*
+ * 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 net.hydromatic.sqllogictest.OptionsParser;
+import net.hydromatic.sqllogictest.TestStatistics;
+
+import org.junit.jupiter.api.AfterAll;
+import org.junit.jupiter.api.Assertions;
+import org.junit.jupiter.api.BeforeAll;
+import org.junit.jupiter.api.Disabled;
+import org.junit.jupiter.api.DynamicTest;
+import org.junit.jupiter.api.Tag;
+import org.junit.jupiter.api.Test;
+import org.junit.jupiter.api.TestFactory;
+import org.junit.jupiter.api.function.Executable;
+
+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.OutputStreamWriter;
+import java.nio.charset.StandardCharsets;
+import java.nio.file.Files;
+import java.time.Duration;
+import java.util.ArrayList;
+import java.util.Comparator;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+
+import static org.hamcrest.CoreMatchers.is;
+import static org.hamcrest.CoreMatchers.notNullValue;
+import static org.hamcrest.MatcherAssert.assertThat;
+import static org.junit.jupiter.api.Assertions.assertTimeoutPreemptively;
+
+/**
+ * Tests using sql-logic-test suite.
+ */
+public class SqlLogicTestsForCalciteTests {
+  /**
+   * Short summary of the results of a test execution.
+   */
+  static class TestSummary {
+    /**
+     * File containing tests.
+     */
+    final String file;
+    /**
+     * Number of tests that have passed.
+     */
+    final int passed;
+    /**
+     * Number of tests that have failed.
+     */
+    final int failed;
+
+    TestSummary(String file, int passed, int failed) {
+      this.file = file;
+      this.passed = passed;
+      this.failed = failed;
+    }
+
+    /**
+     * Parse a TestSummary from a string.

Review Comment:
   nit: 's/Parse/Parses/'
   
   See https://www.oracle.com/technical-resources/articles/java/javadoc-tool.html
   Use 3rd person (descriptive) not 2nd person (prescriptive).
   



##########
plus/src/test/java/org/apache/calcite/slt/SqlLogicTestsForCalciteTests.java:
##########
@@ -0,0 +1,312 @@
+/*
+ * 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 net.hydromatic.sqllogictest.OptionsParser;
+import net.hydromatic.sqllogictest.TestStatistics;
+
+import org.junit.jupiter.api.AfterAll;
+import org.junit.jupiter.api.Assertions;
+import org.junit.jupiter.api.BeforeAll;
+import org.junit.jupiter.api.Disabled;
+import org.junit.jupiter.api.DynamicTest;
+import org.junit.jupiter.api.Tag;
+import org.junit.jupiter.api.Test;
+import org.junit.jupiter.api.TestFactory;
+import org.junit.jupiter.api.function.Executable;
+
+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.OutputStreamWriter;
+import java.nio.charset.StandardCharsets;
+import java.nio.file.Files;
+import java.time.Duration;
+import java.util.ArrayList;
+import java.util.Comparator;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+
+import static org.hamcrest.CoreMatchers.is;
+import static org.hamcrest.CoreMatchers.notNullValue;
+import static org.hamcrest.MatcherAssert.assertThat;
+import static org.junit.jupiter.api.Assertions.assertTimeoutPreemptively;
+
+/**
+ * Tests using sql-logic-test suite.
+ */
+public class SqlLogicTestsForCalciteTests {
+  /**
+   * Short summary of the results of a test execution.
+   */
+  static class TestSummary {
+    /**
+     * File containing tests.
+     */
+    final String file;
+    /**
+     * Number of tests that have passed.
+     */
+    final int passed;
+    /**
+     * Number of tests that have failed.
+     */
+    final int failed;
+
+    TestSummary(String file, int passed, int failed) {
+      this.file = file;
+      this.passed = passed;
+      this.failed = failed;
+    }
+
+    /**
+     * Parse a TestSummary from a string.
+     * The inverse of 'toString'.
+     *
+     * @return The parsed TestSummary or null on failure.
+     */
+    static TestSummary parse(String line) {
+      String[] parts = line.split(":");
+      if (parts.length != 3) {
+        return null;
+      }
+      try {
+        int passed = Integer.parseInt(parts[1]);
+        int failed = Integer.parseInt(parts[2]);
+        return new TestSummary(parts[0], passed, failed);
+      } catch (NumberFormatException ex) {
+        return null;
+      }
+    }
+
+    @Override public String toString() {
+      return this.file + ":" + this.passed + ":" + this.failed;
+    }
+
+    /**
+     * Check if the 'other' TestSummaries are a regressions
+     * when compared to 'this'.
+     *
+     * @param other TestSummary to compare against.
+     * @return 'true' if 'other' is a regression from 'this'.
+     */
+    boolean regression(TestSummary other) {
+      return other.failed > this.failed;
+    }
+  }
+
+  /**
+   * Summary for all tests executed.
+   */
+  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);
+    }
+
+    void read(InputStream stream) throws IOException {
+      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 {
+            System.err.println("Could not parse line " + line);
+          }
+        });
+      }
+    }
+
+    /**
+     * Check if 'other' summaries have regressions compared to `this`.
+     *
+     * @return 'true' if other contains regressions.
+     * @param other  Test results to compare with.
+     *               'other' can contain only a subset of the tests.
+     */
+    boolean regression(AllTestSummaries other) {
+      boolean regression = false;
+      for (TestSummary summary: other.testResults.values()) {
+        TestSummary original = this.testResults.get(summary.file);
+        if (original == null) {
+          System.err.println("No historical data for test " + summary.file);
+          continue;
+        }
+        if (original.regression(summary)) {
+          System.err.println("Regression: " + original.file
+              + " had " + original.failed + " failures, now has " + summary.failed);
+          regression = true;
+        }
+      }
+      return regression;
+    }
+
+    @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());
+      }
+    }
+
+    /**
+     * True if there is an entry for the specified test file.
+     *
+     * @param test Test file name.
+     */
+    public boolean contains(String test) {
+      return this.testResults.containsKey(test);
+    }
+  }
+
+  /**
+   * Summaries produced for the current run.
+   * Must be static since it is written by the `findRegressions`
+   * static method.
+   */
+  static AllTestSummaries testSummaries = new AllTestSummaries();
+  /**
+   * Summaries checked-in as resources that we compare against.
+   */
+  static AllTestSummaries goldenTestSummaries = new AllTestSummaries();
+
+  static final String GOLDENFILE = "/slttestfailures.txt";
+
+  private static TestStatistics launchSqlLogicTest(String... args) throws IOException {
+    OptionsParser options = new OptionsParser(false, System.out, System.err);
+    CalciteExecutor.register(options);
+    return net.hydromatic.sqllogictest.Main.execute(options, args);
+  }
+
+  TestSummary shortSummary(String file, TestStatistics statistics) {
+    return new TestSummary(file, statistics.getPassedTestCount(), statistics.getFailedTestCount());
+  }
+
+  // The following tests currently timeout during execution.
+  // Technically these are Calcite bugs.
+  Set<String> timeout = new HashSet<String>() {
+    {
+      add("test/select5.test");
+      add("test/random/groupby/slt_good_10.test");
+    }
+  };
+
+  // The following tests contain SQL statements that are not supported by HSQLDB
+  Set<String> unsupported = new HashSet<String>() {
+    {
+      add("test/evidence/slt_lang_replace.test");
+      add("test/evidence/slt_lang_createtrigger.test");
+      add("test/evidence/slt_lang_droptrigger.test");
+      add("test/evidence/slt_lang_update.test");
+      add("test/evidence/slt_lang_reindex.test");
+    }
+  };
+
+  void runOneTestFile(String testFile) throws IOException {
+    if (timeout.contains(testFile)) {
+      return;
+    }
+    if (unsupported.contains(testFile)) {
+      return;
+    }
+
+    TestStatistics res = launchSqlLogicTest("-v", "-e", "calcite", testFile);
+    assertThat(res, notNullValue());
+    assertThat(res.getParseFailureCount(), is(0));
+    assertThat(res.getIgnoredTestCount(), is(0));
+    assertThat(res.getTestFileCount(), is(1));
+    res.printStatistics(System.err);  // Print errors found
+    TestSummary summary = this.shortSummary(testFile, res);
+    testSummaries.add(summary);
+  }
+
+  @Test @Tag("slow")
+  public void runOneTestFile() throws IOException {
+    runOneTestFile("select1.test");
+  }
+
+  @TestFactory @Disabled("This takes very long, should be run manually")
+  List<DynamicTest> runAllTests() {
+    // Run in parallel each test file.
+    Set<String> tests = net.hydromatic.sqllogictest.Main.getTestList();
+    List<DynamicTest> result = new ArrayList<>();
+    for (String test: tests) {
+      Executable executable = new Executable() {
+        @Override public void execute() {
+          assertTimeoutPreemptively(Duration.ofMinutes(10), () -> runOneTestFile(test));
+        }
+      };
+      DynamicTest dynamicTest = DynamicTest.dynamicTest(test, executable);
+      result.add(dynamicTest);
+    }
+    return result;
+  }
+
+  @BeforeAll
+  public static void readGoldenFile() throws IOException {
+    // Read the statistics of the previously-failing tests
+    try (InputStream stream = SqlLogicTestsForCalciteTests.class.getResourceAsStream(GOLDENFILE)) {
+      goldenTestSummaries.read(stream);
+    }
+  }
+
+  @AfterAll
+  public static void findRegressions() throws IOException {
+    // Compare with failures produced by a previous execution
+
+    // Code used to create the golden file originally
+    // File file = new File(goldenFile);
+    // if (!file.exists()) {
+    //   testSummaries.writeToFile(file);
+    //   return;
+    // }
+    boolean regression = goldenTestSummaries.regression(testSummaries);
+    Assertions.assertFalse(regression, "Regression discovered");

Review Comment:
   It is not very common to put assertions in `@AfterAll` methods. The assertion should be part of each test and not only do this at the very end.



##########
plus/build.gradle.kts:
##########
@@ -25,9 +25,11 @@ dependencies {
     implementation("com.teradata.tpcds:tpcds")
     implementation("io.prestosql.tpch:tpch")
     implementation("net.hydromatic:chinook-data-hsqldb")
+    implementation("net.hydromatic:sql-logic-test:0.2")
     implementation("net.hydromatic:tpcds")
     implementation("org.apache.calcite.avatica:avatica-server")
     implementation("org.hsqldb:hsqldb")
+    implementation("org.reflections:reflections:0.10.2")

Review Comment:
   This question about the declaration of reflections here is still open.



##########
plus/src/test/java/org/apache/calcite/slt/SqlLogicTestsForCalciteTests.java:
##########
@@ -0,0 +1,312 @@
+/*
+ * 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 net.hydromatic.sqllogictest.OptionsParser;
+import net.hydromatic.sqllogictest.TestStatistics;
+
+import org.junit.jupiter.api.AfterAll;
+import org.junit.jupiter.api.Assertions;
+import org.junit.jupiter.api.BeforeAll;
+import org.junit.jupiter.api.Disabled;
+import org.junit.jupiter.api.DynamicTest;
+import org.junit.jupiter.api.Tag;
+import org.junit.jupiter.api.Test;
+import org.junit.jupiter.api.TestFactory;
+import org.junit.jupiter.api.function.Executable;
+
+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.OutputStreamWriter;
+import java.nio.charset.StandardCharsets;
+import java.nio.file.Files;
+import java.time.Duration;
+import java.util.ArrayList;
+import java.util.Comparator;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+
+import static org.hamcrest.CoreMatchers.is;
+import static org.hamcrest.CoreMatchers.notNullValue;
+import static org.hamcrest.MatcherAssert.assertThat;
+import static org.junit.jupiter.api.Assertions.assertTimeoutPreemptively;
+
+/**
+ * Tests using sql-logic-test suite.
+ */
+public class SqlLogicTestsForCalciteTests {
+  /**
+   * Short summary of the results of a test execution.
+   */
+  static class TestSummary {
+    /**
+     * File containing tests.
+     */
+    final String file;
+    /**
+     * Number of tests that have passed.
+     */
+    final int passed;
+    /**
+     * Number of tests that have failed.
+     */
+    final int failed;
+
+    TestSummary(String file, int passed, int failed) {
+      this.file = file;
+      this.passed = passed;
+      this.failed = failed;
+    }
+
+    /**
+     * Parse a TestSummary from a string.
+     * The inverse of 'toString'.
+     *
+     * @return The parsed TestSummary or null on failure.
+     */
+    static TestSummary parse(String line) {
+      String[] parts = line.split(":");
+      if (parts.length != 3) {
+        return null;
+      }
+      try {
+        int passed = Integer.parseInt(parts[1]);
+        int failed = Integer.parseInt(parts[2]);
+        return new TestSummary(parts[0], passed, failed);
+      } catch (NumberFormatException ex) {
+        return null;
+      }
+    }
+
+    @Override public String toString() {
+      return this.file + ":" + this.passed + ":" + this.failed;
+    }
+
+    /**
+     * Check if the 'other' TestSummaries are a regressions
+     * when compared to 'this'.
+     *
+     * @param other TestSummary to compare against.
+     * @return 'true' if 'other' is a regression from 'this'.
+     */
+    boolean regression(TestSummary other) {
+      return other.failed > this.failed;
+    }
+  }
+
+  /**
+   * Summary for all tests executed.
+   */
+  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);
+    }
+
+    void read(InputStream stream) throws IOException {
+      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 {
+            System.err.println("Could not parse line " + line);

Review Comment:
   For instance when tests run in CI we don't want to clutter the console with error messages.



##########
plus/src/test/java/org/apache/calcite/slt/SqlLogicTestsForCalciteTests.java:
##########
@@ -0,0 +1,312 @@
+/*
+ * 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 net.hydromatic.sqllogictest.OptionsParser;
+import net.hydromatic.sqllogictest.TestStatistics;
+
+import org.junit.jupiter.api.AfterAll;
+import org.junit.jupiter.api.Assertions;
+import org.junit.jupiter.api.BeforeAll;
+import org.junit.jupiter.api.Disabled;
+import org.junit.jupiter.api.DynamicTest;
+import org.junit.jupiter.api.Tag;
+import org.junit.jupiter.api.Test;
+import org.junit.jupiter.api.TestFactory;
+import org.junit.jupiter.api.function.Executable;
+
+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.OutputStreamWriter;
+import java.nio.charset.StandardCharsets;
+import java.nio.file.Files;
+import java.time.Duration;
+import java.util.ArrayList;
+import java.util.Comparator;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+
+import static org.hamcrest.CoreMatchers.is;
+import static org.hamcrest.CoreMatchers.notNullValue;
+import static org.hamcrest.MatcherAssert.assertThat;
+import static org.junit.jupiter.api.Assertions.assertTimeoutPreemptively;
+
+/**
+ * Tests using sql-logic-test suite.
+ */
+public class SqlLogicTestsForCalciteTests {
+  /**
+   * Short summary of the results of a test execution.
+   */
+  static class TestSummary {
+    /**
+     * File containing tests.
+     */
+    final String file;
+    /**
+     * Number of tests that have passed.
+     */
+    final int passed;
+    /**
+     * Number of tests that have failed.
+     */
+    final int failed;
+
+    TestSummary(String file, int passed, int failed) {
+      this.file = file;
+      this.passed = passed;
+      this.failed = failed;
+    }
+
+    /**
+     * Parse a TestSummary from a string.
+     * The inverse of 'toString'.
+     *
+     * @return The parsed TestSummary or null on failure.
+     */
+    static TestSummary parse(String line) {
+      String[] parts = line.split(":");
+      if (parts.length != 3) {
+        return null;
+      }
+      try {
+        int passed = Integer.parseInt(parts[1]);
+        int failed = Integer.parseInt(parts[2]);
+        return new TestSummary(parts[0], passed, failed);
+      } catch (NumberFormatException ex) {
+        return null;
+      }
+    }
+
+    @Override public String toString() {
+      return this.file + ":" + this.passed + ":" + this.failed;
+    }
+
+    /**
+     * Check if the 'other' TestSummaries are a regressions
+     * when compared to 'this'.
+     *
+     * @param other TestSummary to compare against.
+     * @return 'true' if 'other' is a regression from 'this'.
+     */
+    boolean regression(TestSummary other) {
+      return other.failed > this.failed;
+    }
+  }
+
+  /**
+   * Summary for all tests executed.
+   */
+  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);
+    }
+
+    void read(InputStream stream) throws IOException {
+      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 {
+            System.err.println("Could not parse line " + line);
+          }
+        });
+      }
+    }
+
+    /**
+     * Check if 'other' summaries have regressions compared to `this`.
+     *
+     * @return 'true' if other contains regressions.
+     * @param other  Test results to compare with.
+     *               'other' can contain only a subset of the tests.
+     */
+    boolean regression(AllTestSummaries other) {
+      boolean regression = false;
+      for (TestSummary summary: other.testResults.values()) {
+        TestSummary original = this.testResults.get(summary.file);
+        if (original == null) {
+          System.err.println("No historical data for test " + summary.file);
+          continue;
+        }
+        if (original.regression(summary)) {
+          System.err.println("Regression: " + original.file
+              + " had " + original.failed + " failures, now has " + summary.failed);
+          regression = true;
+        }
+      }
+      return regression;
+    }
+
+    @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());
+      }
+    }
+
+    /**
+     * True if there is an entry for the specified test file.
+     *
+     * @param test Test file name.
+     */
+    public boolean contains(String test) {
+      return this.testResults.containsKey(test);
+    }
+  }
+
+  /**
+   * Summaries produced for the current run.
+   * Must be static since it is written by the `findRegressions`
+   * static method.
+   */
+  static AllTestSummaries testSummaries = new AllTestSummaries();
+  /**
+   * Summaries checked-in as resources that we compare against.
+   */
+  static AllTestSummaries goldenTestSummaries = new AllTestSummaries();
+
+  static final String GOLDENFILE = "/slttestfailures.txt";
+
+  private static TestStatistics launchSqlLogicTest(String... args) throws IOException {
+    OptionsParser options = new OptionsParser(false, System.out, System.err);
+    CalciteExecutor.register(options);
+    return net.hydromatic.sqllogictest.Main.execute(options, args);
+  }
+
+  TestSummary shortSummary(String file, TestStatistics statistics) {
+    return new TestSummary(file, statistics.getPassedTestCount(), statistics.getFailedTestCount());
+  }
+
+  // The following tests currently timeout during execution.
+  // Technically these are Calcite bugs.
+  Set<String> timeout = new HashSet<String>() {
+    {
+      add("test/select5.test");
+      add("test/random/groupby/slt_good_10.test");
+    }
+  };
+
+  // The following tests contain SQL statements that are not supported by HSQLDB
+  Set<String> unsupported = new HashSet<String>() {
+    {
+      add("test/evidence/slt_lang_replace.test");
+      add("test/evidence/slt_lang_createtrigger.test");
+      add("test/evidence/slt_lang_droptrigger.test");
+      add("test/evidence/slt_lang_update.test");
+      add("test/evidence/slt_lang_reindex.test");
+    }
+  };
+
+  void runOneTestFile(String testFile) throws IOException {
+    if (timeout.contains(testFile)) {
+      return;
+    }
+    if (unsupported.contains(testFile)) {
+      return;
+    }
+
+    TestStatistics res = launchSqlLogicTest("-v", "-e", "calcite", testFile);
+    assertThat(res, notNullValue());
+    assertThat(res.getParseFailureCount(), is(0));
+    assertThat(res.getIgnoredTestCount(), is(0));
+    assertThat(res.getTestFileCount(), is(1));
+    res.printStatistics(System.err);  // Print errors found
+    TestSummary summary = this.shortSummary(testFile, res);
+    testSummaries.add(summary);
+  }
+
+  @Test @Tag("slow")
+  public void runOneTestFile() throws IOException {
+    runOneTestFile("select1.test");
+  }
+
+  @TestFactory @Disabled("This takes very long, should be run manually")
+  List<DynamicTest> runAllTests() {

Review Comment:
   Since `runAllTests()` is `@Disabled` there is a risk that the code gets broken if it is not run in every day basis. 
   I would like to suggest a slight refactoring to ensure that we maximize the code that is tested daily.
   
   ```
   @TestFactory @Tag("slow")
   List<DynamicTest> testSlow() throws IOException {
     return generateTests(ImmutableSet.of("select1.test"));
   }
   
   @TestFactory @Disabled("This takes very long, should be run manually")
   List<DynamicTest> testAll() throws IOException {
     return generateTests(net.hydromatic.sqllogictest.Main.getTestList());
   }
   
   private static List<DynamicTest> generateTests(Set<String> testFiles) {
   // This is the code in runAllTests
   }
   ```
   



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@calcite.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [calcite] sonarcloud[bot] commented on pull request #3145: [CALCITE-5615] Program to run SQL Logic Tests for Calcite

Posted by "sonarcloud[bot] (via GitHub)" <gi...@apache.org>.
sonarcloud[bot] commented on PR #3145:
URL: https://github.com/apache/calcite/pull/3145#issuecomment-1507575847

   Kudos, SonarCloud Quality Gate passed!&nbsp; &nbsp; [![Quality Gate passed](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/QualityGateBadge/passed-16px.png 'Quality Gate passed')](https://sonarcloud.io/dashboard?id=apache_calcite&pullRequest=3145)
   
   [![Bug](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/bug-16px.png 'Bug')](https://sonarcloud.io/project/issues?id=apache_calcite&pullRequest=3145&resolved=false&types=BUG) [![A](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A-16px.png 'A')](https://sonarcloud.io/project/issues?id=apache_calcite&pullRequest=3145&resolved=false&types=BUG) [0 Bugs](https://sonarcloud.io/project/issues?id=apache_calcite&pullRequest=3145&resolved=false&types=BUG)  
   [![Vulnerability](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/vulnerability-16px.png 'Vulnerability')](https://sonarcloud.io/project/issues?id=apache_calcite&pullRequest=3145&resolved=false&types=VULNERABILITY) [![A](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A-16px.png 'A')](https://sonarcloud.io/project/issues?id=apache_calcite&pullRequest=3145&resolved=false&types=VULNERABILITY) [0 Vulnerabilities](https://sonarcloud.io/project/issues?id=apache_calcite&pullRequest=3145&resolved=false&types=VULNERABILITY)  
   [![Security Hotspot](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/security_hotspot-16px.png 'Security Hotspot')](https://sonarcloud.io/project/security_hotspots?id=apache_calcite&pullRequest=3145&resolved=false&types=SECURITY_HOTSPOT) [![A](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A-16px.png 'A')](https://sonarcloud.io/project/security_hotspots?id=apache_calcite&pullRequest=3145&resolved=false&types=SECURITY_HOTSPOT) [0 Security Hotspots](https://sonarcloud.io/project/security_hotspots?id=apache_calcite&pullRequest=3145&resolved=false&types=SECURITY_HOTSPOT)  
   [![Code Smell](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/code_smell-16px.png 'Code Smell')](https://sonarcloud.io/project/issues?id=apache_calcite&pullRequest=3145&resolved=false&types=CODE_SMELL) [![A](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A-16px.png 'A')](https://sonarcloud.io/project/issues?id=apache_calcite&pullRequest=3145&resolved=false&types=CODE_SMELL) [35 Code Smells](https://sonarcloud.io/project/issues?id=apache_calcite&pullRequest=3145&resolved=false&types=CODE_SMELL)
   
   [![0.0%](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/CoverageChart/0-16px.png '0.0%')](https://sonarcloud.io/component_measures?id=apache_calcite&pullRequest=3145&metric=new_coverage&view=list) [0.0% Coverage](https://sonarcloud.io/component_measures?id=apache_calcite&pullRequest=3145&metric=new_coverage&view=list)  
   [![0.0%](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/Duplications/3-16px.png '0.0%')](https://sonarcloud.io/component_measures?id=apache_calcite&pullRequest=3145&metric=new_duplicated_lines_density&view=list) [0.0% Duplication](https://sonarcloud.io/component_measures?id=apache_calcite&pullRequest=3145&metric=new_duplicated_lines_density&view=list)
   
   


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@calcite.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [calcite] julianhyde commented on pull request #3145: [CALCITE-5615] Program to run SQL Logic Tests for Calcite

Posted by "julianhyde (via GitHub)" <gi...@apache.org>.
julianhyde commented on PR #3145:
URL: https://github.com/apache/calcite/pull/3145#issuecomment-1596334720

   > Any way I can help?
   
   It would be helpful if you merge any pending changes to `main`, then create a branch and PR for Release 0.3. The content would be similar to commit 5195b8d for Release 0.2.
   
   Once the release notes etc. are done I can run the Maven process to publish to Maven Central.


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@calcite.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [calcite] zabetak commented on a diff in pull request #3145: [CALCITE-5615] Program to run SQL Logic Tests for Calcite

Posted by "zabetak (via GitHub)" <gi...@apache.org>.
zabetak commented on code in PR #3145:
URL: https://github.com/apache/calcite/pull/3145#discussion_r1296907558


##########
plus/src/test/java/org/apache/calcite/slt/SqlLogicTests.java:
##########
@@ -0,0 +1,337 @@
+/*
+ * 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.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.junit.jupiter.api.function.Executable;
+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.OutputStreamWriter;
+import java.nio.charset.StandardCharsets;
+import java.nio.file.Files;
+import java.time.Duration;
+import java.util.ArrayList;
+import java.util.Comparator;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+
+import static org.hamcrest.CoreMatchers.is;
+import static org.hamcrest.CoreMatchers.notNullValue;
+import static org.hamcrest.MatcherAssert.assertThat;
+import static org.junit.jupiter.api.Assertions.assertTimeoutPreemptively;
+
+/**
+ * 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.

Review Comment:
   Please ignore, I am applying some small fixes myself.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@calcite.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [calcite] zabetak commented on a diff in pull request #3145: [CALCITE-5615] Program to run SQL Logic Tests for Calcite

Posted by "zabetak (via GitHub)" <gi...@apache.org>.
zabetak commented on code in PR #3145:
URL: https://github.com/apache/calcite/pull/3145#discussion_r1296913798


##########
plus/src/test/java/org/apache/calcite/slt/SqlLogicTests.java:
##########
@@ -0,0 +1,337 @@
+/*
+ * 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.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.junit.jupiter.api.function.Executable;
+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.OutputStreamWriter;
+import java.nio.charset.StandardCharsets;
+import java.nio.file.Files;
+import java.time.Duration;
+import java.util.ArrayList;
+import java.util.Comparator;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+
+import static org.hamcrest.CoreMatchers.is;
+import static org.hamcrest.CoreMatchers.notNullValue;
+import static org.hamcrest.MatcherAssert.assertThat;
+import static org.junit.jupiter.api.Assertions.assertTimeoutPreemptively;
+
+/**
+ * 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;
+    }
+
+    /**
+     * Check if 'other' summaries have regressions compared to `this`.
+     *
+     * @return 'true' if other contains regressions.
+     * @param other  Test results to compare with.
+     *               'other' can contain only a subset of the tests.
+     */
+    boolean regression(AllTestSummaries other) {
+      boolean regression = false;
+      for (TestSummary summary: other.testResults.values()) {
+        regression = regression || this.regression(summary);
+      }
+      return regression;
+    }
+
+    @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());
+      }
+    }
+
+    /**
+     * True if there is an entry for the specified test file.
+     *
+     * @param test Test file name.
+     */
+    public boolean contains(String test) {
+      return this.testResults.containsKey(test);
+    }
+  }
+
+  /**
+   * Summaries produced for the current run.
+   * Must be static since it is written by the `findRegressions`
+   * static method.
+   */
+  static AllTestSummaries testSummaries = new AllTestSummaries();
+
+  static final String GOLDEN_FILE = "/slttestfailures.txt";
+  /**
+   * Summaries checked-in as resources that we compare against.
+   */
+  static AllTestSummaries goldenTestSummaries =
+      new AllTestSummaries()
+          .read(SqlLogicTests.class.getResourceAsStream(GOLDEN_FILE));
+
+  private static TestStatistics launchSqlLogicTest(String... args) throws IOException {
+    OptionsParser options = new OptionsParser(false, System.out, System.err);
+    CalciteExecutor.register(options);
+    return net.hydromatic.sqllogictest.Main.execute(options, args);
+  }
+
+  TestSummary shortSummary(String file, TestStatistics statistics) {
+    return new TestSummary(file, statistics.getFailedTestCount());
+  }
+
+  /**
+   * The following tests currently timeout during execution.
+   * Technically these are Calcite bugs.
+   */
+  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.
+   */
+  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");
+
+  void runOneTestFile(String testFile) throws IOException {
+    if (timeout.contains(testFile)) {
+      return;
+    }
+    if (unsupported.contains(testFile)) {
+      return;
+    }
+
+    // The arguments below are command-line arguments for the sql-logic-test
+    // executable from the hydromatic project.  The verbosity of the
+    // output can be increased by adding more "-v" flags to the command-line.
+    // By increasing verbosity even more you can get in the output a complete stack trace
+    // for each error caused by an exception.
+    TestStatistics res = launchSqlLogicTest("-v", "-e", "calcite", testFile);
+    assertThat(res, notNullValue());
+    assertThat(res.getParseFailureCount(), is(0));
+    assertThat(res.getIgnoredTestCount(), is(0));
+    assertThat(res.getTestFileCount(), is(1));
+    res.printStatistics(System.err);  // Print errors found
+    TestSummary summary = this.shortSummary(testFile, res);
+    boolean regression = goldenTestSummaries.regression(summary);
+    Assumptions.assumeFalse(regression, "Regression in " + summary.file);

Review Comment:
   I thought we were always checking the results against the golden file; if that's the case then why do we have the assertThat statements? Do we need both assertThat and assumeFalse, if yes why?



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@calcite.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [calcite] zabetak commented on pull request #3145: [CALCITE-5615] Program to run SQL Logic Tests for Calcite

Posted by "zabetak (via GitHub)" <gi...@apache.org>.
zabetak commented on PR #3145:
URL: https://github.com/apache/calcite/pull/3145#issuecomment-1691686749

   @mihaibudiu I am now testing everything along with my changes here: https://github.com/zabetak/calcite/tree/slt
   
   If run comes back green I will merge it and we can work on follow-ups after that.


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@calcite.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [calcite] zabetak commented on pull request #3145: [CALCITE-5615] Program to run SQL Logic Tests for Calcite

Posted by "zabetak (via GitHub)" <gi...@apache.org>.
zabetak commented on PR #3145:
URL: https://github.com/apache/calcite/pull/3145#issuecomment-1590951163

   @mihaibudiu In Calcite we haven't used Dynamic Tests so far but we do make use of [Parameterized Tests](https://junit.org/junit5/docs/current/user-guide/#writing-tests-parameterized-tests). The two kinds of tests are very similar so I am fine with any of them; whatever is most convenient for you.
   
   Although a new release may bring in some good enhancements I am not sure if it is strictly mandatory at this point. We know exactly the test files so we could hard code those here and generate a dynamic test for each.
   
   > Rather than checking that the number of failures doesn't change, it's better to check that there are no failures other than the list of 'expected failures'
   
   I will try to elaborate a bit more on the comment by Julian. I think what he had in mind can achieved with the following setup.
   
   For each test file that we decide to run we can create a "golden" file under `src/test/resources/sqllogictest` with the known/expected failures. Once we run the test we create the "actual" file with the failures and compare "golden" with "actual"; if they differ the test fails otherwise it passes. This is very much inline with how `DiffRepository` works so you could try to take advantage of it in these tests. If `DiffRepository` is not pertinent then you could opt for a simple diff or plain string equality comparison between the files.
   
   However, given that we have a large number of failures it might be impractical to create 622 golden files cause we might bloat the repository with test files containing errors. Currently everything under `src/test/resources` (data, golden files, logging configs, etc.) is ~3.5MB.
   ```
   find . -wholename "*/src/test/resources/*" -exec du -ch {} + | grep total
   ```
   If the sqllogic golden files containing the errors are around ~10MB it would be acceptable but going much more beyond that can be problematic and we should probably rethink this. Using compression to keep the file size low should be fine I guess.


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@calcite.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [calcite] zabetak commented on pull request #3145: [CALCITE-5615] Program to run SQL Logic Tests for Calcite

Posted by "zabetak (via GitHub)" <gi...@apache.org>.
zabetak commented on PR #3145:
URL: https://github.com/apache/calcite/pull/3145#issuecomment-1580300860

   > It took about 6 minutes to run 10K tests using CalciteExecutor, so by extrapolation it would take about 2 days to run all tests.
   
   Here are some thoughts regarding duration and frequency:
   * Tests runtime ~5 minutes -> could run on every PR;
   * Tests runtime between 1h and 3h -> could possibly run once daily;
   * Tests runtime under 6h -> could run once per week;
   * Tests runtime spanning days -> per request/ not automated (maybe before release)
   It's up to us I guess to divide them into meaningful buckets (e..g, those that are likely to break more often should run on every PR) and setup the relevant CI infra.
   
   Obviously we don't have to setup everything as part of this PR but it would be nice if we could at least put the 5minute bucket in place.
   
   > Memory consumption increases slightly with time.
   
   I guess we can find ways to address memory pressure; some quick ideas:
   * Run each test file separately; this in principle should reset the memory after each run.
   * Provide options/APIs to save failures in files (instead of keeping ever growing in memory structures)
   


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@calcite.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [calcite] zabetak commented on a diff in pull request #3145: [CALCITE-5615] Program to run SQL Logic Tests for Calcite

Posted by "zabetak (via GitHub)" <gi...@apache.org>.
zabetak commented on code in PR #3145:
URL: https://github.com/apache/calcite/pull/3145#discussion_r1205973836


##########
plus/src/test/java/org/apache/calcite/slt/TestCalcite.java:
##########
@@ -0,0 +1,86 @@
+/*
+ * 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 net.hydromatic.sqllogictest.OptionsParser;
+
+import org.junit.jupiter.api.Test;
+
+import java.io.ByteArrayOutputStream;
+import java.io.IOException;
+import java.io.PrintStream;
+import java.nio.charset.StandardCharsets;
+
+import static org.hamcrest.MatcherAssert.assertThat;
+import static org.hamcrest.core.Is.is;
+
+/**
+ * Tests using sql-logic-test suite.
+ */
+public class TestCalcite {

Review Comment:
   There is nothing preventing us from re-redirecting the streams to files and not keep them in byte buffers. Hopefully by doing this we should avoid running out of memory.
   
   We don't necessarily need to assert that there are no failures but more like that the number of failures doesn't change. Unless the number of failures goes down which I guess is a good thing meaning that we fixed a bug.
   
   As far as I understand, point 2 that I suggested above is hard to implement at this stage given that there are no test files that pass completely so we can definitely defer to another JIRA.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@calcite.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [calcite] sonarcloud[bot] commented on pull request #3145: [CALCITE-5615] Program to run SQL Logic Tests for Calcite

Posted by "sonarcloud[bot] (via GitHub)" <gi...@apache.org>.
sonarcloud[bot] commented on PR #3145:
URL: https://github.com/apache/calcite/pull/3145#issuecomment-1556334530

   Kudos, SonarCloud Quality Gate passed!&nbsp; &nbsp; [![Quality Gate passed](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/QualityGateBadge/passed-16px.png 'Quality Gate passed')](https://sonarcloud.io/dashboard?id=apache_calcite&pullRequest=3145)
   
   [![Bug](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/bug-16px.png 'Bug')](https://sonarcloud.io/project/issues?id=apache_calcite&pullRequest=3145&resolved=false&types=BUG) [![A](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A-16px.png 'A')](https://sonarcloud.io/project/issues?id=apache_calcite&pullRequest=3145&resolved=false&types=BUG) [0 Bugs](https://sonarcloud.io/project/issues?id=apache_calcite&pullRequest=3145&resolved=false&types=BUG)  
   [![Vulnerability](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/vulnerability-16px.png 'Vulnerability')](https://sonarcloud.io/project/issues?id=apache_calcite&pullRequest=3145&resolved=false&types=VULNERABILITY) [![A](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A-16px.png 'A')](https://sonarcloud.io/project/issues?id=apache_calcite&pullRequest=3145&resolved=false&types=VULNERABILITY) [0 Vulnerabilities](https://sonarcloud.io/project/issues?id=apache_calcite&pullRequest=3145&resolved=false&types=VULNERABILITY)  
   [![Security Hotspot](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/security_hotspot-16px.png 'Security Hotspot')](https://sonarcloud.io/project/security_hotspots?id=apache_calcite&pullRequest=3145&resolved=false&types=SECURITY_HOTSPOT) [![A](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A-16px.png 'A')](https://sonarcloud.io/project/security_hotspots?id=apache_calcite&pullRequest=3145&resolved=false&types=SECURITY_HOTSPOT) [0 Security Hotspots](https://sonarcloud.io/project/security_hotspots?id=apache_calcite&pullRequest=3145&resolved=false&types=SECURITY_HOTSPOT)  
   [![Code Smell](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/code_smell-16px.png 'Code Smell')](https://sonarcloud.io/project/issues?id=apache_calcite&pullRequest=3145&resolved=false&types=CODE_SMELL) [![A](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A-16px.png 'A')](https://sonarcloud.io/project/issues?id=apache_calcite&pullRequest=3145&resolved=false&types=CODE_SMELL) [0 Code Smells](https://sonarcloud.io/project/issues?id=apache_calcite&pullRequest=3145&resolved=false&types=CODE_SMELL)
   
   [![No Coverage information](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/CoverageChart/NoCoverageInfo-16px.png 'No Coverage information')](https://sonarcloud.io/component_measures?id=apache_calcite&pullRequest=3145&metric=coverage&view=list) No Coverage information  
   [![No Duplication information](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/Duplications/NoDuplicationInfo-16px.png 'No Duplication information')](https://sonarcloud.io/component_measures?id=apache_calcite&pullRequest=3145&metric=duplicated_lines_density&view=list) No Duplication information
   
   


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@calcite.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [calcite] sonarcloud[bot] commented on pull request #3145: [CALCITE-5615] Program to run SQL Logic Tests for Calcite

Posted by "sonarcloud[bot] (via GitHub)" <gi...@apache.org>.
sonarcloud[bot] commented on PR #3145:
URL: https://github.com/apache/calcite/pull/3145#issuecomment-1556345849

   Kudos, SonarCloud Quality Gate passed!&nbsp; &nbsp; [![Quality Gate passed](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/QualityGateBadge/passed-16px.png 'Quality Gate passed')](https://sonarcloud.io/dashboard?id=apache_calcite&pullRequest=3145)
   
   [![Bug](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/bug-16px.png 'Bug')](https://sonarcloud.io/project/issues?id=apache_calcite&pullRequest=3145&resolved=false&types=BUG) [![A](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A-16px.png 'A')](https://sonarcloud.io/project/issues?id=apache_calcite&pullRequest=3145&resolved=false&types=BUG) [0 Bugs](https://sonarcloud.io/project/issues?id=apache_calcite&pullRequest=3145&resolved=false&types=BUG)  
   [![Vulnerability](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/vulnerability-16px.png 'Vulnerability')](https://sonarcloud.io/project/issues?id=apache_calcite&pullRequest=3145&resolved=false&types=VULNERABILITY) [![A](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A-16px.png 'A')](https://sonarcloud.io/project/issues?id=apache_calcite&pullRequest=3145&resolved=false&types=VULNERABILITY) [0 Vulnerabilities](https://sonarcloud.io/project/issues?id=apache_calcite&pullRequest=3145&resolved=false&types=VULNERABILITY)  
   [![Security Hotspot](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/security_hotspot-16px.png 'Security Hotspot')](https://sonarcloud.io/project/security_hotspots?id=apache_calcite&pullRequest=3145&resolved=false&types=SECURITY_HOTSPOT) [![A](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A-16px.png 'A')](https://sonarcloud.io/project/security_hotspots?id=apache_calcite&pullRequest=3145&resolved=false&types=SECURITY_HOTSPOT) [0 Security Hotspots](https://sonarcloud.io/project/security_hotspots?id=apache_calcite&pullRequest=3145&resolved=false&types=SECURITY_HOTSPOT)  
   [![Code Smell](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/code_smell-16px.png 'Code Smell')](https://sonarcloud.io/project/issues?id=apache_calcite&pullRequest=3145&resolved=false&types=CODE_SMELL) [![A](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A-16px.png 'A')](https://sonarcloud.io/project/issues?id=apache_calcite&pullRequest=3145&resolved=false&types=CODE_SMELL) [0 Code Smells](https://sonarcloud.io/project/issues?id=apache_calcite&pullRequest=3145&resolved=false&types=CODE_SMELL)
   
   [![No Coverage information](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/CoverageChart/NoCoverageInfo-16px.png 'No Coverage information')](https://sonarcloud.io/component_measures?id=apache_calcite&pullRequest=3145&metric=coverage&view=list) No Coverage information  
   [![No Duplication information](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/Duplications/NoDuplicationInfo-16px.png 'No Duplication information')](https://sonarcloud.io/component_measures?id=apache_calcite&pullRequest=3145&metric=duplicated_lines_density&view=list) No Duplication information
   
   


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@calcite.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [calcite] mihaibudiu commented on a diff in pull request #3145: [CALCITE-5615] Program to run SQL Logic Tests for Calcite

Posted by "mihaibudiu (via GitHub)" <gi...@apache.org>.
mihaibudiu commented on code in PR #3145:
URL: https://github.com/apache/calcite/pull/3145#discussion_r1186546490


##########
settings.gradle.kts:
##########
@@ -83,7 +83,8 @@ include(
     "spark",
     "splunk",
     "testkit",
-    "ubenchmark"
+    "ubenchmark",
+    "slt"

Review Comment:
   I am not sure what "plus" is. Should I just move the code into the plus project using the org.apache.calcite.slt package?



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@calcite.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [calcite] mihaibudiu commented on pull request #3145: [CALCITE-5615] Program to run SQL Logic Tests for Calcite

Posted by "mihaibudiu (via GitHub)" <gi...@apache.org>.
mihaibudiu commented on PR #3145:
URL: https://github.com/apache/calcite/pull/3145#issuecomment-1540542474

   With the latest commit this is testing the CalciteExecutor against the candidate 0.2 release of sql-logic-test.


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@calcite.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [calcite] sonarcloud[bot] commented on pull request #3145: Program to run SQL Logic Tests for Calcite

Posted by "sonarcloud[bot] (via GitHub)" <gi...@apache.org>.
sonarcloud[bot] commented on PR #3145:
URL: https://github.com/apache/calcite/pull/3145#issuecomment-1504322315

   Kudos, SonarCloud Quality Gate passed!&nbsp; &nbsp; [![Quality Gate passed](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/QualityGateBadge/passed-16px.png 'Quality Gate passed')](https://sonarcloud.io/dashboard?id=apache_calcite&pullRequest=3145)
   
   [![Bug](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/bug-16px.png 'Bug')](https://sonarcloud.io/project/issues?id=apache_calcite&pullRequest=3145&resolved=false&types=BUG) [![E](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/E-16px.png 'E')](https://sonarcloud.io/project/issues?id=apache_calcite&pullRequest=3145&resolved=false&types=BUG) [6 Bugs](https://sonarcloud.io/project/issues?id=apache_calcite&pullRequest=3145&resolved=false&types=BUG)  
   [![Vulnerability](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/vulnerability-16px.png 'Vulnerability')](https://sonarcloud.io/project/issues?id=apache_calcite&pullRequest=3145&resolved=false&types=VULNERABILITY) [![E](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/E-16px.png 'E')](https://sonarcloud.io/project/issues?id=apache_calcite&pullRequest=3145&resolved=false&types=VULNERABILITY) [1 Vulnerability](https://sonarcloud.io/project/issues?id=apache_calcite&pullRequest=3145&resolved=false&types=VULNERABILITY)  
   [![Security Hotspot](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/security_hotspot-16px.png 'Security Hotspot')](https://sonarcloud.io/project/security_hotspots?id=apache_calcite&pullRequest=3145&resolved=false&types=SECURITY_HOTSPOT) [![E](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/E-16px.png 'E')](https://sonarcloud.io/project/security_hotspots?id=apache_calcite&pullRequest=3145&resolved=false&types=SECURITY_HOTSPOT) [5 Security Hotspots](https://sonarcloud.io/project/security_hotspots?id=apache_calcite&pullRequest=3145&resolved=false&types=SECURITY_HOTSPOT)  
   [![Code Smell](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/code_smell-16px.png 'Code Smell')](https://sonarcloud.io/project/issues?id=apache_calcite&pullRequest=3145&resolved=false&types=CODE_SMELL) [![A](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A-16px.png 'A')](https://sonarcloud.io/project/issues?id=apache_calcite&pullRequest=3145&resolved=false&types=CODE_SMELL) [63 Code Smells](https://sonarcloud.io/project/issues?id=apache_calcite&pullRequest=3145&resolved=false&types=CODE_SMELL)
   
   [![0.0%](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/CoverageChart/0-16px.png '0.0%')](https://sonarcloud.io/component_measures?id=apache_calcite&pullRequest=3145&metric=new_coverage&view=list) [0.0% Coverage](https://sonarcloud.io/component_measures?id=apache_calcite&pullRequest=3145&metric=new_coverage&view=list)  
   [![0.0%](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/Duplications/3-16px.png '0.0%')](https://sonarcloud.io/component_measures?id=apache_calcite&pullRequest=3145&metric=new_duplicated_lines_density&view=list) [0.0% Duplication](https://sonarcloud.io/component_measures?id=apache_calcite&pullRequest=3145&metric=new_duplicated_lines_density&view=list)
   
   


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@calcite.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [calcite] mihaibudiu commented on pull request #3145: [CALCITE-5615] Program to run SQL Logic Tests for Calcite

Posted by "mihaibudiu (via GitHub)" <gi...@apache.org>.
mihaibudiu commented on PR #3145:
URL: https://github.com/apache/calcite/pull/3145#issuecomment-1536875335

   @zabetak thank you for the comments, I will address them once sql-logic-test 2.0 is available in my next PR.
   But I would appreciate help to decide how to package this tool.
   


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@calcite.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [calcite] sonarcloud[bot] commented on pull request #3145: [CALCITE-5615] Program to run SQL Logic Tests for Calcite

Posted by "sonarcloud[bot] (via GitHub)" <gi...@apache.org>.
sonarcloud[bot] commented on PR #3145:
URL: https://github.com/apache/calcite/pull/3145#issuecomment-1603454362

   Kudos, SonarCloud Quality Gate passed!&nbsp; &nbsp; [![Quality Gate passed](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/QualityGateBadge/passed-16px.png 'Quality Gate passed')](https://sonarcloud.io/dashboard?id=apache_calcite&pullRequest=3145)
   
   [![Bug](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/bug-16px.png 'Bug')](https://sonarcloud.io/project/issues?id=apache_calcite&pullRequest=3145&resolved=false&types=BUG) [![A](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A-16px.png 'A')](https://sonarcloud.io/project/issues?id=apache_calcite&pullRequest=3145&resolved=false&types=BUG) [0 Bugs](https://sonarcloud.io/project/issues?id=apache_calcite&pullRequest=3145&resolved=false&types=BUG)  
   [![Vulnerability](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/vulnerability-16px.png 'Vulnerability')](https://sonarcloud.io/project/issues?id=apache_calcite&pullRequest=3145&resolved=false&types=VULNERABILITY) [![A](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A-16px.png 'A')](https://sonarcloud.io/project/issues?id=apache_calcite&pullRequest=3145&resolved=false&types=VULNERABILITY) [0 Vulnerabilities](https://sonarcloud.io/project/issues?id=apache_calcite&pullRequest=3145&resolved=false&types=VULNERABILITY)  
   [![Security Hotspot](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/security_hotspot-16px.png 'Security Hotspot')](https://sonarcloud.io/project/security_hotspots?id=apache_calcite&pullRequest=3145&resolved=false&types=SECURITY_HOTSPOT) [![A](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A-16px.png 'A')](https://sonarcloud.io/project/security_hotspots?id=apache_calcite&pullRequest=3145&resolved=false&types=SECURITY_HOTSPOT) [0 Security Hotspots](https://sonarcloud.io/project/security_hotspots?id=apache_calcite&pullRequest=3145&resolved=false&types=SECURITY_HOTSPOT)  
   [![Code Smell](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/code_smell-16px.png 'Code Smell')](https://sonarcloud.io/project/issues?id=apache_calcite&pullRequest=3145&resolved=false&types=CODE_SMELL) [![A](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A-16px.png 'A')](https://sonarcloud.io/project/issues?id=apache_calcite&pullRequest=3145&resolved=false&types=CODE_SMELL) [0 Code Smells](https://sonarcloud.io/project/issues?id=apache_calcite&pullRequest=3145&resolved=false&types=CODE_SMELL)
   
   [![No Coverage information](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/CoverageChart/NoCoverageInfo-16px.png 'No Coverage information')](https://sonarcloud.io/component_measures?id=apache_calcite&pullRequest=3145&metric=coverage&view=list) No Coverage information  
   [![No Duplication information](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/Duplications/NoDuplicationInfo-16px.png 'No Duplication information')](https://sonarcloud.io/component_measures?id=apache_calcite&pullRequest=3145&metric=duplicated_lines_density&view=list) No Duplication information
   
   


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@calcite.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [calcite] zabetak closed pull request #3145: [CALCITE-5615] Program to run SQL Logic Tests for Calcite

Posted by "zabetak (via GitHub)" <gi...@apache.org>.
zabetak closed pull request #3145: [CALCITE-5615] Program to run SQL Logic Tests for Calcite
URL: https://github.com/apache/calcite/pull/3145


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@calcite.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [calcite] mihaibudiu commented on a diff in pull request #3145: [CALCITE-5615] Program to run SQL Logic Tests for Calcite

Posted by "mihaibudiu (via GitHub)" <gi...@apache.org>.
mihaibudiu commented on code in PR #3145:
URL: https://github.com/apache/calcite/pull/3145#discussion_r1202905661


##########
plus/src/test/java/org/apache/calcite/slt/TestCalcite.java:
##########
@@ -0,0 +1,86 @@
+/*
+ * 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 net.hydromatic.sqllogictest.OptionsParser;
+
+import org.junit.jupiter.api.Test;
+
+import java.io.ByteArrayOutputStream;
+import java.io.IOException;
+import java.io.PrintStream;
+import java.nio.charset.StandardCharsets;
+
+import static org.hamcrest.MatcherAssert.assertThat;
+import static org.hamcrest.core.Is.is;
+
+/**
+ * Tests using sql-logic-test suite.
+ */
+public class TestCalcite {

Review Comment:
   The main problem is that there are *lots* of failures. Only in these 1000 tests from select1.test there are 153 failures. They are due to a small number of underlying causes. If we run all 5M tests in this call, with this expected failure rate there will be 50K failures, and the output will cause the PrintStreams used in launchSqlLogicTest to run out of memory.
   
   Until we implement https://github.com/hydromatic/sql-logic-test/issues/16 we probably cannot run all the tests.
   
   When all the found Calcite bugs are fixed we can modify the test here to assert that there are no failures, but until that happens I am unsure how to use this test. Right now this code is most useful if it outputs a diagnosis for human consumption.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@calcite.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [calcite] sonarcloud[bot] commented on pull request #3145: [CALCITE-5615] Program to run SQL Logic Tests for Calcite

Posted by "sonarcloud[bot] (via GitHub)" <gi...@apache.org>.
sonarcloud[bot] commented on PR #3145:
URL: https://github.com/apache/calcite/pull/3145#issuecomment-1556313766

   Kudos, SonarCloud Quality Gate passed!&nbsp; &nbsp; [![Quality Gate passed](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/QualityGateBadge/passed-16px.png 'Quality Gate passed')](https://sonarcloud.io/dashboard?id=apache_calcite&pullRequest=3145)
   
   [![Bug](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/bug-16px.png 'Bug')](https://sonarcloud.io/project/issues?id=apache_calcite&pullRequest=3145&resolved=false&types=BUG) [![A](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A-16px.png 'A')](https://sonarcloud.io/project/issues?id=apache_calcite&pullRequest=3145&resolved=false&types=BUG) [0 Bugs](https://sonarcloud.io/project/issues?id=apache_calcite&pullRequest=3145&resolved=false&types=BUG)  
   [![Vulnerability](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/vulnerability-16px.png 'Vulnerability')](https://sonarcloud.io/project/issues?id=apache_calcite&pullRequest=3145&resolved=false&types=VULNERABILITY) [![A](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A-16px.png 'A')](https://sonarcloud.io/project/issues?id=apache_calcite&pullRequest=3145&resolved=false&types=VULNERABILITY) [0 Vulnerabilities](https://sonarcloud.io/project/issues?id=apache_calcite&pullRequest=3145&resolved=false&types=VULNERABILITY)  
   [![Security Hotspot](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/security_hotspot-16px.png 'Security Hotspot')](https://sonarcloud.io/project/security_hotspots?id=apache_calcite&pullRequest=3145&resolved=false&types=SECURITY_HOTSPOT) [![A](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A-16px.png 'A')](https://sonarcloud.io/project/security_hotspots?id=apache_calcite&pullRequest=3145&resolved=false&types=SECURITY_HOTSPOT) [0 Security Hotspots](https://sonarcloud.io/project/security_hotspots?id=apache_calcite&pullRequest=3145&resolved=false&types=SECURITY_HOTSPOT)  
   [![Code Smell](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/code_smell-16px.png 'Code Smell')](https://sonarcloud.io/project/issues?id=apache_calcite&pullRequest=3145&resolved=false&types=CODE_SMELL) [![A](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A-16px.png 'A')](https://sonarcloud.io/project/issues?id=apache_calcite&pullRequest=3145&resolved=false&types=CODE_SMELL) [0 Code Smells](https://sonarcloud.io/project/issues?id=apache_calcite&pullRequest=3145&resolved=false&types=CODE_SMELL)
   
   [![No Coverage information](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/CoverageChart/NoCoverageInfo-16px.png 'No Coverage information')](https://sonarcloud.io/component_measures?id=apache_calcite&pullRequest=3145&metric=coverage&view=list) No Coverage information  
   [![No Duplication information](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/Duplications/NoDuplicationInfo-16px.png 'No Duplication information')](https://sonarcloud.io/component_measures?id=apache_calcite&pullRequest=3145&metric=duplicated_lines_density&view=list) No Duplication information
   
   


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@calcite.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [calcite] mihaibudiu commented on pull request #3145: [CALCITE-5615] Program to run SQL Logic Tests for Calcite

Posted by "mihaibudiu (via GitHub)" <gi...@apache.org>.
mihaibudiu commented on PR #3145:
URL: https://github.com/apache/calcite/pull/3145#issuecomment-1581359888

   I have amended this PR to use DynamicTests. This needs https://github.com/hydromatic/sql-logic-test/pull/36 in order to work, so a new release of sql-logic-tests is necessary. This seems a decent way to run all these 622 tests. On my machine it runs 8 tests at a time, each taking about 5 minutes. The errors discovered are just printed on stderr right now.


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@calcite.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [calcite] zabetak commented on a diff in pull request #3145: [CALCITE-5615] Program to run SQL Logic Tests for Calcite

Posted by "zabetak (via GitHub)" <gi...@apache.org>.
zabetak commented on code in PR #3145:
URL: https://github.com/apache/calcite/pull/3145#discussion_r1296836281


##########
plus/src/test/java/org/apache/calcite/slt/SqlLogicTests.java:
##########
@@ -0,0 +1,337 @@
+/*
+ * 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.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.junit.jupiter.api.function.Executable;
+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.OutputStreamWriter;
+import java.nio.charset.StandardCharsets;
+import java.nio.file.Files;
+import java.time.Duration;
+import java.util.ArrayList;
+import java.util.Comparator;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+
+import static org.hamcrest.CoreMatchers.is;
+import static org.hamcrest.CoreMatchers.notNullValue;
+import static org.hamcrest.MatcherAssert.assertThat;
+import static org.junit.jupiter.api.Assertions.assertTimeoutPreemptively;
+
+/**
+ * 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.

Review Comment:
   nit: `* more *` appears strange in the Javadoc reader. Use an appropriate tag or remove the asterisks.



##########
plus/src/test/java/org/apache/calcite/slt/SqlLogicTests.java:
##########
@@ -0,0 +1,337 @@
+/*
+ * 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.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.junit.jupiter.api.function.Executable;
+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.OutputStreamWriter;
+import java.nio.charset.StandardCharsets;
+import java.nio.file.Files;
+import java.time.Duration;
+import java.util.ArrayList;
+import java.util.Comparator;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+
+import static org.hamcrest.CoreMatchers.is;
+import static org.hamcrest.CoreMatchers.notNullValue;
+import static org.hamcrest.MatcherAssert.assertThat;
+import static org.junit.jupiter.api.Assertions.assertTimeoutPreemptively;
+
+/**
+ * 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

Review Comment:
   nit: Usually we don't enclose parameters in quotes ('other'). I haven't seen this pattern before.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@calcite.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [calcite] mihaibudiu commented on pull request #3145: [CALCITE-5615] Program to run SQL Logic Tests for Calcite

Posted by "mihaibudiu (via GitHub)" <gi...@apache.org>.
mihaibudiu commented on PR #3145:
URL: https://github.com/apache/calcite/pull/3145#issuecomment-1682362335

   We could also regenerate the golden file to account for the fixes made in the meantime. But I am on vacation for the next week so I can't do it right away.


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@calcite.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [calcite] zabetak commented on a diff in pull request #3145: [CALCITE-5615] Program to run SQL Logic Tests for Calcite

Posted by "zabetak (via GitHub)" <gi...@apache.org>.
zabetak commented on code in PR #3145:
URL: https://github.com/apache/calcite/pull/3145#discussion_r1166963052


##########
slt/src/main/java/org/apache/calcite/slt/Main.java:
##########
@@ -0,0 +1,208 @@
+/*
+ * Copyright 2022 VMware, Inc.
+ * SPDX-License-Identifier: MIT
+ * SPDX-License-Identifier: Apache-2.0
+ *
+ * Permission is hereby granted, free of charge, to any person obtaining a copy
+ * of this software and associated documentation files (the "Software"), to deal
+ * in the Software without restriction, including without limitation the rights
+ * to use, copy, modify, merge, publish, distribute, sublicense, and/or sell
+ * copies of the Software, and to permit persons to whom the Software is
+ * furnished to do so, subject to the following conditions:
+ *
+ * The above copyright notice and this permission notice shall be included in all
+ * copies or substantial portions of the Software.
+ *
+ * THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR
+ * IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY,
+ * FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE
+ * AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER
+ * LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM,
+ * OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN THE
+ * SOFTWARE.
+ *
+ *
+ */
+
+package org.apache.calcite.slt;
+
+import com.beust.jcommander.ParameterException;
+
+import org.apache.calcite.slt.executors.SqlSLTTestExecutor;
+import org.apache.calcite.sql.parser.SqlParseException;
+
+import javax.annotation.Nullable;
+import java.io.File;
+import java.io.FileOutputStream;
+import java.io.IOException;
+import java.io.InputStream;
+import java.net.URL;
+import java.nio.file.*;
+import java.nio.file.attribute.BasicFileAttributes;
+import java.security.NoSuchAlgorithmException;
+import java.sql.SQLException;
+import java.util.logging.Handler;

Review Comment:
   We are not using java.util.logging in the project; please use SLF4J instead.



##########
slt/src/main/java/org/apache/calcite/slt/Main.java:
##########
@@ -0,0 +1,208 @@
+/*
+ * Copyright 2022 VMware, Inc.
+ * SPDX-License-Identifier: MIT
+ * SPDX-License-Identifier: Apache-2.0
+ *
+ * Permission is hereby granted, free of charge, to any person obtaining a copy
+ * of this software and associated documentation files (the "Software"), to deal
+ * in the Software without restriction, including without limitation the rights
+ * to use, copy, modify, merge, publish, distribute, sublicense, and/or sell
+ * copies of the Software, and to permit persons to whom the Software is
+ * furnished to do so, subject to the following conditions:
+ *
+ * The above copyright notice and this permission notice shall be included in all
+ * copies or substantial portions of the Software.
+ *
+ * THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR
+ * IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY,
+ * FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE
+ * AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER
+ * LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM,
+ * OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN THE
+ * SOFTWARE.
+ *
+ *
+ */
+
+package org.apache.calcite.slt;
+
+import com.beust.jcommander.ParameterException;
+
+import org.apache.calcite.slt.executors.SqlSLTTestExecutor;
+import org.apache.calcite.sql.parser.SqlParseException;
+
+import javax.annotation.Nullable;
+import java.io.File;
+import java.io.FileOutputStream;
+import java.io.IOException;
+import java.io.InputStream;
+import java.net.URL;
+import java.nio.file.*;
+import java.nio.file.attribute.BasicFileAttributes;
+import java.security.NoSuchAlgorithmException;
+import java.sql.SQLException;
+import java.util.logging.Handler;
+import java.util.logging.Level;
+import java.util.logging.LogManager;
+import java.util.logging.Logger;
+import java.util.zip.ZipEntry;
+import java.util.zip.ZipInputStream;
+
+/**
+ * Execute all SqlLogicTest tests.
+ */
+public class Main {
+  static final String SLT_GIT = "https://github.com/gregrahn/sqllogictest/archive/refs/heads/master.zip";
+
+  static class TestLoader extends SimpleFileVisitor<Path> {
+    int errors = 0;
+    final TestStatistics statistics;
+    public final ExecutionOptions options;
+
+    /**
+     * Creates a new class that reads tests from a directory tree and executes them.
+     */
+    TestLoader(ExecutionOptions options) {
+      this.statistics = new TestStatistics(options.stopAtFirstError);
+      this.options = options;
+    }
+
+    @Override
+    public FileVisitResult visitFile(Path file, BasicFileAttributes attrs) {
+      SqlSLTTestExecutor executor;
+      try {
+        executor = this.options.getExecutor();
+      } catch (IOException | SQLException e) {
+        // Can't add exceptions to the overridden method visitFile
+        throw new RuntimeException(e);
+      }
+      String extension = Utilities.getFileExtension(file.toString());
+      if (attrs.isRegularFile() && extension != null && extension.equals("test")) {
+        SLTTestFile test = null;
+        try {
+          System.out.println("Running " + file);
+          test = new SLTTestFile(file.toString());
+          test.parse();
+        } catch (Exception ex) {
+          System.err.println("Error while executing test " + file + ": " + ex.getMessage());
+          this.errors++;
+        }
+        if (test != null) {
+          try {
+            TestStatistics stats = executor.execute(test, options);
+            this.statistics.add(stats);
+          } catch (SqlParseException | IOException |
+                   SQLException | NoSuchAlgorithmException ex) {
+            // Can't add exceptions to the overridden method visitFile
+            throw new IllegalArgumentException(ex);
+          }
+        }
+      }
+      return FileVisitResult.CONTINUE;
+    }
+  }
+
+  static void abort(ExecutionOptions options, @Nullable String message) {
+    if (message != null)
+      System.err.println(message);
+    options.usage();
+    System.exit(1);
+  }
+
+  @Nullable
+  static File newFile(File destinationDir, ZipEntry zipEntry) throws IOException {
+    String name = zipEntry.getName();
+    name = name.replace("sqllogictest-master/", "");
+    if (name.isEmpty())
+      return null;
+    File destFile = new File(destinationDir, name);
+    String destDirPath = destinationDir.getCanonicalPath();
+    String destFilePath = destFile.getCanonicalPath();
+    if (!destFilePath.startsWith(destDirPath + File.separator)) {
+      throw new IOException("Entry is outside of the target dir: " + name);
+    }
+    return destFile;
+  }
+
+  public static void install(File directory) throws IOException {
+    File zip = File.createTempFile("out", ".zip", new File("."));
+    System.out.println("Downloading SLT from " + SLT_GIT + " into " + zip.getAbsolutePath());

Review Comment:
   Is the use of `System.out` intentional? Is the message targetting a user or it is for logging purposes. If the second then please use a logger instead. Same comment to other places using `System.out` and `System.err`.



##########
slt/src/main/java/org/apache/calcite/slt/FieldsAreNonnullByDefault.java:
##########
@@ -0,0 +1,43 @@
+/*
+ * Copyright 2022 VMware, Inc.
+ * SPDX-License-Identifier: Apache-2.0
+ *
+ * Permission is hereby granted, free of charge, to any person obtaining a copy
+ * of this software and associated documentation files (the "Software"), to deal
+ * in the Software without restriction, including without limitation the rights
+ * to use, copy, modify, merge, publish, distribute, sublicense, and/or sell
+ * copies of the Software, and to permit persons to whom the Software is
+ * furnished to do so, subject to the following conditions:
+ *
+ * The above copyright notice and this permission notice shall be included in all
+ * copies or substantial portions of the Software.
+ *
+ * THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR
+ * IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY,
+ * FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE
+ * AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER
+ * LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM,
+ * OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN THE
+ * SOFTWARE.
+ *
+ *
+ */
+
+package org.apache.calcite.slt;
+
+import java.lang.annotation.Documented;
+import java.lang.annotation.ElementType;
+import java.lang.annotation.Retention;
+import java.lang.annotation.RetentionPolicy;
+import javax.annotation.Nonnull;

Review Comment:
   At the moment I think we are using `javax.annotation` only in one place. Let's avoid introducing more if possible:
   https://github.com/apache/calcite/blob/main/site/develop/index.md#null-safety



##########
slt/src/main/java/org/apache/calcite/slt/TestStatistics.java:
##########
@@ -0,0 +1,127 @@
+/*
+ * Copyright 2023 VMware, Inc.
+ * SPDX-License-Identifier: MIT
+ * SPDX-License-Identifier: Apache-2.0
+ *
+ * Permission is hereby granted, free of charge, to any person obtaining a copy
+ * of this software and associated documentation files (the "Software"), to deal
+ * in the Software without restriction, including without limitation the rights
+ * to use, copy, modify, merge, publish, distribute, sublicense, and/or sell
+ * copies of the Software, and to permit persons to whom the Software is
+ * furnished to do so, subject to the following conditions:
+ *
+ * The above copyright notice and this permission notice shall be included in all
+ * copies or substantial portions of the Software.
+ *
+ * THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR
+ * IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY,
+ * FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE
+ * AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER
+ * LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM,
+ * OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN THE
+ * SOFTWARE.
+ */
+
+package org.apache.calcite.slt;
+
+import java.text.DecimalFormat;
+import java.util.ArrayList;
+import java.util.List;
+
+public class TestStatistics {

Review Comment:
   Test prefix usually indicates a class containing tests; please rename.



##########
slt/src/main/java/org/apache/calcite/slt/Main.java:
##########
@@ -0,0 +1,208 @@
+/*
+ * Copyright 2022 VMware, Inc.
+ * SPDX-License-Identifier: MIT
+ * SPDX-License-Identifier: Apache-2.0
+ *
+ * Permission is hereby granted, free of charge, to any person obtaining a copy
+ * of this software and associated documentation files (the "Software"), to deal
+ * in the Software without restriction, including without limitation the rights
+ * to use, copy, modify, merge, publish, distribute, sublicense, and/or sell
+ * copies of the Software, and to permit persons to whom the Software is
+ * furnished to do so, subject to the following conditions:
+ *
+ * The above copyright notice and this permission notice shall be included in all
+ * copies or substantial portions of the Software.
+ *
+ * THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR
+ * IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY,
+ * FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE
+ * AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER
+ * LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM,
+ * OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN THE
+ * SOFTWARE.
+ *
+ *
+ */
+
+package org.apache.calcite.slt;
+
+import com.beust.jcommander.ParameterException;
+
+import org.apache.calcite.slt.executors.SqlSLTTestExecutor;
+import org.apache.calcite.sql.parser.SqlParseException;
+
+import javax.annotation.Nullable;
+import java.io.File;
+import java.io.FileOutputStream;
+import java.io.IOException;
+import java.io.InputStream;
+import java.net.URL;
+import java.nio.file.*;
+import java.nio.file.attribute.BasicFileAttributes;
+import java.security.NoSuchAlgorithmException;
+import java.sql.SQLException;
+import java.util.logging.Handler;
+import java.util.logging.Level;
+import java.util.logging.LogManager;
+import java.util.logging.Logger;
+import java.util.zip.ZipEntry;
+import java.util.zip.ZipInputStream;
+
+/**
+ * Execute all SqlLogicTest tests.
+ */
+public class Main {
+  static final String SLT_GIT = "https://github.com/gregrahn/sqllogictest/archive/refs/heads/master.zip";

Review Comment:
   Why do we need this? Can it change? What's inside the zip? Is it safe to download and execute? Can a malicious user change the content? 
   
   The respective repo does not have a license. Is it OK to use it?



##########
slt/src/main/java/org/apache/calcite/slt/Utilities.java:
##########
@@ -0,0 +1,71 @@
+/*
+ * 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 javax.annotation.Nullable;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.function.Function;
+
+public class Utilities {
+  private Utilities() {}
+
+  /**
+   * Just adds single quotes around a string.  No escaping is performed.
+   */
+  public static String singleQuote(String other) {
+    return "'" + other + "'";
+  }
+
+  @Nullable
+  public static String getFileExtension(String filename) {
+    int i = filename.lastIndexOf('.');
+    if (i > 0)
+      return filename.substring(i+1);
+    return null;
+  }
+
+  private static final char[] hexCode = "0123456789abcdef".toCharArray();
+
+  public static String toHex(byte[] data) {
+    StringBuilder r = new StringBuilder(data.length * 2);
+    for (byte b : data) {
+      r.append(hexCode[(b >> 4) & 0xF]);
+      r.append(hexCode[(b & 0xF)]);
+    }
+    return r.toString();
+  }
+
+  public static <T, S> List<S> map(List<T> data, Function<T, S> function) {
+    List<S> result = new ArrayList<>(data.size());
+    for (T aData : data)
+      result.add(function.apply(aData));
+    return result;
+  }
+
+  public static <T, S> List<S> flatMap(List<T> data, Function<T, List<S>> function) {
+    List<S> result = new ArrayList<>(data.size());
+    for (T aData : data)
+      result.addAll(function.apply(aData));
+    return result;
+  }

Review Comment:
   We have built-in JDK methods for map and flatMap. Do we really need these?



##########
slt/src/main/java/org/apache/calcite/slt/executors/CalciteExecutor.java:
##########
@@ -0,0 +1,129 @@
+/*
+ * Copyright 2023 VMware, Inc.
+ * SPDX-License-Identifier: MIT
+ * SPDX-License-Identifier: Apache-2.0
+ *
+ * Permission is hereby granted, free of charge, to any person obtaining a copy
+ * of this software and associated documentation files (the "Software"), to deal
+ * in the Software without restriction, including without limitation the rights
+ * to use, copy, modify, merge, publish, distribute, sublicense, and/or sell
+ * copies of the Software, and to permit persons to whom the Software is
+ * furnished to do so, subject to the following conditions:
+ *
+ * The above copyright notice and this permission notice shall be included in all
+ * copies or substantial portions of the Software.
+ *
+ * THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR
+ * IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY,
+ * FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE
+ * AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER
+ * LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM,
+ * OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN THE
+ * SOFTWARE.
+ */
+
+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 org.apache.calcite.slt.*;
+
+import java.io.IOException;
+import java.io.UnsupportedEncodingException;
+import java.security.NoSuchAlgorithmException;
+import java.sql.*;

Review Comment:
   Please avoid wildcard imports. I don't think we use/accept them in the project.



##########
slt/src/main/java/org/apache/calcite/slt/Utilities.java:
##########
@@ -0,0 +1,71 @@
+/*
+ * 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 javax.annotation.Nullable;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.function.Function;
+
+public class Utilities {
+  private Utilities() {}
+
+  /**
+   * Just adds single quotes around a string.  No escaping is performed.
+   */
+  public static String singleQuote(String other) {
+    return "'" + other + "'";
+  }
+
+  @Nullable
+  public static String getFileExtension(String filename) {
+    int i = filename.lastIndexOf('.');
+    if (i > 0)
+      return filename.substring(i+1);
+    return null;
+  }
+
+  private static final char[] hexCode = "0123456789abcdef".toCharArray();
+
+  public static String toHex(byte[] data) {
+    StringBuilder r = new StringBuilder(data.length * 2);
+    for (byte b : data) {
+      r.append(hexCode[(b >> 4) & 0xF]);
+      r.append(hexCode[(b & 0xF)]);
+    }
+    return r.toString();
+  }

Review Comment:
   We probably have this method somewhere in the code base or common deps. Isn't there something in Guava?



##########
slt/src/main/java/org/apache/calcite/slt/Utilities.java:
##########
@@ -0,0 +1,71 @@
+/*
+ * 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 javax.annotation.Nullable;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.function.Function;
+
+public class Utilities {
+  private Utilities() {}
+
+  /**
+   * Just adds single quotes around a string.  No escaping is performed.
+   */
+  public static String singleQuote(String other) {
+    return "'" + other + "'";
+  }
+
+  @Nullable
+  public static String getFileExtension(String filename) {
+    int i = filename.lastIndexOf('.');
+    if (i > 0)
+      return filename.substring(i+1);
+    return null;
+  }
+
+  private static final char[] hexCode = "0123456789abcdef".toCharArray();
+
+  public static String toHex(byte[] data) {
+    StringBuilder r = new StringBuilder(data.length * 2);
+    for (byte b : data) {
+      r.append(hexCode[(b >> 4) & 0xF]);
+      r.append(hexCode[(b & 0xF)]);
+    }
+    return r.toString();
+  }
+
+  public static <T, S> List<S> map(List<T> data, Function<T, S> function) {
+    List<S> result = new ArrayList<>(data.size());
+    for (T aData : data)
+      result.add(function.apply(aData));
+    return result;
+  }
+
+  public static <T, S> List<S> flatMap(List<T> data, Function<T, List<S>> function) {
+    List<S> result = new ArrayList<>(data.size());
+    for (T aData : data)
+      result.addAll(function.apply(aData));
+    return result;
+  }
+
+  public static String getEmptyString() {
+    return "";
+  }

Review Comment:
   More verbose than just having `""`. Why do we need this?



##########
slt/src/main/java/org/apache/calcite/slt/SLTTestFile.java:
##########
@@ -0,0 +1,292 @@
+/*
+ * Copyright 2022 VMware, Inc.
+ * SPDX-License-Identifier: MIT
+ * SPDX-License-Identifier: Apache-2.0
+ *
+ * Permission is hereby granted, free of charge, to any person obtaining a copy
+ * of this software and associated documentation files (the "Software"), to deal
+ * in the Software without restriction, including without limitation the rights
+ * to use, copy, modify, merge, publish, distribute, sublicense, and/or sell
+ * copies of the Software, and to permit persons to whom the Software is
+ * furnished to do so, subject to the following conditions:
+ *
+ * The above copyright notice and this permission notice shall be included in all
+ * copies or substantial portions of the Software.
+ *
+ * THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR
+ * IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY,
+ * FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE
+ * AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER
+ * LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM,
+ * OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN THE
+ * SOFTWARE.
+ *
+ *
+ */
+
+package org.apache.calcite.slt;
+
+import java.io.BufferedReader;
+import java.io.File;
+import java.io.FileReader;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.logging.Logger;
+import javax.annotation.Nullable;
+
+/**
+ * Represents the data from a .test file from the
+ * SqlLogicTest test framework.
+ */
+/*
+ *         The Test file format is described at
+ *         https://www.sqlite.org/sqllogictest/doc/tip/about.wiki.
+ *
+ *         Here is an example:
+ *
+ *         hash-threshold 8
+ *
+ *         statement ok
+ *         CREATE TABLE t1(a INTEGER, b INTEGER, c INTEGER, d INTEGER, e INTEGER)
+ *
+ *         statement ok
+ *         INSERT INTO t1(e,c,b,d,a) VALUES(NULL,102,NULL,101,104)
+ *
+ *         statement ok
+ *         INSERT INTO t1(a,c,d,e,b) VALUES(107,106,108,109,105)
+ *
+ *         query I nosort
+ *         SELECT CASE WHEN c>(SELECT avg(c) FROM t1) THEN a*2 ELSE b*10 END
+ *           FROM t1
+ *          ORDER BY 1
+ *         ----
+ *         30 values hashing to 3c13dee48d9356ae19af2515e05e6b54
+ *
+ */
+public class SLTTestFile {
+  Logger logger = Logger.getLogger("SLTTestFile");

Review Comment:
   Aren't loggers always `private static final`? Do we need to create one per instance?



##########
slt/src/main/java/org/apache/calcite/slt/ExecutionOptions.java:
##########
@@ -0,0 +1,176 @@
+/*
+ * Copyright 2022 VMware, Inc.
+ * SPDX-License-Identifier: MIT
+ * SPDX-License-Identifier: Apache-2.0
+ *
+ * Permission is hereby granted, free of charge, to any person obtaining a copy
+ * of this software and associated documentation files (the "Software"), to deal
+ * in the Software without restriction, including without limitation the rights
+ * to use, copy, modify, merge, publish, distribute, sublicense, and/or sell
+ * copies of the Software, and to permit persons to whom the Software is
+ * furnished to do so, subject to the following conditions:
+ *
+ * The above copyright notice and this permission notice shall be included in all
+ * copies or substantial portions of the Software.
+ *
+ * THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR
+ * IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY,
+ * FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE
+ * AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER
+ * LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM,
+ * OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN THE
+ * SOFTWARE.
+ */
+
+package org.apache.calcite.slt;
+
+import com.beust.jcommander.IParameterValidator;
+import com.beust.jcommander.JCommander;
+import com.beust.jcommander.Parameter;
+import com.beust.jcommander.ParameterException;
+
+import org.apache.calcite.slt.executors.CalciteExecutor;
+
+import org.apache.calcite.slt.executors.JDBCExecutor;
+import org.apache.calcite.slt.executors.NoExecutor;
+
+import org.apache.calcite.slt.executors.SqlSLTTestExecutor;
+
+import java.io.BufferedReader;
+import java.io.File;
+import java.io.FileReader;
+import java.io.IOException;
+import java.sql.SQLException;
+import java.util.ArrayList;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Set;
+import javax.annotation.Nullable;
+
+@SuppressWarnings("CanBeFinal")
+public class ExecutionOptions {

Review Comment:
   All public classes should have at least some top level javadoc. I think there are even checks for enforcing this.



##########
slt/src/main/java/org/apache/calcite/slt/SLTTestFile.java:
##########
@@ -0,0 +1,292 @@
+/*
+ * Copyright 2022 VMware, Inc.
+ * SPDX-License-Identifier: MIT
+ * SPDX-License-Identifier: Apache-2.0
+ *
+ * Permission is hereby granted, free of charge, to any person obtaining a copy
+ * of this software and associated documentation files (the "Software"), to deal
+ * in the Software without restriction, including without limitation the rights
+ * to use, copy, modify, merge, publish, distribute, sublicense, and/or sell
+ * copies of the Software, and to permit persons to whom the Software is
+ * furnished to do so, subject to the following conditions:
+ *
+ * The above copyright notice and this permission notice shall be included in all
+ * copies or substantial portions of the Software.
+ *
+ * THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR
+ * IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY,
+ * FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE
+ * AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER
+ * LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM,
+ * OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN THE
+ * SOFTWARE.
+ *
+ *
+ */
+
+package org.apache.calcite.slt;
+
+import java.io.BufferedReader;
+import java.io.File;
+import java.io.FileReader;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.logging.Logger;
+import javax.annotation.Nullable;
+
+/**
+ * Represents the data from a .test file from the
+ * SqlLogicTest test framework.
+ */
+/*

Review Comment:
   It's confusing to have javadoc and non-javadoc comment coming one after the other. Please combine them together and use proper javadoc annotations to ensure it displays properly.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@calcite.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [calcite] github-code-scanning[bot] commented on a diff in pull request #3145: [CALCITE-5615] Program to run SQL Logic Tests for Calcite

Posted by "github-code-scanning[bot] (via GitHub)" <gi...@apache.org>.
github-code-scanning[bot] commented on code in PR #3145:
URL: https://github.com/apache/calcite/pull/3145#discussion_r1165863795


##########
slt/src/main/java/org/apache/calcite/slt/executors/JDBCExecutor.java:
##########
@@ -0,0 +1,348 @@
+/*
+ * Copyright 2022 VMware, Inc.
+ * SPDX-License-Identifier: MIT
+ * SPDX-License-Identifier: Apache-2.0
+ *
+ * Permission is hereby granted, free of charge, to any person obtaining a copy
+ * of this software and associated documentation files (the "Software"), to deal
+ * in the Software without restriction, including without limitation the rights
+ * to use, copy, modify, merge, publish, distribute, sublicense, and/or sell
+ * copies of the Software, and to permit persons to whom the Software is
+ * furnished to do so, subject to the following conditions:
+ *
+ * The above copyright notice and this permission notice shall be included in all
+ * copies or substantial portions of the Software.
+ *
+ * THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR
+ * IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY,
+ * FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE
+ * AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER
+ * LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM,
+ * OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN THE
+ * SOFTWARE.
+ */
+
+package org.apache.calcite.slt.executors;
+
+import org.apache.calcite.slt.*;
+
+import java.security.MessageDigest;
+import java.security.NoSuchAlgorithmException;
+import java.sql.*;
+import java.util.ArrayList;
+import java.util.Comparator;
+import java.util.List;
+import java.util.logging.Logger;
+import javax.annotation.Nullable;
+
+@SuppressWarnings({"SqlDialectInspection", "SqlNoDataSourceInspection"})
+public class JDBCExecutor extends SqlSLTTestExecutor {
+  Logger logger = Logger.getLogger("JDBCExecutor");
+  public final String dbUrl;
+  @Nullable
+  Connection connection;
+
+  // These could be static final, but then perhaps this would be flagged as a security vulnerability.
+  // There is no security issue because we use a local temporary hsqldb database.
+  String DEFAULT_USER = "";  // no user needed for hsqldb
+  String DEFAULT_PASSWORD = "";  // no password needed for hsqldb
+
+  // In the end everything is decoded as a string
+  static class Row {
+    public final List<String> values;
+
+    Row() {
+      this.values = new ArrayList<>();
+    }
+
+    void add(String v) {
+      this.values.add(v);
+    }
+
+    @Override
+    public String toString() {
+      return String.join("\n", this.values);
+    }
+  }
+
+  static class Rows {
+    List<Row> allRows;
+
+    Rows() {
+      this.allRows = new ArrayList<>();
+    }
+
+    void add(Row row) {
+      this.allRows.add(row);
+    }
+
+    @Override
+    public String toString() {
+      return String.join("\n", Utilities.map(this.allRows, Row::toString));
+    }
+
+    public int size() {
+      return this.allRows.size();
+    }
+
+    public void sort(SqlTestQueryOutputDescription.SortOrder order) {
+      switch (order) {
+      case NONE:
+        break;
+      case ROW:
+        this.allRows.sort(new RowComparator());
+        break;
+      case VALUE:
+        this.allRows = Utilities.flatMap(this.allRows,
+            r -> Utilities.map(r.values,
+                r0 -> {
+                  Row res = new Row();
+                  res.add(r0);
+                  return res;
+                }));
+        this.allRows.sort(new RowComparator());
+        break;
+      }
+    }
+
+    List<Row> getRows() {
+      return this.allRows;
+    }
+  }
+
+  public JDBCExecutor(String db_url) {
+    this.dbUrl = db_url;
+    this.connection = null;
+  }
+
+  void statement(SLTSqlStatement statement) throws SQLException {
+    logger.info(() -> this.statementsExecuted + ": " + statement.statement);
+    assert this.connection != null;
+    try (Statement stmt = this.connection.createStatement()) {
+      stmt.execute(statement.statement);
+    } catch (SQLException ex) {
+      logger.severe("ERROR: " + ex.getMessage());
+      // Failures during the execution of statements are fatal.
+      // Only failures in queries are handled.
+      throw ex;
+    }
+    this.statementsExecuted++;
+  }
+
+  void query(SqlTestQuery query, TestStatistics statistics) throws SQLException, NoSuchAlgorithmException {
+    assert this.connection != null;
+    if (this.buggyOperations.contains(query.getQuery())) {
+      logger.warning(() -> "Skipping " + query.getQuery());
+      return;
+    }
+    try (Statement stmt = this.connection.createStatement()) {
+      ResultSet resultSet = stmt.executeQuery(query.getQuery());
+      this.validate(query, resultSet, query.outputDescription, statistics);
+      resultSet.close();
+    }
+    logger.info(() -> statistics.testsRun() + ": " + query.getQuery());
+  }
+
+  Row getValue(ResultSet rs, String columnTypes) throws SQLException {
+    Row row = new Row();
+    // Column numbers start from 1
+    for (int i = 1; i <= columnTypes.length(); i++) {
+      char c = columnTypes.charAt(i - 1);
+      switch (c) {
+      case 'R':
+        double d = rs.getDouble(i);
+        if (rs.wasNull())
+          row.add("NULL");
+        else
+          row.add(String.format("%.3f", d));
+        break;
+      case 'I':
+        try {
+          long integer = rs.getLong(i);
+          if (rs.wasNull())
+            row.add("NULL");
+          else
+            row.add(String.format("%d", integer));
+        } catch (SQLDataException | NumberFormatException ignore) {
+          // This probably indicates a bug in the query, since
+          // the query expects an integer, but the result cannot
+          // be interpreted as such.
+          // unparsable string: replace with 0
+          row.add("0");
+        }
+        break;
+      case 'T':
+        String s = rs.getString(i);
+        if (s == null)
+          row.add("NULL");
+        else {
+          StringBuilder result = new StringBuilder();
+          for (int j = 0; j < s.length(); j++) {
+            char sc = s.charAt(j);
+            if (sc < ' ' || sc > '~')
+              sc = '@';
+            result.append(sc);
+          }
+          row.add(result.toString());
+        }
+        break;
+      default:
+        throw new RuntimeException("Unexpected column type " + c);
+      }
+    }
+    return row;
+  }
+
+  static class RowComparator implements Comparator<Row> {
+    @Override
+    public int compare(Row o1, Row o2) {
+      if (o1.values.size() != o2.values.size())
+        throw new RuntimeException("Comparing rows of different lengths");
+      for (int i = 0; i < o1.values.size(); i++) {
+        int r = o1.values.get(i).compareTo(o2.values.get(i));
+        if (r != 0)
+          return r;
+      }
+      return 0;
+    }
+  }
+
+  @SuppressWarnings("java:S4790")  // MD5 checksum
+  void validate(SqlTestQuery query, ResultSet rs,
+      SqlTestQueryOutputDescription description,
+      TestStatistics statistics)
+      throws SQLException, NoSuchAlgorithmException {
+    assert description.columnTypes != null;
+    Rows rows = new Rows();
+    while (rs.next()) {
+      Row row = this.getValue(rs, description.columnTypes);
+      rows.add(row);
+    }
+    if (description.getValueCount() != rows.size() * description.columnTypes.length()) {
+      statistics.addFailure(new TestStatistics.FailedTestDescription(
+          query, "Expected " + description.getValueCount() + " rows, got " +
+          rows.size() * description.columnTypes.length()));
+      return;
+    }
+    rows.sort(description.getOrder());
+    if (description.getQueryResults() != null) {
+      String r = rows.toString();
+      String q = String.join("\n", description.getQueryResults());
+      if (!r.equals(q)) {
+        statistics.addFailure(new TestStatistics.FailedTestDescription(
+            query, "Output differs: computed\n" + r + "\nExpected:\n" + q));
+        return;
+      }
+    }
+    if (description.hash != null) {
+      // MD5 is considered insecure, but we have no choice because this is
+      // the algorithm used to compute the checksums by SLT.
+      MessageDigest md = MessageDigest.getInstance("MD5");
+      String repr = rows + "\n";
+      md.update(repr.getBytes());
+      byte[] digest = md.digest();
+      String hash = Utilities.toHex(digest);
+      if (!description.hash.equals(hash)) {
+        statistics.addFailure(new TestStatistics.FailedTestDescription(
+            query, "Hash of data does not match expected value"));
+        return;
+      }
+    }
+    statistics.incPassed();
+  }
+
+  List<String> getTableList() throws SQLException {
+    List<String> result = new ArrayList<>();
+    assert this.connection != null;
+    DatabaseMetaData md = this.connection.getMetaData();
+    ResultSet rs = md.getTables(null, null, "%", new String[]{"TABLE"});
+    while (rs.next()) {
+      String tableName = rs.getString(3);
+      if (tableName.equals("PUBLIC"))
+        // The catalog table in HSQLDB
+        continue;
+      result.add(tableName);
+    }
+    rs.close();
+    return result;
+  }
+
+  List<String> getViewList() throws SQLException {
+    List<String> result = new ArrayList<>();
+    assert this.connection != null;
+    DatabaseMetaData md = this.connection.getMetaData();
+    ResultSet rs = md.getTables(null, null, "%", new String[]{"VIEW"});
+    while (rs.next()) {
+      String tableName = rs.getString(3);
+      result.add(tableName);
+    }
+    rs.close();
+    return result;
+  }
+
+  void dropAllTables() throws SQLException {
+    assert this.connection != null;
+    List<String> tables = this.getTableList();
+    for (String tableName : tables) {
+      String del = "DROP TABLE ?";
+      logger.info(del);
+      try (PreparedStatement drop = this.connection.prepareStatement(del)) {
+        drop.setString(1, tableName);
+        drop.execute(del);
+      }
+    }
+  }
+
+  void dropAllViews() throws SQLException {
+    assert this.connection != null;
+    List<String> tables = this.getViewList();
+    for (String tableName : tables) {
+      String del = "DROP VIEW IF EXISTS ? CASCADE";
+      logger.info(del);
+      try (PreparedStatement drop = this.connection.prepareStatement(del)) {
+        drop.setString(1, tableName);
+        drop.execute(del);
+      }
+    }
+  }
+
+  public void establishConnection() throws SQLException {
+    this.connection = DriverManager.getConnection(this.dbUrl, DEFAULT_USER, DEFAULT_PASSWORD);

Review Comment:
   ## A secure password should be used when connecting to a database
   
   <!--SONAR_ISSUE_KEY:AYd7xlZJjm25RUNvuIKp-->Add password protection to this database. <p>See more on <a href="https://sonarcloud.io/project/issues?id=apache_calcite&issues=AYd7xlZJjm25RUNvuIKp&open=AYd7xlZJjm25RUNvuIKp&pullRequest=3145">SonarCloud</a></p>
   
   [Show more details](https://github.com/apache/calcite/security/code-scanning/3)



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@calcite.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [calcite] sonarcloud[bot] commented on pull request #3145: [CALCITE-5615] Program to run SQL Logic Tests for Calcite

Posted by "sonarcloud[bot] (via GitHub)" <gi...@apache.org>.
sonarcloud[bot] commented on PR #3145:
URL: https://github.com/apache/calcite/pull/3145#issuecomment-1507660531

   Kudos, SonarCloud Quality Gate passed!&nbsp; &nbsp; [![Quality Gate passed](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/QualityGateBadge/passed-16px.png 'Quality Gate passed')](https://sonarcloud.io/dashboard?id=apache_calcite&pullRequest=3145)
   
   [![Bug](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/bug-16px.png 'Bug')](https://sonarcloud.io/project/issues?id=apache_calcite&pullRequest=3145&resolved=false&types=BUG) [![A](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A-16px.png 'A')](https://sonarcloud.io/project/issues?id=apache_calcite&pullRequest=3145&resolved=false&types=BUG) [0 Bugs](https://sonarcloud.io/project/issues?id=apache_calcite&pullRequest=3145&resolved=false&types=BUG)  
   [![Vulnerability](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/vulnerability-16px.png 'Vulnerability')](https://sonarcloud.io/project/issues?id=apache_calcite&pullRequest=3145&resolved=false&types=VULNERABILITY) [![A](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A-16px.png 'A')](https://sonarcloud.io/project/issues?id=apache_calcite&pullRequest=3145&resolved=false&types=VULNERABILITY) [0 Vulnerabilities](https://sonarcloud.io/project/issues?id=apache_calcite&pullRequest=3145&resolved=false&types=VULNERABILITY)  
   [![Security Hotspot](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/security_hotspot-16px.png 'Security Hotspot')](https://sonarcloud.io/project/security_hotspots?id=apache_calcite&pullRequest=3145&resolved=false&types=SECURITY_HOTSPOT) [![A](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A-16px.png 'A')](https://sonarcloud.io/project/security_hotspots?id=apache_calcite&pullRequest=3145&resolved=false&types=SECURITY_HOTSPOT) [0 Security Hotspots](https://sonarcloud.io/project/security_hotspots?id=apache_calcite&pullRequest=3145&resolved=false&types=SECURITY_HOTSPOT)  
   [![Code Smell](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/code_smell-16px.png 'Code Smell')](https://sonarcloud.io/project/issues?id=apache_calcite&pullRequest=3145&resolved=false&types=CODE_SMELL) [![A](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A-16px.png 'A')](https://sonarcloud.io/project/issues?id=apache_calcite&pullRequest=3145&resolved=false&types=CODE_SMELL) [36 Code Smells](https://sonarcloud.io/project/issues?id=apache_calcite&pullRequest=3145&resolved=false&types=CODE_SMELL)
   
   [![0.0%](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/CoverageChart/0-16px.png '0.0%')](https://sonarcloud.io/component_measures?id=apache_calcite&pullRequest=3145&metric=new_coverage&view=list) [0.0% Coverage](https://sonarcloud.io/component_measures?id=apache_calcite&pullRequest=3145&metric=new_coverage&view=list)  
   [![0.0%](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/Duplications/3-16px.png '0.0%')](https://sonarcloud.io/component_measures?id=apache_calcite&pullRequest=3145&metric=new_duplicated_lines_density&view=list) [0.0% Duplication](https://sonarcloud.io/component_measures?id=apache_calcite&pullRequest=3145&metric=new_duplicated_lines_density&view=list)
   
   


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@calcite.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [calcite] zabetak commented on pull request #3145: [CALCITE-5615] Program to run SQL Logic Tests for Calcite

Posted by "zabetak (via GitHub)" <gi...@apache.org>.
zabetak commented on PR #3145:
URL: https://github.com/apache/calcite/pull/3145#issuecomment-1509115056

   @mbudiu-vmw Sure it makes sense to wait a bit. I will actually run the code during the weekend; I am already convinced that it is a very useful contribution :) I am just not sure yet where/how it should land. Thanks again for taking the time to contribute this to the community.


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@calcite.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [calcite] zabetak commented on a diff in pull request #3145: [CALCITE-5615] Program to run SQL Logic Tests for Calcite

Posted by "zabetak (via GitHub)" <gi...@apache.org>.
zabetak commented on code in PR #3145:
URL: https://github.com/apache/calcite/pull/3145#discussion_r1205976037


##########
plus/src/test/java/org/apache/calcite/slt/executors/CalciteExecutor.java:
##########
@@ -0,0 +1,170 @@
+/*
+ * 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.ISqlTestOperation;
+import net.hydromatic.sqllogictest.OptionsParser;
+import net.hydromatic.sqllogictest.SltSqlStatement;
+import net.hydromatic.sqllogictest.SltTestFile;
+import net.hydromatic.sqllogictest.SqlTestQuery;
+import net.hydromatic.sqllogictest.TestStatistics;
+import net.hydromatic.sqllogictest.executors.HsqldbExecutor;
+import net.hydromatic.sqllogictest.executors.JdbcExecutor;
+import net.hydromatic.sqllogictest.executors.SqlSltTestExecutor;
+
+import java.io.IOException;
+import java.security.NoSuchAlgorithmException;
+import java.sql.Connection;
+import java.sql.DriverManager;
+import java.sql.PreparedStatement;
+import java.sql.ResultSet;
+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 SqlSltTestExecutor {

Review Comment:
   I cannot find them either so let's consider this resolved :D Maybe they never came out of my mind!



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@calcite.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [calcite] julianhyde commented on pull request #3145: [CALCITE-5615] Program to run SQL Logic Tests for Calcite

Posted by "julianhyde (via GitHub)" <gi...@apache.org>.
julianhyde commented on PR #3145:
URL: https://github.com/apache/calcite/pull/3145#issuecomment-1563491610

   Rather than checking that the number of failures doesn't change, it's better to check that there are no failures other than the list of 'expected failures'
   
   You don't want the situation where you fix one test and introduce a new failure. It's much better to have a list of known failures rather than just a count.


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@calcite.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [calcite] mihaibudiu commented on a diff in pull request #3145: [CALCITE-5615] Program to run SQL Logic Tests for Calcite

Posted by "mihaibudiu (via GitHub)" <gi...@apache.org>.
mihaibudiu commented on code in PR #3145:
URL: https://github.com/apache/calcite/pull/3145#discussion_r1186545906


##########
slt/src/main/java/org/apache/calcite/slt/Main1.java:
##########
@@ -0,0 +1,77 @@
+/*
+ * 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 java.io.IOException;
+import java.io.PrintStream;
+import java.net.URL;
+import java.nio.file.Files;
+import java.nio.file.Path;
+import java.nio.file.Paths;
+
+import net.hydromatic.sqllogictest.ExecutionOptions;
+import net.hydromatic.sqllogictest.executors.HsqldbExecutor;
+import net.hydromatic.sqllogictest.executors.NoExecutor;
+import net.hydromatic.sqllogictest.TestLoader;
+
+import org.apache.calcite.slt.executors.CalciteExecutor;
+
+/**
+ * Execute all SqlLogicTest tests.
+ */
+public class Main1 {

Review Comment:
   It contains the 'main' entry point.
   I am not sure how SLT will be used by Calcite yet, so the plan was to provide an executable that people could run to find bugs. Until the bugs it finds are fixed I don't know exactly how it can be integrated in the test suite.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@calcite.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [calcite] zabetak commented on pull request #3145: [CALCITE-5615] Program to run SQL Logic Tests for Calcite

Posted by "zabetak (via GitHub)" <gi...@apache.org>.
zabetak commented on PR #3145:
URL: https://github.com/apache/calcite/pull/3145#issuecomment-1614413964

   Apart from `Slow` there is no other speed tag currently in Calcite. 
   
   Let's keep it simple for the time being and use three buckets:
   * Fast bucket (under a minute) with no special annotation running in unit tests
   * Slow bucket (5-10 minutes) tagged with the existing slow annotation
   * Very slow bucket (over 10 minutes) disabled for the time being; to be discussed and addressed in a follow-up PR.
   
   WDYT?


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@calcite.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org