You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@calcite.apache.org by jh...@apache.org on 2023/06/08 23:28:06 UTC

[calcite] 06/06: [CALCITE-5765] Add LintTest, to apply custom lint rules to source code

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

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

commit c0e6ba264b3e522010a982ed35de9f3dd03be6af
Author: Julian Hyde <jh...@apache.org>
AuthorDate: Sun Jun 4 11:24:38 2023 -0700

    [CALCITE-5765] Add LintTest, to apply custom lint rules to source code
    
    LintTest.testLint applies rules to Calcite's java source files,
    most of them concerned with the formatting of javadoc comments,
    and fails if any of the rules give warnings.
    
    LintTest.testLog verifies recent git commit messages. The
    goal is to prevent malformed messages that begin with a
    space, end in a '.', or contain the word 'fix'. (Many commit
    messages that contain the word 'fix' are describing the code
    change; they should instead describe the problem seen by the
    user, which is what will be most useful to those reading the
    release notes. If the message describes the problem, the
    word 'fix' is implicit.)
    
    Also create method `TestUtil.getBase(File)`, based on some
    logic previously in `DocumentationTest`.
    
    Close apache/calcite#3250
---
 .../apache/calcite/sql/test/DocumentationTest.java |  32 +--
 .../java/org/apache/calcite/test/LintTest.java     | 283 +++++++++++++++++++++
 .../java/org/apache/calcite/util/TestUnsafe.java   |  97 +++++++
 .../java/org/apache/calcite/util/TestUtil.java     |  41 ++-
 4 files changed, 421 insertions(+), 32 deletions(-)

diff --git a/core/src/test/java/org/apache/calcite/sql/test/DocumentationTest.java b/core/src/test/java/org/apache/calcite/sql/test/DocumentationTest.java
index 5085a38a68..dda80bc7b0 100644
--- a/core/src/test/java/org/apache/calcite/sql/test/DocumentationTest.java
+++ b/core/src/test/java/org/apache/calcite/sql/test/DocumentationTest.java
@@ -27,7 +27,7 @@ import org.apache.calcite.sql.fun.SqlStdOperatorTable;
 import org.apache.calcite.sql.parser.SqlAbstractParserImpl;
 import org.apache.calcite.sql.parser.SqlParserTest;
 import org.apache.calcite.test.DiffTestCase;
-import org.apache.calcite.util.Sources;
+import org.apache.calcite.util.TestUtil;
 import org.apache.calcite.util.Util;
 
 import org.junit.jupiter.api.Test;
@@ -50,7 +50,6 @@ import java.util.stream.Collectors;
 
 import static org.hamcrest.CoreMatchers.is;
 import static org.hamcrest.MatcherAssert.assertThat;
-import static org.junit.jupiter.api.Assertions.fail;
 
 /** Various automated checks on the documentation. */
 class DocumentationTest {
@@ -189,35 +188,8 @@ class DocumentationTest {
     final File inFile;
     final File outFile;
 
-    private boolean isProjectDir(File dir) {
-      return new File(dir, "pom.xml").isFile()
-          || new File(dir, "build.gradle.kts").isFile()
-          || new File(dir, "gradle.properties").isFile();
-    }
-
     FileFixture() {
-      // Algorithm:
-      // 1) Find location of DocumentationTest.class
-      // 2) Climb via getParentFile() until we detect pom.xml
-      // 3) It means we've got core/pom.xml, and we need to get core/../site/
-      Class<DocumentationTest> klass = DocumentationTest.class;
-      File docTestClass =
-          Sources.of(klass.getResource(klass.getSimpleName() + ".class")).file();
-
-      File core = docTestClass.getAbsoluteFile();
-      for (int i = 0; i < 42; i++) {
-        if (isProjectDir(core)) {
-          // Ok, core == core/
-          break;
-        }
-        core = core.getParentFile();
-      }
-      if (!isProjectDir(core)) {
-        fail("Unable to find neither core/pom.xml nor core/build.gradle.kts. Started with "
-            + docTestClass.getAbsolutePath()
-            + ", the current path is " + core.getAbsolutePath());
-      }
-      base = core.getParentFile();
+      base = TestUtil.getBaseDir(DocumentationTest.class);
       inFile = new File(base, "site/_docs/reference.md");
       // TODO: replace with core/build/ when Maven is migrated to Gradle
       // It does work in Gradle, however, we don't want to create "target" folder in Gradle
diff --git a/core/src/test/java/org/apache/calcite/test/LintTest.java b/core/src/test/java/org/apache/calcite/test/LintTest.java
new file mode 100644
index 0000000000..d84ea99dc0
--- /dev/null
+++ b/core/src/test/java/org/apache/calcite/test/LintTest.java
@@ -0,0 +1,283 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to you under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.calcite.test;
+
+import org.apache.calcite.util.Puffin;
+import org.apache.calcite.util.Source;
+import org.apache.calcite.util.Sources;
+import org.apache.calcite.util.TestUnsafe;
+import org.apache.calcite.util.Util;
+
+import org.junit.jupiter.api.Test;
+
+import java.io.File;
+import java.io.PrintWriter;
+import java.io.StringWriter;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.function.Consumer;
+import java.util.function.Function;
+import java.util.stream.Stream;
+
+import static org.hamcrest.CoreMatchers.is;
+import static org.hamcrest.MatcherAssert.assertThat;
+import static org.hamcrest.Matchers.empty;
+import static org.hamcrest.Matchers.hasItem;
+import static org.hamcrest.Matchers.hasSize;
+import static org.junit.jupiter.api.Assumptions.assumeTrue;
+
+/** Various automated checks on the code and git history. */
+class LintTest {
+  @SuppressWarnings("Convert2MethodRef") // JDK 8 requires lambdas
+  private Puffin.Program<GlobalState> makeProgram() {
+    return Puffin.builder(GlobalState::new, global -> new FileState(global))
+        .add(line -> line.fnr() == 1,
+            line -> line.globalState().fileCount++)
+
+        // Javadoc does not require '</p>', so we do not allow '</p>'
+        .add(line -> line.state().inJavadoc()
+                && line.contains("</p>"),
+            line -> line.state().message("no '</p>'", line))
+
+        // No "**/"
+        .add(line -> line.contains("**/")
+                && line.state().inJavadoc(),
+            line ->
+                line.state().message("no '**/'; use '*/'",
+                    line))
+
+        // A Javadoc paragraph '<p>' must not be on its own line.
+        .add(line -> line.matches("^ *\\* <p>"),
+            line ->
+                line.state().message("<p> must not be on its own line",
+                    line))
+
+        // A Javadoc paragraph '<p>' must be preceded by a blank Javadoc
+        // line.
+        .add(line -> line.matches("^ *\\*"),
+            line -> line.state().starLine = line.fnr())
+        .add(line -> line.matches("^ *\\* <p>.*")
+                && line.fnr() - 1 != line.state().starLine,
+            line ->
+                line.state().message("<p> must be preceded by blank line",
+                    line))
+
+        // The first "@param" of a javadoc block must be preceded by a blank
+        // line.
+        .add(line -> line.matches("^ */\\*\\*.*"),
+            line -> line.state().javadocStartLine = line.fnr())
+        .add(line -> line.matches(".*\\*/"),
+            line -> line.state().javadocEndLine = line.fnr())
+        .add(line -> line.matches("^ *\\* @.*"),
+            line -> {
+              if (line.state().inJavadoc()
+                  && line.state().atLine < line.state().javadocStartLine
+                  && line.fnr() - 1 != line.state().starLine) {
+                line.state().message(
+                    "First @tag must be preceded by blank line",
+                    line);
+              }
+              line.state().atLine = line.fnr();
+            })
+        .build();
+  }
+
+  @Test void testProgramWorks() {
+    final String code = "class MyClass {\n"
+        + "  /** Paragraph.\n"
+        + "   *\n"
+        + "   * Missing p.\n"
+        + "   *\n"
+        + "   * <p>\n"
+        + "   * <p>A paragraph (p must be preceded by blank line).\n"
+        + "   *\n"
+        + "   * <p>no p</p>\n"
+        + "   * @see java.lang.String (should be preceded by blank line)\n"
+        + "   **/\n"
+        + "  String x = \"ok because it's not in javadoc:</p>\";\n"
+        + "}\n";
+    final String expectedMessages = "["
+        + "GuavaCharSource{memory}:6:"
+        + "<p> must not be on its own line\n"
+        + "GuavaCharSource{memory}:7:"
+        + "<p> must be preceded by blank line\n"
+        + "GuavaCharSource{memory}:9:"
+        + "no '</p>'\n"
+        + "GuavaCharSource{memory}:10:"
+        + "First @tag must be preceded by blank line\n"
+        + "GuavaCharSource{memory}:11:"
+        + "no '**/'; use '*/']";
+    final Puffin.Program<GlobalState> program = makeProgram();
+    final StringWriter sw = new StringWriter();
+    final GlobalState g;
+    try (PrintWriter pw = new PrintWriter(sw)) {
+      g = program.execute(Stream.of(Sources.of(code)), pw);
+    }
+    assertThat(g.messages.toString().replace(", ", "\n"),
+        is(expectedMessages));
+  }
+
+  /** Tests that source code has no flaws. */
+  @Test void testLint() {
+    assumeTrue(TestUnsafe.haveGit(), "Invalid git environment");
+
+    final Puffin.Program<GlobalState> program = makeProgram();
+    final List<File> javaFiles = TestUnsafe.getJavaFiles();
+
+    final GlobalState g;
+    try (PrintWriter pw = Util.printWriter(System.out)) {
+      g = program.execute(javaFiles.parallelStream().map(Sources::of), pw);
+    }
+
+    g.messages.forEach(System.out::println);
+    assertThat(g.messages, empty());
+  }
+
+  /** Tests that the most recent N commit messages are good.
+   *
+   * <p>N needs to be large enough to verify multi-commit PRs, but not so large
+   * that it fails because of historical commits. */
+  @Test void testLintLog() {
+    assumeTrue(TestUnsafe.haveGit(), "Invalid git environment");
+
+    int n = 7;
+    final List<String> messages = TestUnsafe.getCommitMessages(n);
+    final List<String> warnings = new ArrayList<>();
+    for (String message : messages) {
+      checkMessage(message, warning ->
+          warnings.add("invalid git log message '" + message + "'; "
+              + warning));
+    }
+    warnings.forEach(System.out::println);
+    assertThat(warnings, empty());
+  }
+
+  @Test void testLogMatcher() {
+    final Function<String, List<String>> f = message -> {
+      final List<String> warnings = new ArrayList<>();
+      checkMessage(message, warnings::add);
+      return warnings;
+    };
+    assertThat(f.apply(" [CALCITE-1234] abc"),
+        hasItem("starts with space"));
+    assertThat(f.apply("[CALCITE-1234]  abc"),
+        hasItem("malformed [CALCITE-nnnn] reference"));
+    assertThat(f.apply("[CALCITE-12b]  abc"),
+        hasItem("malformed [CALCITE-nnnn] reference"));
+    assertThat(f.apply("[CALCITE-12345]  abc"),
+        hasItem("malformed [CALCITE-nnnn] reference"));
+    assertThat(f.apply("[CALCITE-1234]: abc"),
+        hasItem("malformed [CALCITE-nnnn] reference"));
+    assertThat(f.apply("CALCITE-1234: abc"),
+        hasItem("malformed [CALCITE-nnnn] reference"));
+    assertThat(f.apply("[CALCITE-12] abc"),
+        empty());
+    assertThat(f.apply("[CALCITE-123] abc"),
+        empty());
+    assertThat(f.apply("[CALCITE-1234] Fix problem with foo"),
+        hasItem("contains 'fix' or 'fixes'; you should describe the "
+            + "problem, not what you did"));
+    assertThat(f.apply("[CALCITE-1234] Baz doesn't buzz"),
+        empty());
+    assertThat(f.apply("[CALCITE-1234] Baz doesn't buzz."),
+        hasItem("ends with period"));
+    assertThat(f.apply("[CALCITE-1234]  Two problems."),
+        hasSize(2));
+    assertThat(f.apply("[CALCITE-1234]  Two problems."),
+        hasItem("ends with period"));
+    assertThat(f.apply("[CALCITE-1234]  Two problems."),
+        hasItem("malformed [CALCITE-nnnn] reference"));
+    assertThat(f.apply("Cosmetic: Move everything one character to the left"),
+        empty());
+    assertThat(
+        f.apply("Finishing up [CALCITE-4937], remove workarounds for "
+            + "[CALCITE-4877]"),
+        empty());
+    assertThat(f.apply("Fix typo in filterable-model.yaml"),
+        empty());
+    assertThat(f.apply("Fix typo in filterable-model.yaml"),
+        empty());
+    assertThat(f.apply("Revert \"[CALCITE-4817] Expand SubstitutionVisitor\""),
+        empty());
+  }
+
+  private static void checkMessage(String message, Consumer<String> consumer) {
+    if (message.startsWith(" ")) {
+      consumer.accept("starts with space");
+    }
+    if (message.endsWith(".")) {
+      consumer.accept("ends with period");
+    }
+    if (message.endsWith(" ")) {
+      consumer.accept("ends with space");
+    }
+    if (message.startsWith("[CALCITE-")
+        || message.startsWith("CALCITE-")) {
+      if (!message.matches("^\\[CALCITE-[0-9]{1,4}] [^ ].*")) {
+        consumer.accept("malformed [CALCITE-nnnn] reference");
+      }
+      if (message.matches("(?i).*\\b(fix|fixes)\\b.*")) {
+        consumer.accept("contains 'fix' or 'fixes'; you should describe the "
+            + "problem, not what you did");
+      }
+    }
+  }
+
+  /** Warning that code is not as it should be. */
+  private static class Message {
+    final Source source;
+    final int line;
+    final String message;
+
+    Message(Source source, int line, String message) {
+      this.source = source;
+      this.line = line;
+      this.message = message;
+    }
+
+    @Override public String toString() {
+      return source + ":" + line + ":" + message;
+    }
+  }
+
+  /** Internal state of the lint rules. */
+  private static class GlobalState {
+    int fileCount = 0;
+    final List<Message> messages = new ArrayList<>();
+  }
+
+  /** Internal state of the lint rules, per file. */
+  private static class FileState {
+    final GlobalState global;
+    int starLine;
+    int atLine;
+    int javadocStartLine;
+    int javadocEndLine;
+
+    FileState(GlobalState global) {
+      this.global = global;
+    }
+
+    void message(String message, Puffin.Line<GlobalState, FileState> line) {
+      global.messages.add(new Message(line.source(), line.fnr(), message));
+    }
+
+    public boolean inJavadoc() {
+      return javadocEndLine < javadocStartLine;
+    }
+  }
+}
diff --git a/core/src/test/java/org/apache/calcite/util/TestUnsafe.java b/core/src/test/java/org/apache/calcite/util/TestUnsafe.java
index e0d2f8f7ce..89e6395aac 100644
--- a/core/src/test/java/org/apache/calcite/util/TestUnsafe.java
+++ b/core/src/test/java/org/apache/calcite/util/TestUnsafe.java
@@ -16,16 +16,21 @@
  */
 package org.apache.calcite.util;
 
+import com.google.common.collect.ImmutableList;
+
 import org.checkerframework.checker.nullness.qual.Nullable;
 import org.slf4j.Logger;
 
 import java.io.BufferedInputStream;
 import java.io.BufferedOutputStream;
+import java.io.BufferedReader;
 import java.io.File;
 import java.io.IOException;
 import java.io.InputStream;
 import java.io.OutputStream;
 import java.io.Reader;
+import java.io.StringReader;
+import java.io.StringWriter;
 import java.io.Writer;
 import java.util.List;
 
@@ -110,4 +115,96 @@ public abstract class TestUnsafe {
     }
     return status;
   }
+
+  /** Returns whether we seem are in a valid environment. */
+  public static boolean haveGit() {
+    // Is there a '.git' directory? If not, we may be in a source tree
+    // unzipped from a tarball.
+    final File base = TestUtil.getBaseDir(TestUnsafe.class);
+    final File gitDir = new File(base, ".git");
+    if (!gitDir.exists()
+        || !gitDir.isDirectory()
+        || !gitDir.canRead()) {
+      return false;
+    }
+
+    // Execute a simple git command. If it fails, we're probably not in a
+    // valid git environment.
+    final List<String> argumentList =
+        ImmutableList.of("git", "--version");
+    try {
+      final StringWriter sw = new StringWriter();
+      int status =
+          runAppProcess(argumentList, base, null, null, sw);
+      final String s = sw.toString();
+      if (status != 0) {
+        return false;
+      }
+    } catch (Exception e) {
+      return false;
+    }
+    return true;
+  }
+
+  /** Returns a list of Java files in git under a given directory.
+   *
+   * <p>Assumes running Linux or macOS, and that git is available. */
+  public static List<File> getJavaFiles() {
+    String s;
+    try {
+      final List<String> argumentList =
+          ImmutableList.of("git", "ls-files", "*.java");
+      final File base = TestUtil.getBaseDir(TestUnsafe.class);
+      try {
+        final StringWriter sw = new StringWriter();
+        int status =
+            runAppProcess(argumentList, base, null, null, sw);
+        if (status != 0) {
+          throw new RuntimeException("command " + argumentList
+              + ": exited with status " + status);
+        }
+        s = sw.toString();
+      } catch (Exception e) {
+        throw new RuntimeException("command " + argumentList
+            + ": failed with exception", e);
+      }
+
+      final ImmutableList.Builder<File> files = ImmutableList.builder();
+      try (StringReader r = new StringReader(s);
+           BufferedReader br = new BufferedReader(r)) {
+        for (;;) {
+          String line = br.readLine();
+          if (line == null) {
+            break;
+          }
+          files.add(new File(base, line));
+        }
+      }
+      return files.build();
+    } catch (IOException e) {
+      throw new RuntimeException(e);
+    }
+  }
+
+  /** Returns the messages of the {@code n} most recent commits. */
+  public static List<String> getCommitMessages(int n) {
+    final File base = TestUtil.getBaseDir(TestUnsafe.class);
+    final List<String> argumentList =
+        ImmutableList.of("git", "log", "-n" + n, "--pretty=format:%s");
+    try {
+      final StringWriter sw = new StringWriter();
+      int status =
+          runAppProcess(argumentList, base, null, null, sw);
+      String s = sw.toString();
+      if (status != 0) {
+        throw new RuntimeException("command " + argumentList
+            + ": exited with status " + status
+            + (s.isEmpty() ? "" : "; output [" + s + "]"));
+      }
+      return ImmutableList.copyOf(s.split("\n"));
+    } catch (Exception e) {
+      throw new RuntimeException("command " + argumentList
+          + ": failed with exception", e);
+    }
+  }
 }
diff --git a/testkit/src/main/java/org/apache/calcite/util/TestUtil.java b/testkit/src/main/java/org/apache/calcite/util/TestUtil.java
index 68f753d6b1..a313fbd604 100644
--- a/testkit/src/main/java/org/apache/calcite/util/TestUtil.java
+++ b/testkit/src/main/java/org/apache/calcite/util/TestUtil.java
@@ -22,16 +22,21 @@ import com.google.common.collect.ImmutableSortedSet;
 
 import org.junit.jupiter.api.Assertions;
 
+import java.io.File;
 import java.io.PrintWriter;
 import java.io.StringWriter;
 import java.lang.reflect.InvocationTargetException;
+import java.net.URL;
 import java.util.List;
-import java.util.Objects;
 import java.util.SortedSet;
 import java.util.function.Supplier;
 import java.util.regex.Matcher;
 import java.util.regex.Pattern;
 
+import static org.junit.jupiter.api.Assertions.fail;
+
+import static java.util.Objects.requireNonNull;
+
 /**
  * Static utilities for JUnit tests.
  */
@@ -269,7 +274,7 @@ public abstract class TestUtil {
    */
   @VisibleForTesting
   static int majorVersionFromString(String version) {
-    Objects.requireNonNull(version, "version");
+    requireNonNull(version, "version");
 
     if (version.startsWith("1.")) {
       // running on version <= 8 (expecting string of type: x.y.z*)
@@ -310,6 +315,38 @@ public abstract class TestUtil {
     return System.getProperty("java.vm.vendor");
   }
 
+  /** Returns the root directory of the source tree. */
+  public static File getBaseDir(Class<?> klass) {
+    // Algorithm:
+    // 1) Find location of TestUtil.class
+    // 2) Climb via getParentFile() until we detect pom.xml
+    // 3) It means we've got BASE/testkit/pom.xml, and we need to get BASE
+    final URL resource = klass.getResource(klass.getSimpleName() + ".class");
+    final File classFile =
+        Sources.of(requireNonNull(resource, "resource")).file();
+
+    File file = classFile.getAbsoluteFile();
+    for (int i = 0; i < 42; i++) {
+      if (isProjectDir(file)) {
+        // Ok, file == BASE/testkit/
+        break;
+      }
+      file = file.getParentFile();
+    }
+    if (!isProjectDir(file)) {
+      fail("Could not find pom.xml, build.gradle.kts or gradle.properties. "
+          + "Started with " + classFile.getAbsolutePath()
+          + ", the current path is " + file.getAbsolutePath());
+    }
+    return file.getParentFile();
+  }
+
+  private static boolean isProjectDir(File dir) {
+    return new File(dir, "pom.xml").isFile()
+        || new File(dir, "build.gradle.kts").isFile()
+        || new File(dir, "gradle.properties").isFile();
+  }
+
   /** Given a list, returns the number of elements that are not between an
    * element that is less and an element that is greater. */
   public static <E extends Comparable<E>> SortedSet<E> outOfOrderItems(List<E> list) {