You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@lucene.apache.org by ab...@apache.org on 2020/05/06 12:41:59 UTC

[lucene-solr] branch jira/solr-14423 updated: SOLR-14423: Add a unit test for FilterCalciteConnection. Fix some issues from review.

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

ab pushed a commit to branch jira/solr-14423
in repository https://gitbox.apache.org/repos/asf/lucene-solr.git


The following commit(s) were added to refs/heads/jira/solr-14423 by this push:
     new b1ca0a5  SOLR-14423: Add a unit test for FilterCalciteConnection. Fix some issues from review.
b1ca0a5 is described below

commit b1ca0a5e36da5010588f23fe503b28838a1b29ac
Author: Andrzej Bialecki <ab...@apache.org>
AuthorDate: Wed May 6 14:41:16 2020 +0200

    SOLR-14423: Add a unit test for FilterCalciteConnection. Fix some issues from review.
---
 .../java/org/apache/solr/handler/GraphHandler.java |   2 +-
 .../apache/solr/handler/sql/CalciteSolrDriver.java |   2 +-
 ...onWrapper.java => FilterCalciteConnection.java} | 144 +++++++++++----------
 .../org/apache/solr/handler/sql/SolrSchema.java    |   6 +
 .../solr/metrics/reporters/solr/SolrReporter.java  |  34 ++++-
 .../org/apache/solr/search/join/XCJFQuery.java     |   1 -
 .../handler/sql/TestFilterCalciteConnection.java   |  93 +++++++++++++
 7 files changed, 203 insertions(+), 79 deletions(-)

diff --git a/solr/core/src/java/org/apache/solr/handler/GraphHandler.java b/solr/core/src/java/org/apache/solr/handler/GraphHandler.java
index 4d6f539..22c082f 100644
--- a/solr/core/src/java/org/apache/solr/handler/GraphHandler.java
+++ b/solr/core/src/java/org/apache/solr/handler/GraphHandler.java
@@ -95,8 +95,8 @@ public class GraphHandler extends RequestHandlerBase implements SolrCoreAware, P
     String defaultCollection;
     String defaultZkhost;
     CoreContainer coreContainer = core.getCoreContainer();
-    this.solrClientCache = coreContainer.getSolrClientCache();
     this.coreName = core.getName();
+    this.solrClientCache = coreContainer.getSolrClientCache();
 
     if(coreContainer.isZooKeeperAware()) {
       defaultCollection = core.getCoreDescriptor().getCollectionName();
diff --git a/solr/core/src/java/org/apache/solr/handler/sql/CalciteSolrDriver.java b/solr/core/src/java/org/apache/solr/handler/sql/CalciteSolrDriver.java
index 3a995f6..88e8b95 100644
--- a/solr/core/src/java/org/apache/solr/handler/sql/CalciteSolrDriver.java
+++ b/solr/core/src/java/org/apache/solr/handler/sql/CalciteSolrDriver.java
@@ -65,7 +65,7 @@ public class CalciteSolrDriver extends Driver {
     // Set the default schema
     calciteConnection.setSchema(schemaName);
 
-    return new CalciteConnectionWrapper(calciteConnection) {
+    return new FilterCalciteConnection(calciteConnection) {
       @Override
       public void close() throws SQLException {
         solrSchema.close();
diff --git a/solr/core/src/java/org/apache/solr/handler/sql/CalciteConnectionWrapper.java b/solr/core/src/java/org/apache/solr/handler/sql/FilterCalciteConnection.java
similarity index 70%
rename from solr/core/src/java/org/apache/solr/handler/sql/CalciteConnectionWrapper.java
rename to solr/core/src/java/org/apache/solr/handler/sql/FilterCalciteConnection.java
index cd102b6..1984e6c 100644
--- a/solr/core/src/java/org/apache/solr/handler/sql/CalciteConnectionWrapper.java
+++ b/solr/core/src/java/org/apache/solr/handler/sql/FilterCalciteConnection.java
@@ -45,334 +45,338 @@ import org.apache.calcite.linq4j.tree.Expression;
 import org.apache.calcite.schema.SchemaPlus;
 
 /**
- * A wrapper that allows adding pre- post-method behaviors to
- * {@link CalciteConnection}.
+ * A filter that contains another {@link CalciteConnection} and
+ * allows adding pre- post-method behaviors.
  */
-class CalciteConnectionWrapper implements CalciteConnection {
+class FilterCalciteConnection implements CalciteConnection {
 
-  protected final CalciteConnection delegate;
+  protected final CalciteConnection in;
 
-  CalciteConnectionWrapper(CalciteConnection delegate) {
-    this.delegate = delegate;
+  FilterCalciteConnection(CalciteConnection in) {
+    this.in = in;
+  }
+
+  public CalciteConnection getDelegate() {
+    return in;
   }
 
   @Override
   public SchemaPlus getRootSchema() {
-    return delegate.getRootSchema();
+    return in.getRootSchema();
   }
 
   @Override
   public JavaTypeFactory getTypeFactory() {
-    return delegate.getTypeFactory();
+    return in.getTypeFactory();
   }
 
   @Override
   public Properties getProperties() {
-    return delegate.getProperties();
+    return in.getProperties();
   }
 
   @Override
   public Statement createStatement() throws SQLException {
-    return delegate.createStatement();
+    return in.createStatement();
   }
 
   @Override
   public PreparedStatement prepareStatement(String sql) throws SQLException {
-    return delegate.prepareStatement(sql);
+    return in.prepareStatement(sql);
   }
 
   @Override
   public CallableStatement prepareCall(String sql) throws SQLException {
-    return delegate.prepareCall(sql);
+    return in.prepareCall(sql);
   }
 
   @Override
   public String nativeSQL(String sql) throws SQLException {
-    return delegate.nativeSQL(sql);
+    return in.nativeSQL(sql);
   }
 
   @Override
   public void setAutoCommit(boolean autoCommit) throws SQLException {
-    delegate.setAutoCommit(autoCommit);
+    in.setAutoCommit(autoCommit);
   }
 
   @Override
   public boolean getAutoCommit() throws SQLException {
-    return delegate.getAutoCommit();
+    return in.getAutoCommit();
   }
 
   @Override
   public void commit() throws SQLException {
-    delegate.commit();
+    in.commit();
   }
 
   @Override
   public void rollback() throws SQLException {
-    delegate.rollback();
+    in.rollback();
   }
 
   @Override
   public void close() throws SQLException {
-    delegate.close();
+    in.close();
   }
 
   @Override
   public boolean isClosed() throws SQLException {
-    return delegate.isClosed();
+    return in.isClosed();
   }
 
   @Override
   public DatabaseMetaData getMetaData() throws SQLException {
-    return delegate.getMetaData();
+    return in.getMetaData();
   }
 
   @Override
   public void setReadOnly(boolean readOnly) throws SQLException {
-    delegate.setReadOnly(readOnly);
+    in.setReadOnly(readOnly);
   }
 
   @Override
   public boolean isReadOnly() throws SQLException {
-    return delegate.isReadOnly();
+    return in.isReadOnly();
   }
 
   @Override
   public void setCatalog(String catalog) throws SQLException {
-    delegate.setCatalog(catalog);
+    in.setCatalog(catalog);
   }
 
   @Override
   public String getCatalog() throws SQLException {
-    return delegate.getCatalog();
+    return in.getCatalog();
   }
 
   @Override
   public void setTransactionIsolation(int level) throws SQLException {
-    delegate.setTransactionIsolation(level);
+    in.setTransactionIsolation(level);
   }
 
   @Override
   public int getTransactionIsolation() throws SQLException {
-    return delegate.getTransactionIsolation();
+    return in.getTransactionIsolation();
   }
 
   @Override
   public SQLWarning getWarnings() throws SQLException {
-    return delegate.getWarnings();
+    return in.getWarnings();
   }
 
   @Override
   public void clearWarnings() throws SQLException {
-    delegate.clearWarnings();
+    in.clearWarnings();
   }
 
   @Override
   public Statement createStatement(int resultSetType, int resultSetConcurrency) throws SQLException {
-    return delegate.createStatement(resultSetType, resultSetConcurrency);
+    return in.createStatement(resultSetType, resultSetConcurrency);
   }
 
   @Override
   public PreparedStatement prepareStatement(String sql, int resultSetType, int resultSetConcurrency) throws SQLException {
-    return delegate.prepareStatement(sql, resultSetType, resultSetConcurrency);
+    return in.prepareStatement(sql, resultSetType, resultSetConcurrency);
   }
 
   @Override
   public CallableStatement prepareCall(String sql, int resultSetType, int resultSetConcurrency) throws SQLException {
-    return delegate.prepareCall(sql, resultSetType, resultSetConcurrency);
+    return in.prepareCall(sql, resultSetType, resultSetConcurrency);
   }
 
   @Override
   public Map<String, Class<?>> getTypeMap() throws SQLException {
-    return delegate.getTypeMap();
+    return in.getTypeMap();
   }
 
   @Override
   public void setTypeMap(Map<String, Class<?>> map) throws SQLException {
-    delegate.setTypeMap(map);
+    in.setTypeMap(map);
   }
 
   @Override
   public void setHoldability(int holdability) throws SQLException {
-    delegate.setHoldability(holdability);
+    in.setHoldability(holdability);
   }
 
   @Override
   public int getHoldability() throws SQLException {
-    return delegate.getHoldability();
+    return in.getHoldability();
   }
 
   @Override
   public Savepoint setSavepoint() throws SQLException {
-    return delegate.setSavepoint();
+    return in.setSavepoint();
   }
 
   @Override
   public Savepoint setSavepoint(String name) throws SQLException {
-    return delegate.setSavepoint(name);
+    return in.setSavepoint(name);
   }
 
   @Override
   public void rollback(Savepoint savepoint) throws SQLException {
-    delegate.rollback(savepoint);
+    in.rollback(savepoint);
   }
 
   @Override
   public void releaseSavepoint(Savepoint savepoint) throws SQLException {
-    delegate.releaseSavepoint(savepoint);
+    in.releaseSavepoint(savepoint);
   }
 
   @Override
   public Statement createStatement(int resultSetType, int resultSetConcurrency, int resultSetHoldability) throws SQLException {
-    return delegate.createStatement(resultSetType, resultSetConcurrency, resultSetHoldability);
+    return in.createStatement(resultSetType, resultSetConcurrency, resultSetHoldability);
   }
 
   @Override
   public PreparedStatement prepareStatement(String sql, int resultSetType, int resultSetConcurrency, int resultSetHoldability) throws SQLException {
-    return delegate.prepareStatement(sql, resultSetType, resultSetConcurrency, resultSetHoldability);
+    return in.prepareStatement(sql, resultSetType, resultSetConcurrency, resultSetHoldability);
   }
 
   @Override
   public CallableStatement prepareCall(String sql, int resultSetType, int resultSetConcurrency, int resultSetHoldability) throws SQLException {
-    return delegate.prepareCall(sql, resultSetType, resultSetConcurrency, resultSetHoldability);
+    return in.prepareCall(sql, resultSetType, resultSetConcurrency, resultSetHoldability);
   }
 
   @Override
   public PreparedStatement prepareStatement(String sql, int autoGeneratedKeys) throws SQLException {
-    return delegate.prepareStatement(sql, autoGeneratedKeys);
+    return in.prepareStatement(sql, autoGeneratedKeys);
   }
 
   @Override
   public PreparedStatement prepareStatement(String sql, int[] columnIndexes) throws SQLException {
-    return delegate.prepareStatement(sql, columnIndexes);
+    return in.prepareStatement(sql, columnIndexes);
   }
 
   @Override
   public PreparedStatement prepareStatement(String sql, String[] columnNames) throws SQLException {
-    return delegate.prepareStatement(sql, columnNames);
+    return in.prepareStatement(sql, columnNames);
   }
 
   @Override
   public Clob createClob() throws SQLException {
-    return delegate.createClob();
+    return in.createClob();
   }
 
   @Override
   public Blob createBlob() throws SQLException {
-    return delegate.createBlob();
+    return in.createBlob();
   }
 
   @Override
   public NClob createNClob() throws SQLException {
-    return delegate.createNClob();
+    return in.createNClob();
   }
 
   @Override
   public SQLXML createSQLXML() throws SQLException {
-    return delegate.createSQLXML();
+    return in.createSQLXML();
   }
 
   @Override
   public boolean isValid(int timeout) throws SQLException {
-    return delegate.isValid(timeout);
+    return in.isValid(timeout);
   }
 
   @Override
   public void setClientInfo(String name, String value) throws SQLClientInfoException {
-    delegate.setClientInfo(name, value);
+    in.setClientInfo(name, value);
   }
 
   @Override
   public void setClientInfo(Properties properties) throws SQLClientInfoException {
-    delegate.setClientInfo(properties);
+    in.setClientInfo(properties);
   }
 
   @Override
   public String getClientInfo(String name) throws SQLException {
-    return delegate.getClientInfo(name);
+    return in.getClientInfo(name);
   }
 
   @Override
   public Properties getClientInfo() throws SQLException {
-    return delegate.getClientInfo();
+    return in.getClientInfo();
   }
 
   @Override
   public Array createArrayOf(String typeName, Object[] elements) throws SQLException {
-    return delegate.createArrayOf(typeName, elements);
+    return in.createArrayOf(typeName, elements);
   }
 
   @Override
   public Struct createStruct(String typeName, Object[] attributes) throws SQLException {
-    return delegate.createStruct(typeName, attributes);
+    return in.createStruct(typeName, attributes);
   }
 
   @Override
   public void setSchema(String schema) throws SQLException {
-    delegate.setSchema(schema);
+    in.setSchema(schema);
   }
 
   @Override
   public String getSchema() throws SQLException {
-    return delegate.getSchema();
+    return in.getSchema();
   }
 
   @Override
   public void abort(Executor executor) throws SQLException {
-    delegate.abort(executor);
+    in.abort(executor);
   }
 
   @Override
   public void setNetworkTimeout(Executor executor, int milliseconds) throws SQLException {
-    delegate.setNetworkTimeout(executor, milliseconds);
+    in.setNetworkTimeout(executor, milliseconds);
   }
 
   @Override
   public int getNetworkTimeout() throws SQLException {
-    return delegate.getNetworkTimeout();
+    return in.getNetworkTimeout();
   }
 
   @Override
   public CalciteConnectionConfig config() {
-    return delegate.config();
+    return in.config();
   }
 
   @Override
   public CalcitePrepare.Context createPrepareContext() {
-    return delegate.createPrepareContext();
+    return in.createPrepareContext();
   }
 
   @Override
   public <T> T unwrap(Class<T> iface) throws SQLException {
-    return delegate.unwrap(iface);
+    return in.unwrap(iface);
   }
 
   @Override
   public boolean isWrapperFor(Class<?> iface) throws SQLException {
-    return delegate.isWrapperFor(iface);
+    return in.isWrapperFor(iface);
   }
 
   @Override
   public <T> Queryable<T> createQuery(Expression expression, Class<T> aClass) {
-    return delegate.createQuery(expression, aClass);
+    return in.createQuery(expression, aClass);
   }
 
   @Override
   public <T> Queryable<T> createQuery(Expression expression, Type type) {
-    return delegate.createQuery(expression, type);
+    return in.createQuery(expression, type);
   }
 
   @Override
   public <T> T execute(Expression expression, Class<T> aClass) {
-    return delegate.execute(expression, aClass);
+    return in.execute(expression, aClass);
   }
 
   @Override
   public <T> T execute(Expression expression, Type type) {
-    return delegate.execute(expression, type);
+    return in.execute(expression, type);
   }
 
   @Override
   public <T> Enumerator<T> executeQuery(Queryable<T> queryable) {
-    return delegate.executeQuery(queryable);
+    return in.executeQuery(queryable);
   }
 }
diff --git a/solr/core/src/java/org/apache/solr/handler/sql/SolrSchema.java b/solr/core/src/java/org/apache/solr/handler/sql/SolrSchema.java
index 0d7466f..54ce899 100644
--- a/solr/core/src/java/org/apache/solr/handler/sql/SolrSchema.java
+++ b/solr/core/src/java/org/apache/solr/handler/sql/SolrSchema.java
@@ -44,6 +44,7 @@ import com.google.common.collect.ImmutableMap;
 class SolrSchema extends AbstractSchema implements Closeable {
   final Properties properties;
   final SolrClientCache solrClientCache = new SolrClientCache();
+  private volatile boolean isClosed = false;
 
   SolrSchema(Properties properties) {
     super();
@@ -57,6 +58,11 @@ class SolrSchema extends AbstractSchema implements Closeable {
   @Override
   public void close() {
     solrClientCache.close();
+    isClosed = true;
+  }
+
+  public boolean isClosed() {
+    return isClosed;
   }
 
   @Override
diff --git a/solr/core/src/java/org/apache/solr/metrics/reporters/solr/SolrReporter.java b/solr/core/src/java/org/apache/solr/metrics/reporters/solr/SolrReporter.java
index 713137c..7e8ffd7 100644
--- a/solr/core/src/java/org/apache/solr/metrics/reporters/solr/SolrReporter.java
+++ b/solr/core/src/java/org/apache/solr/metrics/reporters/solr/SolrReporter.java
@@ -257,15 +257,24 @@ public class SolrReporter extends ScheduledReporter {
      *                    null to indicate that reporting should be skipped. Note: this
      *                    function will be called every time just before report is sent.
      * @return configured instance of reporter
+     * @deprecated use {@link #build(SolrClientCache, Supplier)} instead.
      */
     public SolrReporter build(HttpClient client, Supplier<String> urlProvider) {
-      return new SolrReporter(new SolrClientCache(client), urlProvider, metricManager, reports, handler, reporterId, rateUnit, durationUnit,
-          params, skipHistograms, skipAggregateValues, cloudClient, compact, true);
+      return new SolrReporter(client, urlProvider, metricManager, reports, handler, reporterId, rateUnit, durationUnit,
+          params, skipHistograms, skipAggregateValues, cloudClient, compact);
     }
 
+    /**
+     * Build it.
+     * @param solrClientCache an instance of {@link SolrClientCache} to be used for making calls.
+     * @param urlProvider function that returns the base URL of Solr instance to target. May return
+     *                    null to indicate that reporting should be skipped. Note: this
+     *                    function will be called every time just before report is sent.
+     * @return configured instance of reporter
+     */
     public SolrReporter build(SolrClientCache solrClientCache, Supplier<String> urlProvider) {
-      return new SolrReporter(solrClientCache, urlProvider, metricManager, reports, handler, reporterId, rateUnit, durationUnit,
-          params, skipHistograms, skipAggregateValues, cloudClient, compact, false);
+      return new SolrReporter(solrClientCache, false, urlProvider, metricManager, reports, handler, reporterId, rateUnit, durationUnit,
+          params, skipHistograms, skipAggregateValues, cloudClient, compact);
     }
 
   }
@@ -312,11 +321,24 @@ public class SolrReporter extends ScheduledReporter {
   // We delegate to registries anyway, so having a dummy registry is harmless.
   private static final MetricRegistry dummyRegistry = new MetricRegistry();
 
-  public SolrReporter(SolrClientCache solrClientCache, Supplier<String> urlProvider, SolrMetricManager metricManager,
+  // back-compat constructor
+  @Deprecated(since = "8.6.0")
+  public SolrReporter(HttpClient httpClient, Supplier<String> urlProvider, SolrMetricManager metricManager,
+                      List<Report> metrics, String handler,
+                      String reporterId, TimeUnit rateUnit, TimeUnit durationUnit,
+                      SolrParams params, boolean skipHistograms, boolean skipAggregateValues,
+                      boolean cloudClient, boolean compact) {
+    this (new SolrClientCache(httpClient), true, urlProvider, metricManager,
+        metrics, handler, reporterId, rateUnit, durationUnit,
+        params, skipHistograms, skipAggregateValues, cloudClient, compact);
+  }
+
+  public SolrReporter(SolrClientCache solrClientCache, boolean closeClientCache,
+                      Supplier<String> urlProvider, SolrMetricManager metricManager,
                       List<Report> metrics, String handler,
                       String reporterId, TimeUnit rateUnit, TimeUnit durationUnit,
                       SolrParams params, boolean skipHistograms, boolean skipAggregateValues,
-                      boolean cloudClient, boolean compact, boolean closeClientCache) {
+                      boolean cloudClient, boolean compact) {
     super(dummyRegistry, "solr-reporter", MetricFilter.ALL, rateUnit, durationUnit, null, true);
 
     this.metricManager = metricManager;
diff --git a/solr/core/src/java/org/apache/solr/search/join/XCJFQuery.java b/solr/core/src/java/org/apache/solr/search/join/XCJFQuery.java
index 32e7c51..fbe12e8 100644
--- a/solr/core/src/java/org/apache/solr/search/join/XCJFQuery.java
+++ b/solr/core/src/java/org/apache/solr/search/join/XCJFQuery.java
@@ -299,7 +299,6 @@ public class XCJFQuery extends Query {
         throw new SolrException(SolrException.ErrorCode.SERVER_ERROR, e);
       } finally {
         solrStream.close();
-        solrClientCache.close();
       }
 
       return collector.getDocSet();
diff --git a/solr/core/src/test/org/apache/solr/handler/sql/TestFilterCalciteConnection.java b/solr/core/src/test/org/apache/solr/handler/sql/TestFilterCalciteConnection.java
new file mode 100644
index 0000000..180817f
--- /dev/null
+++ b/solr/core/src/test/org/apache/solr/handler/sql/TestFilterCalciteConnection.java
@@ -0,0 +1,93 @@
+/*
+ * 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.solr.handler.sql;
+
+import java.lang.reflect.Method;
+import java.lang.reflect.Modifier;
+import java.sql.Connection;
+import java.sql.DriverManager;
+import java.util.Properties;
+
+import org.apache.calcite.jdbc.CalciteConnection;
+import org.apache.calcite.schema.SchemaPlus;
+import org.apache.solr.cloud.SolrCloudTestCase;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+/**
+ *
+ */
+public class TestFilterCalciteConnection extends SolrCloudTestCase {
+
+  @BeforeClass
+  public static void setupCluster() throws Exception {
+    configureCluster(1)
+        .addConfig("conf", configset("cloud-minimal"))
+        .configure();
+  }
+
+  @Test
+  public void testConnectionClosesSchema() throws Exception {
+    String zkHost = cluster.getSolrClient().getZkHost();
+    Properties props = new Properties();
+    props.setProperty("zk", zkHost);
+    Connection conn = null;
+    try {
+      // load the driver
+      Class.forName(CalciteSolrDriver.class.getName());
+      conn = DriverManager.getConnection("jdbc:calcitesolr:test", props);
+      assertTrue("unexpected connection impl: " + conn.getClass().getName(),
+          conn instanceof FilterCalciteConnection);
+      SchemaPlus rootSchema = ((FilterCalciteConnection)conn).getRootSchema();
+      SchemaPlus subSchema = rootSchema.getSubSchema(zkHost);
+      assertNotNull("missing SolrSchema", subSchema);
+      SolrSchema solrSchema = subSchema.unwrap(SolrSchema.class);
+      // test that conn.close() propagates to the schema
+      conn.close();
+      assertTrue("SolrSchema not closed after connection close!", solrSchema.isClosed());
+    } finally {
+      if (conn != null && !conn.isClosed()) {
+        conn.close();
+      }
+    }
+  }
+
+  @Test
+  public void testMethodsOverriden() throws Exception {
+    implTestDeclaredMethodsOverridden(CalciteConnection.class, FilterCalciteConnection.class);
+  }
+
+  private void implTestDeclaredMethodsOverridden(Class<?> superClass, Class<?> subClass) throws Exception {
+    for (final Method superClassMethod : superClass.getDeclaredMethods()) {
+      final int modifiers = superClassMethod.getModifiers();
+      if (Modifier.isPrivate(modifiers)) continue;
+      if (Modifier.isFinal(modifiers)) continue;
+      if (Modifier.isStatic(modifiers)) continue;
+      try {
+        final Method subClassMethod = subClass.getDeclaredMethod(
+            superClassMethod.getName(),
+            superClassMethod.getParameterTypes());
+        assertEquals("getReturnType() difference",
+            superClassMethod.getReturnType(),
+            subClassMethod.getReturnType());
+      } catch (NoSuchMethodException e) {
+        fail(subClass + " needs to override '" + superClassMethod + "'");
+      }
+    }
+  }
+}