You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@calcite.apache.org by la...@apache.org on 2020/05/12 18:34:04 UTC

[calcite] 01/01: [CALCITE-3517] Rework DiffRepository as a JUnit extension

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

laurent pushed a commit to branch laurentgo/CALCITE-3517
in repository https://gitbox.apache.org/repos/asf/calcite.git

commit 244c11a25b26c03e974c07cad9d43d9df4585f35
Author: Laurent Goujon <la...@apache.org>
AuthorDate: Mon May 4 15:28:15 2020 -0700

    [CALCITE-3517] Rework DiffRepository as a JUnit extension
    
    Introduce JUnit DiffRepositoryExtension extension to manage concurrent
    access to DiffRepository instances by test classes, and dumping their
    content on disk at the end of the test run.
    
    DiffRepositoryExtension automatically injects DiffRepository instance to
    the constructor based on the test class, but a different repository can
    be used by using the DiffRepositoryName annotation.
    
    Update all the tests using DiffRepository to leverage the extension.
    
    Cleanup DiffRepository class, and remove all unused code.
---
 .../calcite/sql/test/SqlPrettyWriterTest.java      |  13 +-
 .../org/apache/calcite/test/DiffRepository.java    | 193 +--------------------
 .../calcite/test/DiffRepositoryExtension.java      | 112 ++++++++++++
 .../apache/calcite/test/DiffRepositoryName.java    |  32 ++++
 .../org/apache/calcite/test/HepPlannerTest.java    |   7 +-
 .../org/apache/calcite/test/RelMetadataTest.java   |   2 +-
 .../org/apache/calcite/test/RelOptRulesTest.java   |  10 +-
 .../org/apache/calcite/test/RelOptTestBase.java    |   3 +
 .../apache/calcite/test/SqlHintsConverterTest.java |  13 +-
 .../org/apache/calcite/test/SqlLimitsTest.java     |  10 +-
 .../test/SqlToRelConverterExtendedTest.java        |   5 +
 .../apache/calcite/test/SqlToRelConverterTest.java |   8 +-
 .../org/apache/calcite/test/SqlToRelTestBase.java  | 132 +++++++-------
 .../calcite/test/TypeCoercionConverterTest.java    |   4 +-
 14 files changed, 270 insertions(+), 274 deletions(-)

diff --git a/core/src/test/java/org/apache/calcite/sql/test/SqlPrettyWriterTest.java b/core/src/test/java/org/apache/calcite/sql/test/SqlPrettyWriterTest.java
index 51b1c04..e173d6c 100644
--- a/core/src/test/java/org/apache/calcite/sql/test/SqlPrettyWriterTest.java
+++ b/core/src/test/java/org/apache/calcite/sql/test/SqlPrettyWriterTest.java
@@ -25,10 +25,12 @@ import org.apache.calcite.sql.parser.SqlParseException;
 import org.apache.calcite.sql.parser.SqlParser;
 import org.apache.calcite.sql.pretty.SqlPrettyWriter;
 import org.apache.calcite.test.DiffRepository;
+import org.apache.calcite.test.DiffRepositoryExtension;
 import org.apache.calcite.util.Litmus;
 
 import org.junit.jupiter.api.Disabled;
 import org.junit.jupiter.api.Test;
+import org.junit.jupiter.api.extension.ExtendWith;
 
 import java.io.PrintWriter;
 import java.io.StringWriter;
@@ -42,9 +44,12 @@ import static org.junit.jupiter.api.Assertions.assertTrue;
  *
  * <p>You must provide the system property "source.dir".
  */
+@ExtendWith(DiffRepositoryExtension.class)
 class SqlPrettyWriterTest {
-  protected DiffRepository getDiffRepos() {
-    return DiffRepository.lookup(SqlPrettyWriterTest.class);
+  private final DiffRepository repository;
+
+  SqlPrettyWriterTest(DiffRepository repository) {
+    this.repository = repository;
   }
 
   /**
@@ -125,12 +130,12 @@ class SqlPrettyWriterTest {
         prettyWriter.describe(pw, true);
         pw.flush();
         final String desc = sw.toString();
-        getDiffRepos().assertEquals("desc", this.desc, desc);
+        repository.assertEquals("desc", this.desc, desc);
       }
 
       // Format
       final String formatted = prettyWriter.format(node);
-      getDiffRepos().assertEquals("formatted", this.formatted, formatted);
+      repository.assertEquals("formatted", this.formatted, formatted);
 
       // Now parse the result, and make sure it is structurally equivalent
       // to the original.
diff --git a/core/src/test/java/org/apache/calcite/test/DiffRepository.java b/core/src/test/java/org/apache/calcite/test/DiffRepository.java
index 875e2dc..02d5703 100644
--- a/core/src/test/java/org/apache/calcite/test/DiffRepository.java
+++ b/core/src/test/java/org/apache/calcite/test/DiffRepository.java
@@ -18,14 +18,9 @@ package org.apache.calcite.test;
 
 import org.apache.calcite.avatica.util.Spaces;
 import org.apache.calcite.util.Pair;
-import org.apache.calcite.util.Sources;
 import org.apache.calcite.util.Util;
 import org.apache.calcite.util.XmlOutput;
 
-import com.google.common.cache.CacheBuilder;
-import com.google.common.cache.CacheLoader;
-import com.google.common.cache.LoadingCache;
-
 import org.junit.jupiter.api.Assertions;
 import org.opentest4j.AssertionFailedError;
 import org.w3c.dom.CDATASection;
@@ -45,7 +40,6 @@ import java.net.URL;
 import java.util.ArrayList;
 import java.util.HashSet;
 import java.util.List;
-import java.util.Objects;
 import java.util.Set;
 import javax.xml.parsers.DocumentBuilder;
 import javax.xml.parsers.DocumentBuilderFactory;
@@ -155,46 +149,23 @@ public class DiffRepository {
   private static final String ROOT_TAG = "Root";
   private static final String TEST_CASE_TAG = "TestCase";
   private static final String TEST_CASE_NAME_ATTR = "name";
-  private static final String TEST_CASE_OVERRIDES_ATTR = "overrides";
   private static final String RESOURCE_TAG = "Resource";
   private static final String RESOURCE_NAME_ATTR = "name";
 
-  /**
-   * Holds one diff-repository per class. It is necessary for all test cases in
-   * the same class to share the same diff-repository: if the repository gets
-   * loaded once per test case, then only one diff is recorded.
-   */
-  private static final LoadingCache<Key, DiffRepository> REPOSITORY_CACHE =
-      CacheBuilder.newBuilder().build(CacheLoader.from(Key::toRepo));
-
   //~ Instance fields --------------------------------------------------------
 
-  private final DiffRepository baseRepository;
-  private final int indent;
   private Document doc;
   private final Element root;
-  private final File logFile;
-  private final Filter filter;
 
   /**
    * Creates a DiffRepository.
    *
    * @param refFile   Reference file
-   * @param logFile   Log file
-   * @param baseRepository Parent repository or null
-   * @param filter    Filter or null
    */
-  private DiffRepository(
-      URL refFile,
-      File logFile,
-      DiffRepository baseRepository,
-      Filter filter) {
-    this.baseRepository = baseRepository;
-    this.filter = filter;
+  DiffRepository(URL refFile) {
     if (refFile == null) {
       throw new IllegalArgumentException("url must not be null");
     }
-    this.logFile = logFile;
 
     // Load the document.
     DocumentBuilderFactory fac = DocumentBuilderFactory.newInstance();
@@ -210,27 +181,16 @@ public class DiffRepository {
         this.doc = docBuilder.newDocument();
         this.doc.appendChild(
             doc.createElement(ROOT_TAG));
-        flushDoc();
       }
       this.root = doc.getDocumentElement();
       validate(this.root);
     } catch (ParserConfigurationException | SAXException e) {
       throw new RuntimeException("error while creating xml parser", e);
     }
-    indent = logFile.getPath().contains("RelOptRulesTest")
-        || logFile.getPath().contains("SqlToRelConverterTest")
-        || logFile.getPath().contains("SqlLimitsTest") ? 4 : 2;
   }
 
   //~ Methods ----------------------------------------------------------------
 
-  private static URL findFile(Class<?> clazz, final String suffix) {
-    // The reference file for class "com.foo.Bar" is "com/foo/Bar.xml"
-    String rest = "/" + clazz.getName().replace('.', File.separatorChar)
-        + suffix;
-    return clazz.getResource(rest);
-  }
-
   /**
    * Expands a string containing one or more variables. (Currently only works
    * if there is one variable.)
@@ -254,19 +214,9 @@ public class DiffRepository {
         // log file.
         return text;
       }
-      if (filter != null) {
-        expanded =
-            filter.filter(this, testCaseName, tag, text, expanded);
-      }
       return expanded;
     } else {
-      // Make sure what appears in the resource file is consistent with
-      // what is in the Java. It helps to have a redundant copy in the
-      // resource file.
-      final String testCaseName = getCurrentTestCaseName(true);
-      if (baseRepository == null || baseRepository.get(testCaseName, tag) == null) {
-        set(tag, text);
-      }
+      set(tag, text);
       return text;
     }
   }
@@ -301,13 +251,9 @@ public class DiffRepository {
   private synchronized String get(
       final String testCaseName,
       String resourceName) {
-    Element testCaseElement = getTestCaseElement(testCaseName, true, null);
+    Element testCaseElement = getTestCaseElement(testCaseName, null);
     if (testCaseElement == null) {
-      if (baseRepository != null) {
-        return baseRepository.get(testCaseName, resourceName);
-      } else {
-        return null;
-      }
+      return null;
     }
     final Element resourceElement =
         getResourceElement(testCaseElement, resourceName);
@@ -354,7 +300,6 @@ public class DiffRepository {
    */
   private synchronized Element getTestCaseElement(
       final String testCaseName,
-      boolean checkOverride,
       List<Pair<String, Element>> elements) {
     final NodeList childNodes = root.getChildNodes();
     for (int i = 0; i < childNodes.getLength(); i++) {
@@ -363,16 +308,6 @@ public class DiffRepository {
         Element testCase = (Element) child;
         final String name = testCase.getAttribute(TEST_CASE_NAME_ATTR);
         if (testCaseName.equals(name)) {
-          if (checkOverride
-              && (baseRepository != null)
-              && (baseRepository.getTestCaseElement(testCaseName, false, null) != null)
-              && !"true".equals(
-                  testCase.getAttribute(TEST_CASE_OVERRIDES_ATTR))) {
-            throw new RuntimeException(
-                "TestCase  '" + testCaseName + "' overrides a "
-                + "test case in the base repository, but does "
-                + "not specify 'overrides=true'");
-          }
           return testCase;
         }
         if (elements != null) {
@@ -457,7 +392,7 @@ public class DiffRepository {
       String resourceName,
       String value) {
     final List<Pair<String, Element>> map = new ArrayList<>();
-    Element testCaseElement = getTestCaseElement(testCaseName, true, map);
+    Element testCaseElement = getTestCaseElement(testCaseName, map);
     if (testCaseElement == null) {
       testCaseElement = doc.createElement(TEST_CASE_TAG);
       testCaseElement.setAttribute(TEST_CASE_NAME_ATTR, testCaseName);
@@ -476,9 +411,6 @@ public class DiffRepository {
     if (!value.equals("")) {
       resourceElement.appendChild(doc.createCDATASection(value));
     }
-
-    // Write out the document.
-    flushDoc();
   }
 
   private Node ref(String testCaseName, List<Pair<String, Element>> map) {
@@ -517,7 +449,7 @@ public class DiffRepository {
   /**
    * Flushes the reference document to the file system.
    */
-  private void flushDoc() {
+  public void writeToFile(File logFile, int indent) {
     try {
       boolean b = logFile.getParentFile().mkdirs();
       Util.discard(b);
@@ -707,117 +639,4 @@ public class DiffRepository {
     }
     return true;
   }
-
-  /**
-   * Finds the repository instance for a given class, with no base
-   * repository or filter.
-   *
-   * @param clazz Test case class
-   * @return The diff repository shared between test cases in this class.
-   */
-  public static DiffRepository lookup(Class<?> clazz) {
-    return lookup(clazz, null);
-  }
-
-  /**
-   * Finds the repository instance for a given class and inheriting from
-   * a given repository.
-   *
-   * @param clazz     Test case class
-   * @param baseRepository Base class of test class
-   * @return The diff repository shared between test cases in this class.
-   */
-  public static DiffRepository lookup(
-      Class<?> clazz,
-      DiffRepository baseRepository) {
-    return lookup(clazz, baseRepository, null);
-  }
-
-  /**
-   * Finds the repository instance for a given class.
-   *
-   * <p>It is important that all test cases in a class share the same
-   * repository instance. This ensures that, if two or more test cases fail,
-   * the log file will contains the actual results of both test cases.
-   *
-   * <p>The <code>baseRepository</code> parameter is useful if the test is an
-   * extension to a previous test. If the test class has a base class which
-   * also has a repository, specify the repository here. DiffRepository will
-   * look for resources in the base class if it cannot find them in this
-   * repository. If test resources from test cases in the base class are
-   * missing or incorrect, it will not write them to the log file -- you
-   * probably need to fix the base test.
-   *
-   * <p>Use the <code>filter</code> parameter if you expect the test to
-   * return results slightly different than in the repository. This happens
-   * if the behavior of a derived test is slightly different than a base
-   * test. If you do not specify a filter, no filtering will happen.
-   *
-   * @param clazz     Test case class
-   * @param baseRepository Base repository
-   * @param filter    Filters each string returned by the repository
-   * @return The diff repository shared between test cases in this class.
-   */
-  public static DiffRepository lookup(Class<?> clazz,
-      DiffRepository baseRepository,
-      Filter filter) {
-    final Key key = new Key(clazz, baseRepository, filter);
-    return REPOSITORY_CACHE.getUnchecked(key);
-  }
-
-  /**
-   * Callback to filter strings before returning them.
-   */
-  public interface Filter {
-    /**
-     * Filters a string.
-     *
-     * @param diffRepository Repository
-     * @param testCaseName   Test case name
-     * @param tag            Tag being expanded
-     * @param text           Text being expanded
-     * @param expanded       Expanded text
-     * @return Expanded text after filtering
-     */
-    String filter(
-        DiffRepository diffRepository,
-        String testCaseName,
-        String tag,
-        String text,
-        String expanded);
-  }
-
-  /** Cache key. */
-  private static class Key {
-    private final Class<?> clazz;
-    private final DiffRepository baseRepository;
-    private final Filter filter;
-
-    Key(Class<?> clazz, DiffRepository baseRepository, Filter filter) {
-      this.clazz = Objects.requireNonNull(clazz);
-      this.baseRepository = baseRepository;
-      this.filter = filter;
-    }
-
-    @Override public int hashCode() {
-      return Objects.hash(clazz, baseRepository, filter);
-    }
-
-    @Override public boolean equals(Object obj) {
-      return this == obj
-          || obj instanceof Key
-          && clazz.equals(((Key) obj).clazz)
-          && Objects.equals(baseRepository, ((Key) obj).baseRepository)
-          && Objects.equals(filter, ((Key) obj).filter);
-    }
-
-    DiffRepository toRepo() {
-      final URL refFile = findFile(clazz, ".xml");
-      final String refFilePath = Sources.of(refFile).file().getAbsolutePath();
-      final String logFilePath = refFilePath.replace(".xml", "_actual.xml");
-      final File logFile = new File(logFilePath);
-      assert !refFilePath.equals(logFile.getAbsolutePath());
-      return new DiffRepository(refFile, logFile, baseRepository, filter);
-    }
-  }
 }
diff --git a/core/src/test/java/org/apache/calcite/test/DiffRepositoryExtension.java b/core/src/test/java/org/apache/calcite/test/DiffRepositoryExtension.java
new file mode 100644
index 0000000..0c2f3c5
--- /dev/null
+++ b/core/src/test/java/org/apache/calcite/test/DiffRepositoryExtension.java
@@ -0,0 +1,112 @@
+/*
+ * 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.Sources;
+
+import org.junit.jupiter.api.extension.AfterAllCallback;
+import org.junit.jupiter.api.extension.BeforeAllCallback;
+import org.junit.jupiter.api.extension.ExtensionContext;
+import org.junit.jupiter.api.extension.ExtensionContext.Namespace;
+import org.junit.jupiter.api.extension.ExtensionContext.Store;
+import org.junit.jupiter.api.extension.ParameterContext;
+import org.junit.jupiter.api.extension.ParameterResolutionException;
+import org.junit.jupiter.api.extension.support.TypeBasedParameterResolver;
+
+import java.io.File;
+import java.net.URL;
+import java.util.Map;
+import java.util.concurrent.ConcurrentHashMap;
+
+/**
+ * A JUnit extension to manage access to {@code DiffRepository}
+ *
+ * The extension injects a {@code DiffRepository} instance to constructor or test method parameter.
+ * By default the repository is based on the test class name, but a different repository can be
+ * used thanks to the {@code DiffRepositoryName} annotation.
+ *
+ * At the end of the test run, all instances are dumped on disk.
+ */
+public class DiffRepositoryExtension extends TypeBasedParameterResolver<DiffRepository>
+    implements BeforeAllCallback, AfterAllCallback {
+  private static final String DIFF_REPOSITORIES = "diff repositories";
+
+  public DiffRepositoryExtension() {
+  }
+
+  @Override public DiffRepository resolveParameter(ParameterContext parameterContext,
+      ExtensionContext extensionContext) throws ParameterResolutionException {
+    // Get the enclosing class
+    final Class<?> clazz = parameterContext.findAnnotation(DiffRepositoryName.class)
+        .<Class<?>>map(DiffRepositoryName::value)
+        .orElseGet(() -> getDefaultClass(parameterContext));
+
+    @SuppressWarnings("unchecked")
+    final Map<Class<?>, DiffRepository> repositories = (Map<Class<?>, DiffRepository>) getStore(
+        extensionContext).get(DIFF_REPOSITORIES);
+    DiffRepository repository = repositories.computeIfAbsent(clazz, this::toRepo);
+    return repository;
+  }
+
+  private Class<?> getDefaultClass(ParameterContext parameterContext) {
+    return parameterContext.getTarget().<Class<?>>map(Object::getClass)
+        .orElse(parameterContext.getDeclaringExecutable().getDeclaringClass());
+  }
+
+  @Override public void beforeAll(ExtensionContext context) throws Exception {
+    final Store store = getStore(context);
+    store.put(DIFF_REPOSITORIES, new ConcurrentHashMap<String, DiffRepository>());
+  }
+
+  @Override public void afterAll(ExtensionContext context) throws Exception {
+    @SuppressWarnings("unchecked")
+    Map<Class<?>, DiffRepository> repositories = (Map<Class<?>, DiffRepository>) getStore(context)
+        .get(DIFF_REPOSITORIES);
+    for (Map.Entry<Class<?>, DiffRepository> entry : repositories.entrySet()) {
+      final Class<?> clazz = entry.getKey();
+      final DiffRepository repository = entry.getValue();
+
+      final URL refFile = findFile(clazz, ".xml");
+      final String refFilePath = Sources.of(refFile).file().getAbsolutePath();
+      final String logFilePath = refFilePath.replace(".xml", "_actual.xml");
+      final File logFile = new File(logFilePath);
+      assert !refFilePath.equals(logFile.getAbsolutePath());
+
+      int indent = logFile.getPath().contains("RelOptRulesTest")
+          || logFile.getPath().contains("SqlToRelConverterTest")
+          || logFile.getPath().contains("SqlLimitsTest") ? 4 : 2;
+
+      repository.writeToFile(logFile, indent);
+    }
+  }
+
+  private Store getStore(ExtensionContext context) {
+    return context.getStore(Namespace.create(getClass(), context.getRoot()));
+  }
+
+  private static URL findFile(Class<?> clazz, final String suffix) {
+    // The reference file for class "com.foo.Bar" is "com/foo/Bar.xml"
+    String rest = "/" + clazz.getName().replace('.', File.separatorChar) + suffix;
+    return clazz.getResource(rest);
+  }
+
+  private DiffRepository toRepo(Class<?> clazz) {
+    final URL refFile = findFile(clazz, ".xml");
+
+    return new DiffRepository(refFile);
+  }
+}
diff --git a/core/src/test/java/org/apache/calcite/test/DiffRepositoryName.java b/core/src/test/java/org/apache/calcite/test/DiffRepositoryName.java
new file mode 100644
index 0000000..7874d97
--- /dev/null
+++ b/core/src/test/java/org/apache/calcite/test/DiffRepositoryName.java
@@ -0,0 +1,32 @@
+/*
+ * 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 java.lang.annotation.ElementType;
+import java.lang.annotation.Retention;
+import java.lang.annotation.RetentionPolicy;
+import java.lang.annotation.Target;
+
+/**
+ * Annotation to specify a different {@code DiffRepository} name to be injected
+ * by {@code DiffRepositoryExtension}
+ */
+@Retention(RetentionPolicy.RUNTIME)
+@Target(ElementType.PARAMETER)
+public @interface DiffRepositoryName {
+  Class<?> value();
+}
diff --git a/core/src/test/java/org/apache/calcite/test/HepPlannerTest.java b/core/src/test/java/org/apache/calcite/test/HepPlannerTest.java
index 5b5fcc2..444ac27 100644
--- a/core/src/test/java/org/apache/calcite/test/HepPlannerTest.java
+++ b/core/src/test/java/org/apache/calcite/test/HepPlannerTest.java
@@ -38,6 +38,7 @@ import org.apache.calcite.rel.rules.UnionToDistinctRule;
 import com.google.common.collect.ImmutableList;
 
 import org.junit.jupiter.api.Test;
+import org.junit.jupiter.api.extension.ExtendWith;
 
 import static org.hamcrest.CoreMatchers.is;
 import static org.hamcrest.MatcherAssert.assertThat;
@@ -50,6 +51,7 @@ import static org.junit.jupiter.api.Assertions.assertEquals;
  * convenience only, whereas the tests in that class are targeted at exercising
  * specific rules, and use the planner for convenience only. Hence the split.
  */
+@ExtendWith(DiffRepositoryExtension.class)
 class HepPlannerTest extends RelOptTestBase {
   //~ Static fields/initializers ---------------------------------------------
 
@@ -82,9 +84,8 @@ class HepPlannerTest extends RelOptTestBase {
       + ") a";
 
   //~ Methods ----------------------------------------------------------------
-
-  protected DiffRepository getDiffRepos() {
-    return DiffRepository.lookup(HepPlannerTest.class);
+  HepPlannerTest(DiffRepository repository) {
+    super(repository);
   }
 
   @Test void testRuleClass() throws Exception {
diff --git a/core/src/test/java/org/apache/calcite/test/RelMetadataTest.java b/core/src/test/java/org/apache/calcite/test/RelMetadataTest.java
index fcc54aa..ec65ccc 100644
--- a/core/src/test/java/org/apache/calcite/test/RelMetadataTest.java
+++ b/core/src/test/java/org/apache/calcite/test/RelMetadataTest.java
@@ -3098,7 +3098,7 @@ public class RelMetadataTest extends SqlToRelTestBase {
       implements MetadataHandler<ColType> {
     static final ThreadLocal<List<String>> THREAD_LIST = new ThreadLocal<>();
 
-    public MetadataDef<ColType> getDef() {
+    @Override public MetadataDef<ColType> getDef() {
       return ColType.DEF;
     }
 
diff --git a/core/src/test/java/org/apache/calcite/test/RelOptRulesTest.java b/core/src/test/java/org/apache/calcite/test/RelOptRulesTest.java
index 4ff8902..2299a8e 100644
--- a/core/src/test/java/org/apache/calcite/test/RelOptRulesTest.java
+++ b/core/src/test/java/org/apache/calcite/test/RelOptRulesTest.java
@@ -228,8 +228,8 @@ class RelOptRulesTest extends RelOptTestBase {
       expr instanceof RexCall
           && "item".equalsIgnoreCase(((RexCall) expr).getOperator().getName());
 
-  protected DiffRepository getDiffRepos() {
-    return DiffRepository.lookup(RelOptRulesTest.class);
+  RelOptRulesTest(DiffRepository repository) {
+    super(repository);
   }
 
   @Test void testReduceNot() {
@@ -3924,7 +3924,7 @@ class RelOptRulesTest extends RelOptTestBase {
                     new Predicate<Project>() {
                       int matchCount = 0;
 
-                      public boolean test(Project project) {
+                      @Override public boolean test(Project project) {
                         return matchCount++ == 0;
                       }
                     },
@@ -6616,7 +6616,7 @@ class RelOptRulesTest extends RelOptTestBase {
       super(cluster, traitSet, child, condition);
     }
 
-    public MyFilter copy(RelTraitSet traitSet, RelNode input,
+    @Override public MyFilter copy(RelTraitSet traitSet, RelNode input,
         RexNode condition) {
       return new MyFilter(getCluster(), traitSet, input, condition);
     }
@@ -6661,7 +6661,7 @@ class RelOptRulesTest extends RelOptTestBase {
       super(cluster, traitSet, ImmutableList.of(), input, projects, rowType);
     }
 
-    public MyProject copy(RelTraitSet traitSet, RelNode input,
+    @Override public MyProject copy(RelTraitSet traitSet, RelNode input,
         List<RexNode> projects, RelDataType rowType) {
       return new MyProject(getCluster(), traitSet, input, projects, rowType);
     }
diff --git a/core/src/test/java/org/apache/calcite/test/RelOptTestBase.java b/core/src/test/java/org/apache/calcite/test/RelOptTestBase.java
index 6f3ee5c..15938b2 100644
--- a/core/src/test/java/org/apache/calcite/test/RelOptTestBase.java
+++ b/core/src/test/java/org/apache/calcite/test/RelOptTestBase.java
@@ -59,6 +59,9 @@ import static org.junit.jupiter.api.Assertions.assertNotNull;
  */
 abstract class RelOptTestBase extends SqlToRelTestBase {
   //~ Methods ----------------------------------------------------------------
+  protected RelOptTestBase(DiffRepository repository) {
+    super(repository);
+  }
 
   @Override protected Tester createTester() {
     return super.createTester().withDecorrelation(false);
diff --git a/core/src/test/java/org/apache/calcite/test/SqlHintsConverterTest.java b/core/src/test/java/org/apache/calcite/test/SqlHintsConverterTest.java
index adc40b2..86fda7c 100644
--- a/core/src/test/java/org/apache/calcite/test/SqlHintsConverterTest.java
+++ b/core/src/test/java/org/apache/calcite/test/SqlHintsConverterTest.java
@@ -96,10 +96,11 @@ import static org.junit.jupiter.api.Assertions.fail;
 /**
  * Unit test for {@link org.apache.calcite.rel.hint.RelHint}.
  */
+
 class SqlHintsConverterTest extends SqlToRelTestBase {
 
-  protected DiffRepository getDiffRepos() {
-    return DiffRepository.lookup(SqlHintsConverterTest.class);
+  SqlHintsConverterTest(DiffRepository repository) {
+    super(repository);
   }
 
   //~ Tests ------------------------------------------------------------------
@@ -503,7 +504,7 @@ class SqlHintsConverterTest extends SqlToRelTestBase {
       super(operand(LogicalJoin.class, any()), "MockJoinRule");
     }
 
-    public void onMatch(RelOptRuleCall call) {
+    @Override public void onMatch(RelOptRuleCall call) {
       LogicalJoin join = call.rel(0);
       assertThat(1, is(join.getHints().size()));
       call.transformTo(
@@ -695,15 +696,15 @@ class SqlHintsConverterTest extends SqlToRelTestBase {
   private static class MockAppender extends AppenderSkeleton {
     public final List<LoggingEvent> loggingEvents = new ArrayList<>();
 
-    protected void append(org.apache.log4j.spi.LoggingEvent event) {
+    @Override protected void append(org.apache.log4j.spi.LoggingEvent event) {
       loggingEvents.add(event);
     }
 
-    public void close() {
+    @Override public void close() {
       // no-op
     }
 
-    public boolean requiresLayout() {
+    @Override public boolean requiresLayout() {
       return false;
     }
   }
diff --git a/core/src/test/java/org/apache/calcite/test/SqlLimitsTest.java b/core/src/test/java/org/apache/calcite/test/SqlLimitsTest.java
index f5562f6..d1ded3b 100644
--- a/core/src/test/java/org/apache/calcite/test/SqlLimitsTest.java
+++ b/core/src/test/java/org/apache/calcite/test/SqlLimitsTest.java
@@ -31,6 +31,7 @@ import org.apache.calcite.testlib.annotations.LocaleEnUs;
 import com.google.common.collect.ImmutableList;
 
 import org.junit.jupiter.api.Test;
+import org.junit.jupiter.api.extension.ExtendWith;
 
 import java.io.PrintWriter;
 import java.io.StringWriter;
@@ -43,10 +44,13 @@ import java.util.Locale;
 /**
  * Unit test for SQL limits.
  */
+@ExtendWith(DiffRepositoryExtension.class)
 @LocaleEnUs
 public class SqlLimitsTest {
-  protected DiffRepository getDiffRepos() {
-    return DiffRepository.lookup(SqlLimitsTest.class);
+  private final DiffRepository repository;
+
+  public SqlLimitsTest(DiffRepository repository) {
+    this.repository = repository;
   }
 
   /** Returns a list of typical types. */
@@ -151,7 +155,7 @@ public class SqlLimitsTest {
       pw.println();
     }
     pw.flush();
-    getDiffRepos().assertEquals("output", "${output}", sw.toString());
+    repository.assertEquals("output", "${output}", sw.toString());
   }
 
   private void printLimit(
diff --git a/core/src/test/java/org/apache/calcite/test/SqlToRelConverterExtendedTest.java b/core/src/test/java/org/apache/calcite/test/SqlToRelConverterExtendedTest.java
index c18608a..0e8f700 100644
--- a/core/src/test/java/org/apache/calcite/test/SqlToRelConverterExtendedTest.java
+++ b/core/src/test/java/org/apache/calcite/test/SqlToRelConverterExtendedTest.java
@@ -49,6 +49,11 @@ class SqlToRelConverterExtendedTest extends SqlToRelConverterTest {
     }
   }
 
+  SqlToRelConverterExtendedTest(
+      @DiffRepositoryName(SqlToRelConverterTest.class) DiffRepository repository) {
+    super(repository);
+  }
+
   public static void foo(RelNode rel) {
     // Convert rel tree to JSON.
     final RelJsonWriter writer = new RelJsonWriter();
diff --git a/core/src/test/java/org/apache/calcite/test/SqlToRelConverterTest.java b/core/src/test/java/org/apache/calcite/test/SqlToRelConverterTest.java
index 3144d1e..4ea8005 100644
--- a/core/src/test/java/org/apache/calcite/test/SqlToRelConverterTest.java
+++ b/core/src/test/java/org/apache/calcite/test/SqlToRelConverterTest.java
@@ -78,8 +78,8 @@ import static org.junit.jupiter.api.Assertions.assertTrue;
  * Unit test for {@link org.apache.calcite.sql2rel.SqlToRelConverter}.
  */
 class SqlToRelConverterTest extends SqlToRelTestBase {
-  protected DiffRepository getDiffRepos() {
-    return DiffRepository.lookup(SqlToRelConverterTest.class);
+  SqlToRelConverterTest(DiffRepository repository) {
+    super(repository);
   }
 
   /** Sets the SQL statement for a test. */
@@ -3843,7 +3843,7 @@ class SqlToRelConverterTest extends SqlToRelTestBase {
     int invalidCount;
     final Deque<RelNode> stack = new ArrayDeque<>();
 
-    public Set<CorrelationId> correlationIds() {
+    @Override public Set<CorrelationId> correlationIds() {
       final ImmutableSet.Builder<CorrelationId> builder =
           ImmutableSet.builder();
       for (RelNode r : stack) {
@@ -3852,7 +3852,7 @@ class SqlToRelConverterTest extends SqlToRelTestBase {
       return builder.build();
     }
 
-    public void visit(RelNode node, int ordinal, RelNode parent) {
+    @Override public void visit(RelNode node, int ordinal, RelNode parent) {
       try {
         stack.push(node);
         if (!node.isValid(Litmus.THROW, this)) {
diff --git a/core/src/test/java/org/apache/calcite/test/SqlToRelTestBase.java b/core/src/test/java/org/apache/calcite/test/SqlToRelTestBase.java
index e7b5028..0c7c406 100644
--- a/core/src/test/java/org/apache/calcite/test/SqlToRelTestBase.java
+++ b/core/src/test/java/org/apache/calcite/test/SqlToRelTestBase.java
@@ -75,6 +75,8 @@ import org.apache.calcite.util.TestUtil;
 import com.google.common.collect.ImmutableList;
 import com.google.common.collect.Iterables;
 
+import org.junit.jupiter.api.extension.ExtendWith;
+
 import java.util.ArrayList;
 import java.util.List;
 import java.util.Objects;
@@ -93,16 +95,28 @@ import static org.junit.jupiter.api.Assertions.assertNotNull;
  * SQL, look down in the stack until you see "Caused by", which will usually
  * tell you the real error.
  */
+@ExtendWith(DiffRepositoryExtension.class)
 public abstract class SqlToRelTestBase {
   //~ Static fields/initializers ---------------------------------------------
 
   protected static final String NL = System.getProperty("line.separator");
 
   //~ Instance fields --------------------------------------------------------
+  private final DiffRepository repository;
 
-  protected final Tester tester = createTester();
+  protected final Tester tester;
   // Same as tester but without implicit type coercion.
-  protected final Tester strictTester = tester.enableTypeCoercion(false);
+  protected final Tester strictTester;
+
+  protected SqlToRelTestBase() {
+    this(null);
+  }
+
+  protected SqlToRelTestBase(DiffRepository repository) {
+    this.repository = repository;
+    tester = createTester();
+    strictTester = tester.enableTypeCoercion(false);
+  }
 
   protected Tester createTester() {
     return new TesterImpl(getDiffRepos(), false, false, true, false,
@@ -134,8 +148,8 @@ public abstract class SqlToRelTestBase {
    *
    * @return Diff repository
    */
-  protected DiffRepository getDiffRepos() {
-    return null;
+  protected final DiffRepository getDiffRepos() {
+    return repository;
   }
 
   /**
@@ -284,7 +298,7 @@ public abstract class SqlToRelTestBase {
       this.typeFactory = typeFactory;
     }
 
-    public RelOptTable getTableForMember(List<String> names) {
+    @Override public RelOptTable getTableForMember(List<String> names) {
       final SqlValidatorTable table =
           catalogReader.getTable(names);
       final RelDataType rowType = table.getRowType();
@@ -324,7 +338,7 @@ public abstract class SqlToRelTestBase {
       return collationList;
     }
 
-    public RelOptTable getTableForMember(
+    @Override public RelOptTable getTableForMember(
         List<String> names,
         final String datasetName,
         boolean[] usedDataset) {
@@ -334,7 +348,7 @@ public abstract class SqlToRelTestBase {
       // assume there's a table called "<table>:<sample>".
       RelOptTable datasetTable =
           new DelegatingRelOptTable(table) {
-            public List<String> getQualifiedName() {
+            @Override public List<String> getQualifiedName() {
               final List<String> list =
                   new ArrayList<>(super.getQualifiedName());
               list.set(
@@ -358,11 +372,11 @@ public abstract class SqlToRelTestBase {
       return new MockColumnSet(names, rowType, collationList);
     }
 
-    public RelDataTypeFactory getTypeFactory() {
+    @Override public RelDataTypeFactory getTypeFactory() {
       return typeFactory;
     }
 
-    public void registerRules(RelOptPlanner planner) throws Exception {
+    @Override public void registerRules(RelOptPlanner planner) throws Exception {
     }
 
     /** Mock column set. */
@@ -380,18 +394,18 @@ public abstract class SqlToRelTestBase {
         this.collationList = collationList;
       }
 
-      public <T> T unwrap(Class<T> clazz) {
+      @Override public <T> T unwrap(Class<T> clazz) {
         if (clazz.isInstance(this)) {
           return clazz.cast(this);
         }
         return null;
       }
 
-      public List<String> getQualifiedName() {
+      @Override public List<String> getQualifiedName() {
         return names;
       }
 
-      public double getRowCount() {
+      @Override public double getRowCount() {
         // use something other than 0 to give costing tests
         // some room, and make emps bigger than depts for
         // join asymmetry
@@ -402,48 +416,48 @@ public abstract class SqlToRelTestBase {
         }
       }
 
-      public RelDataType getRowType() {
+      @Override public RelDataType getRowType() {
         return rowType;
       }
 
-      public RelOptSchema getRelOptSchema() {
+      @Override public RelOptSchema getRelOptSchema() {
         return MockRelOptSchema.this;
       }
 
-      public RelNode toRel(ToRelContext context) {
+      @Override public RelNode toRel(ToRelContext context) {
         return LogicalTableScan.create(context.getCluster(), this,
             context.getTableHints());
       }
 
-      public List<RelCollation> getCollationList() {
+      @Override public List<RelCollation> getCollationList() {
         return collationList;
       }
 
-      public RelDistribution getDistribution() {
+      @Override public RelDistribution getDistribution() {
         return RelDistributions.BROADCAST_DISTRIBUTED;
       }
 
-      public boolean isKey(ImmutableBitSet columns) {
+      @Override public boolean isKey(ImmutableBitSet columns) {
         return false;
       }
 
-      public List<ImmutableBitSet> getKeys() {
+      @Override public List<ImmutableBitSet> getKeys() {
         return ImmutableList.of();
       }
 
-      public List<RelReferentialConstraint> getReferentialConstraints() {
+      @Override public List<RelReferentialConstraint> getReferentialConstraints() {
         return ImmutableList.of();
       }
 
-      public List<ColumnStrategy> getColumnStrategies() {
+      @Override public List<ColumnStrategy> getColumnStrategies() {
         throw new UnsupportedOperationException();
       }
 
-      public Expression getExpression(Class clazz) {
+      @Override public Expression getExpression(Class clazz) {
         return null;
       }
 
-      public RelOptTable extend(List<RelDataTypeField> extendedFields) {
+      @Override public RelOptTable extend(List<RelDataTypeField> extendedFields) {
         final RelDataType extendedRowType =
             getRelOptSchema().getTypeFactory().builder()
                 .addAll(rowType.getFieldList())
@@ -462,63 +476,63 @@ public abstract class SqlToRelTestBase {
       this.parent = parent;
     }
 
-    public <T> T unwrap(Class<T> clazz) {
+    @Override public <T> T unwrap(Class<T> clazz) {
       if (clazz.isInstance(this)) {
         return clazz.cast(this);
       }
       return parent.unwrap(clazz);
     }
 
-    public Expression getExpression(Class clazz) {
+    @Override public Expression getExpression(Class clazz) {
       return parent.getExpression(clazz);
     }
 
-    public RelOptTable extend(List<RelDataTypeField> extendedFields) {
+    @Override public RelOptTable extend(List<RelDataTypeField> extendedFields) {
       return parent.extend(extendedFields);
     }
 
-    public List<String> getQualifiedName() {
+    @Override public List<String> getQualifiedName() {
       return parent.getQualifiedName();
     }
 
-    public double getRowCount() {
+    @Override public double getRowCount() {
       return parent.getRowCount();
     }
 
-    public RelDataType getRowType() {
+    @Override public RelDataType getRowType() {
       return parent.getRowType();
     }
 
-    public RelOptSchema getRelOptSchema() {
+    @Override public RelOptSchema getRelOptSchema() {
       return parent.getRelOptSchema();
     }
 
-    public RelNode toRel(ToRelContext context) {
+    @Override public RelNode toRel(ToRelContext context) {
       return LogicalTableScan.create(context.getCluster(), this,
           context.getTableHints());
     }
 
-    public List<RelCollation> getCollationList() {
+    @Override public List<RelCollation> getCollationList() {
       return parent.getCollationList();
     }
 
-    public RelDistribution getDistribution() {
+    @Override public RelDistribution getDistribution() {
       return parent.getDistribution();
     }
 
-    public boolean isKey(ImmutableBitSet columns) {
+    @Override public boolean isKey(ImmutableBitSet columns) {
       return parent.isKey(columns);
     }
 
-    public List<ImmutableBitSet> getKeys() {
+    @Override public List<ImmutableBitSet> getKeys() {
       return parent.getKeys();
     }
 
-    public List<RelReferentialConstraint> getReferentialConstraints() {
+    @Override public List<RelReferentialConstraint> getReferentialConstraints() {
       return parent.getReferentialConstraints();
     }
 
-    public List<ColumnStrategy> getColumnStrategies() {
+    @Override public List<ColumnStrategy> getColumnStrategies() {
       return parent.getColumnStrategies();
     }
   }
@@ -590,7 +604,7 @@ public abstract class SqlToRelTestBase {
       this.context = context;
     }
 
-    public RelRoot convertSqlToRel(String sql) {
+    @Override public RelRoot convertSqlToRel(String sql) {
       Objects.requireNonNull(sql);
       final SqlNode sqlQuery;
       final SqlToRelConverter.Config localConfig;
@@ -677,18 +691,18 @@ public abstract class SqlToRelTestBase {
       return planner;
     }
 
-    public SqlNode parseQuery(String sql) throws Exception {
+    @Override public SqlNode parseQuery(String sql) throws Exception {
       final SqlParser.Config config =
           SqlParser.configBuilder().setConformance(getConformance()).build();
       SqlParser parser = SqlParser.create(sql, config);
       return parser.parseQuery();
     }
 
-    public SqlConformance getConformance() {
+    @Override public SqlConformance getConformance() {
       return conformance;
     }
 
-    public SqlValidator createValidator(
+    @Override public SqlValidator createValidator(
         SqlValidatorCatalogReader catalogReader,
         RelDataTypeFactory typeFactory) {
       return new FarragoTestValidator(
@@ -701,7 +715,7 @@ public abstract class SqlToRelTestBase {
               .withIdentifierExpansion(true));
     }
 
-    public final SqlOperatorTable getOperatorTable() {
+    @Override public final SqlOperatorTable getOperatorTable() {
       if (opTab == null) {
         opTab = createOperatorTable();
       }
@@ -720,7 +734,7 @@ public abstract class SqlToRelTestBase {
       return opTab;
     }
 
-    public Prepare.CatalogReader createCatalogReader(
+    @Override public Prepare.CatalogReader createCatalogReader(
         RelDataTypeFactory typeFactory) {
       MockCatalogReader catalogReader;
       if (this.catalogReaderFactory != null) {
@@ -731,17 +745,17 @@ public abstract class SqlToRelTestBase {
       return catalogReader.init();
     }
 
-    public RelOptPlanner createPlanner() {
+    @Override public RelOptPlanner createPlanner() {
       return new MockRelOptPlanner(context);
     }
 
-    public void assertConvertsTo(
+    @Override public void assertConvertsTo(
         String sql,
         String plan) {
       assertConvertsTo(sql, plan, false);
     }
 
-    public void assertConvertsTo(
+    @Override public void assertConvertsTo(
         String sql,
         String plan,
         boolean trim) {
@@ -777,18 +791,18 @@ public abstract class SqlToRelTestBase {
       return new RelFieldTrimmer(getValidator(), relBuilder);
     }
 
-    public DiffRepository getDiffRepos() {
+    @Override public DiffRepository getDiffRepos() {
       return diffRepos;
     }
 
-    public SqlValidator getValidator() {
+    @Override public SqlValidator getValidator() {
       final RelDataTypeFactory typeFactory = getTypeFactory();
       final SqlValidatorCatalogReader catalogReader =
           createCatalogReader(typeFactory);
       return createValidator(catalogReader, typeFactory);
     }
 
-    public TesterImpl withDecorrelation(boolean enableDecorrelate) {
+    @Override public TesterImpl withDecorrelation(boolean enableDecorrelate) {
       return this.enableDecorrelate == enableDecorrelate
           ? this
           : new TesterImpl(diffRepos, enableDecorrelate, enableTrim,
@@ -796,7 +810,7 @@ public abstract class SqlToRelTestBase {
               clusterFactory, config, conformance, context);
     }
 
-    public Tester withLateDecorrelation(boolean enableLateDecorrelate) {
+    @Override public Tester withLateDecorrelation(boolean enableLateDecorrelate) {
       return this.enableLateDecorrelate == enableLateDecorrelate
           ? this
           : new TesterImpl(diffRepos, enableDecorrelate, enableTrim,
@@ -804,7 +818,7 @@ public abstract class SqlToRelTestBase {
               clusterFactory, config, conformance, context);
     }
 
-    public TesterImpl withConfig(SqlToRelConverter.Config config) {
+    @Override public TesterImpl withConfig(SqlToRelConverter.Config config) {
       return this.config == config
           ? this
           : new TesterImpl(diffRepos, enableDecorrelate, enableTrim,
@@ -812,7 +826,7 @@ public abstract class SqlToRelTestBase {
               clusterFactory, config, conformance, context);
     }
 
-    public Tester withTrim(boolean enableTrim) {
+    @Override public Tester withTrim(boolean enableTrim) {
       return this.enableTrim == enableTrim
           ? this
           : new TesterImpl(diffRepos, enableDecorrelate, enableTrim,
@@ -820,7 +834,7 @@ public abstract class SqlToRelTestBase {
               clusterFactory, config, conformance, context);
     }
 
-    public Tester withExpand(boolean enableExpand) {
+    @Override public Tester withExpand(boolean enableExpand) {
       return this.enableExpand == enableExpand
           ? this
           : new TesterImpl(diffRepos, enableDecorrelate, enableTrim,
@@ -828,39 +842,39 @@ public abstract class SqlToRelTestBase {
               clusterFactory, config, conformance, context);
     }
 
-    public Tester withConformance(SqlConformance conformance) {
+    @Override public Tester withConformance(SqlConformance conformance) {
       return new TesterImpl(diffRepos, enableDecorrelate, false,
           enableExpand, enableLateDecorrelate, enableTypeCoercion, catalogReaderFactory,
           clusterFactory, config, conformance, context);
     }
 
-    public Tester enableTypeCoercion(boolean enableTypeCoercion) {
+    @Override public Tester enableTypeCoercion(boolean enableTypeCoercion) {
       return new TesterImpl(diffRepos, enableDecorrelate, false,
           enableExpand, enableLateDecorrelate, enableTypeCoercion, catalogReaderFactory,
           clusterFactory, config, conformance, context);
     }
 
-    public Tester withCatalogReaderFactory(
+    @Override public Tester withCatalogReaderFactory(
         SqlTestFactory.MockCatalogReaderFactory factory) {
       return new TesterImpl(diffRepos, enableDecorrelate, false,
           enableExpand, enableLateDecorrelate, enableTypeCoercion, factory,
           clusterFactory, config, conformance, context);
     }
 
-    public Tester withClusterFactory(
+    @Override public Tester withClusterFactory(
         Function<RelOptCluster, RelOptCluster> clusterFactory) {
       return new TesterImpl(diffRepos, enableDecorrelate, false,
           enableExpand, enableLateDecorrelate, enableTypeCoercion, catalogReaderFactory,
           clusterFactory, config, conformance, context);
     }
 
-    public Tester withContext(Context context) {
+    @Override public Tester withContext(Context context) {
       return new TesterImpl(diffRepos, enableDecorrelate, false,
           enableExpand, enableLateDecorrelate, enableTypeCoercion, catalogReaderFactory,
           clusterFactory, config, conformance, context);
     }
 
-    public boolean isLateDecorrelate() {
+    @Override public boolean isLateDecorrelate() {
       return enableLateDecorrelate;
     }
   }
diff --git a/core/src/test/java/org/apache/calcite/test/TypeCoercionConverterTest.java b/core/src/test/java/org/apache/calcite/test/TypeCoercionConverterTest.java
index 029f44f..475f440 100644
--- a/core/src/test/java/org/apache/calcite/test/TypeCoercionConverterTest.java
+++ b/core/src/test/java/org/apache/calcite/test/TypeCoercionConverterTest.java
@@ -27,8 +27,8 @@ import org.junit.jupiter.api.Test;
  */
 class TypeCoercionConverterTest extends SqlToRelTestBase {
 
-  @Override protected DiffRepository getDiffRepos() {
-    return DiffRepository.lookup(TypeCoercionConverterTest.class);
+  TypeCoercionConverterTest(DiffRepository repository) {
+    super(repository);
   }
 
   @Override protected Tester createTester() {