You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@calcite.apache.org by "zabetak (via GitHub)" <gi...@apache.org> on 2023/04/14 16:13:09 UTC

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

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