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/03/03 01:51:44 UTC

[calcite] branch main updated: [CALCITE-5545] Allow for overriding SqlValidator to enable custom SqlNode validation

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


The following commit(s) were added to refs/heads/main by this push:
     new 4e19520962 [CALCITE-5545] Allow for overriding SqlValidator to enable custom SqlNode validation
4e19520962 is described below

commit 4e195209629f1a0ebd1e6f547ea3f0e7587b309f
Author: Oliver Lee <ol...@google.com>
AuthorDate: Fri Feb 24 22:47:51 2023 +0000

    [CALCITE-5545] Allow for overriding SqlValidator to enable custom SqlNode validation
    
    Make methods in SqlValidatorFixture, RelOptFixture public to
    enable testing.
    
    In Driver, add createPrepare() and withPrepareFactory()
    methods, and deprecate createPrepareFactory().
    
    Change the type of the Driver.prepareFactory field from
    Function0 (Guava) to Supplier (JDK). A few other places must
    remain Function0 because of backwards compatibility.
    
    Add tests for overriding createPrepareFactory, calling
    withPrepareFactory, and overriding createPrepare.
    
    Close apache/calcite#3084
    
    Co-authored-by: Oliver Lee <ol...@google.com>
    Co-authored-by: Julian Hyde <jh...@apache.org>
---
 .../apache/calcite/jdbc/CalciteConnectionImpl.java |   8 +-
 .../org/apache/calcite/jdbc/CalciteStatement.java  |   2 +-
 .../main/java/org/apache/calcite/jdbc/Driver.java  |  54 +++++++-
 .../apache/calcite/prepare/CalcitePrepareImpl.java |  64 ++++++----
 .../calcite/prepare/CalciteSqlValidator.java       |   7 +-
 .../java/org/apache/calcite/test/JdbcTest.java     | 137 ++++++++++++++++-----
 .../org/apache/calcite/test/RelOptFixture.java     |   2 +-
 .../apache/calcite/test/SqlValidatorFixture.java   |   8 +-
 8 files changed, 210 insertions(+), 72 deletions(-)

diff --git a/core/src/main/java/org/apache/calcite/jdbc/CalciteConnectionImpl.java b/core/src/main/java/org/apache/calcite/jdbc/CalciteConnectionImpl.java
index cf01ec65e1..12fae54572 100644
--- a/core/src/main/java/org/apache/calcite/jdbc/CalciteConnectionImpl.java
+++ b/core/src/main/java/org/apache/calcite/jdbc/CalciteConnectionImpl.java
@@ -39,7 +39,6 @@ import org.apache.calcite.linq4j.Enumerator;
 import org.apache.calcite.linq4j.Ord;
 import org.apache.calcite.linq4j.QueryProvider;
 import org.apache.calcite.linq4j.Queryable;
-import org.apache.calcite.linq4j.function.Function0;
 import org.apache.calcite.linq4j.tree.Expression;
 import org.apache.calcite.linq4j.tree.Expressions;
 import org.apache.calcite.materialize.Lattice;
@@ -87,6 +86,7 @@ import java.util.Map;
 import java.util.Properties;
 import java.util.TimeZone;
 import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.function.Supplier;
 
 import static org.apache.calcite.linq4j.Nullness.castNonNull;
 
@@ -104,7 +104,7 @@ abstract class CalciteConnectionImpl
   public final JavaTypeFactory typeFactory;
 
   final CalciteSchema rootSchema;
-  final Function0<CalcitePrepare> prepareFactory;
+  final Supplier<CalcitePrepare> prepareFactory;
   final CalciteServer server = new CalciteServerImpl();
 
   // must be package-protected
@@ -127,7 +127,7 @@ abstract class CalciteConnectionImpl
       @Nullable JavaTypeFactory typeFactory) {
     super(driver, factory, url, info);
     CalciteConnectionConfig cfg = new CalciteConnectionConfigImpl(info);
-    this.prepareFactory = driver.prepareFactory;
+    this.prepareFactory = driver::createPrepare;
     if (typeFactory != null) {
       this.typeFactory = typeFactory;
     } else {
@@ -232,7 +232,7 @@ abstract class CalciteConnectionImpl
       CalcitePrepare.Context prepareContext, long maxRowCount) {
     CalcitePrepare.Dummy.push(prepareContext);
     try {
-      final CalcitePrepare prepare = prepareFactory.apply();
+      final CalcitePrepare prepare = prepareFactory.get();
       return prepare.prepareSql(prepareContext, query, Object[].class,
           maxRowCount);
     } finally {
diff --git a/core/src/main/java/org/apache/calcite/jdbc/CalciteStatement.java b/core/src/main/java/org/apache/calcite/jdbc/CalciteStatement.java
index 3c7c2be7c4..9b06b7fc2a 100644
--- a/core/src/main/java/org/apache/calcite/jdbc/CalciteStatement.java
+++ b/core/src/main/java/org/apache/calcite/jdbc/CalciteStatement.java
@@ -68,7 +68,7 @@ public abstract class CalciteStatement extends AvaticaStatement {
   protected <T> CalcitePrepare.CalciteSignature<T> prepare(
       Queryable<T> queryable) {
     final CalciteConnectionImpl calciteConnection = getConnection();
-    final CalcitePrepare prepare = calciteConnection.prepareFactory.apply();
+    final CalcitePrepare prepare = calciteConnection.prepareFactory.get();
     final CalciteServerStatement serverStatement;
     try {
       serverStatement = calciteConnection.server.getStatement(handle);
diff --git a/core/src/main/java/org/apache/calcite/jdbc/Driver.java b/core/src/main/java/org/apache/calcite/jdbc/Driver.java
index 06aa637b68..20753ebd82 100644
--- a/core/src/main/java/org/apache/calcite/jdbc/Driver.java
+++ b/core/src/main/java/org/apache/calcite/jdbc/Driver.java
@@ -45,6 +45,9 @@ import java.util.Collections;
 import java.util.List;
 import java.util.Map;
 import java.util.Properties;
+import java.util.function.Supplier;
+
+import static java.util.Objects.requireNonNull;
 
 /**
  * Calcite JDBC driver.
@@ -52,18 +55,61 @@ import java.util.Properties;
 public class Driver extends UnregisteredDriver {
   public static final String CONNECT_STRING_PREFIX = "jdbc:calcite:";
 
-  final Function0<CalcitePrepare> prepareFactory;
+  protected final @Nullable Supplier<CalcitePrepare> prepareFactory;
 
   static {
     new Driver().register();
   }
 
-  @SuppressWarnings("method.invocation.invalid")
+  /** Creates a Driver. */
   public Driver() {
-    super();
-    this.prepareFactory = createPrepareFactory();
+    this(null);
+  }
+
+  /** Creates a Driver with a factory for {@code CalcitePrepare} objects;
+   * if the factory is null, the driver will call
+   * {@link #createPrepareFactory()}. */
+  protected Driver(@Nullable Supplier<CalcitePrepare> prepareFactory) {
+    this.prepareFactory = prepareFactory;
+  }
+
+  /** Creates a copy of this Driver with a new factory for creating
+   * {@link CalcitePrepare}.
+   *
+   * <p>Allows users of the Driver to change the factory without subclassing
+   * the Driver. But subclasses of the driver should override this method to
+   * create an instance of their subclass.
+   *
+   * @param prepareFactory Supplier of a {@code CalcitePrepare}
+   * @return Driver with the provided prepareFactory
+   */
+  public Driver withPrepareFactory(Supplier<CalcitePrepare> prepareFactory) {
+    requireNonNull(prepareFactory, "prepareFactory");
+    if (this.prepareFactory == prepareFactory) {
+      return this;
+    }
+    return new Driver(prepareFactory);
+  }
+
+  /** Creates a {@link CalcitePrepare} to be used to prepare a statement for
+   * execution.
+   *
+   * <p>If you wish to use a custom prepare, either override this method or
+   * call {@link #withPrepareFactory(Supplier)}. */
+  public CalcitePrepare createPrepare() {
+    if (prepareFactory != null) {
+      return prepareFactory.get();
+    }
+    return createPrepareFactory().apply();
   }
 
+  /** Returns a factory with which to create a {@link CalcitePrepare}.
+   *
+   * <p>Now deprecated; if you wish to use a custom prepare, overrides of this
+   * method will still work, but we prefer that you call
+   * {@link #withPrepareFactory(Supplier)}
+   * or override {@link #createPrepare()}. */
+  @Deprecated // to be removed before 2.0
   protected Function0<CalcitePrepare> createPrepareFactory() {
     return CalcitePrepare.DEFAULT_FACTORY;
   }
diff --git a/core/src/main/java/org/apache/calcite/prepare/CalcitePrepareImpl.java b/core/src/main/java/org/apache/calcite/prepare/CalcitePrepareImpl.java
index 4da377b366..d9655a7f7a 100644
--- a/core/src/main/java/org/apache/calcite/prepare/CalcitePrepareImpl.java
+++ b/core/src/main/java/org/apache/calcite/prepare/CalcitePrepareImpl.java
@@ -112,7 +112,6 @@ import org.apache.calcite.util.ImmutableIntList;
 import org.apache.calcite.util.Pair;
 import org.apache.calcite.util.Util;
 
-import com.google.common.base.Supplier;
 import com.google.common.collect.ImmutableList;
 import com.google.common.collect.ImmutableMap;
 import com.google.common.collect.ImmutableSet;
@@ -130,6 +129,7 @@ import java.util.LinkedHashMap;
 import java.util.List;
 import java.util.Map;
 import java.util.Set;
+import java.util.function.Supplier;
 
 import static org.apache.calcite.linq4j.Nullness.castNonNull;
 import static org.apache.calcite.util.Static.RESOURCE;
@@ -514,8 +514,10 @@ public class CalcitePrepareImpl implements CalcitePrepare {
         throw new AssertionError("factory returned null planner");
       }
       try {
+        CalcitePreparingStmt preparingStmt =
+            getPreparingStmt(context, elementType, catalogReader, planner);
         return prepare2_(context, query, elementType, maxRowCount,
-            catalogReader, planner);
+            catalogReader, preparingStmt);
       } catch (RelOptPlanner.CannotPlanException e) {
         exception = e;
       }
@@ -523,6 +525,31 @@ public class CalcitePrepareImpl implements CalcitePrepare {
     throw exception;
   }
 
+  /** Returns CalcitePreparingStmt
+   *
+   * <p>Override this function to return a custom {@link CalcitePreparingStmt} and
+   * {@link #createSqlValidator} to enable custom validation logic.
+   */
+  protected CalcitePreparingStmt getPreparingStmt(
+      Context context,
+      Type elementType,
+      CalciteCatalogReader catalogReader,
+      RelOptPlanner planner) {
+    final JavaTypeFactory typeFactory = context.getTypeFactory();
+    final EnumerableRel.Prefer prefer;
+    if (elementType == Object[].class) {
+      prefer = EnumerableRel.Prefer.ARRAY;
+    } else {
+      prefer = EnumerableRel.Prefer.CUSTOM;
+    }
+    final Convention resultConvention =
+        enableBindable ? BindableConvention.INSTANCE
+            : EnumerableConvention.INSTANCE;
+    return new CalcitePreparingStmt(this, context, catalogReader, typeFactory,
+            context.getRootSchema(), prefer, createCluster(planner, new RexBuilder(typeFactory)),
+            resultConvention, createConvertletTable());
+  }
+
   /** Quickly prepares a simple SQL statement, circumventing the usual
    * preparation process. */
   private static <T> CalciteSignature<T> simplePrepare(Context context, String sql) {
@@ -590,21 +617,8 @@ public class CalcitePrepareImpl implements CalcitePrepare {
       Type elementType,
       long maxRowCount,
       CalciteCatalogReader catalogReader,
-      RelOptPlanner planner) {
+      CalcitePreparingStmt preparingStmt) {
     final JavaTypeFactory typeFactory = context.getTypeFactory();
-    final EnumerableRel.Prefer prefer;
-    if (elementType == Object[].class) {
-      prefer = EnumerableRel.Prefer.ARRAY;
-    } else {
-      prefer = EnumerableRel.Prefer.CUSTOM;
-    }
-    final Convention resultConvention =
-        enableBindable ? BindableConvention.INSTANCE
-            : EnumerableConvention.INSTANCE;
-    final CalcitePreparingStmt preparingStmt =
-        new CalcitePreparingStmt(this, context, catalogReader, typeFactory,
-            context.getRootSchema(), prefer, createCluster(planner, new RexBuilder(typeFactory)),
-            resultConvention, createConvertletTable());
 
     final RelDataType x;
     final Prepare.PreparedResult preparedResult;
@@ -645,8 +659,7 @@ public class CalcitePrepareImpl implements CalcitePrepare {
             Meta.StatementType.OTHER_DDL);
       }
 
-      final SqlValidator validator =
-          createSqlValidator(context, catalogReader);
+      final SqlValidator validator = preparingStmt.createSqlValidator(catalogReader);
 
       preparedResult =
           preparingStmt.prepareSql(sqlNode, Object.class, validator, true);
@@ -937,8 +950,12 @@ public class CalcitePrepareImpl implements CalcitePrepare {
         prepareContext.getRootSchema().plus(), statement);
   }
 
-  /** Holds state for the process of preparing a SQL statement. */
-  static class CalcitePreparingStmt extends Prepare
+  /** Holds state for the process of preparing a SQL statement.
+   *
+   * <p>Overload this class and {@link #createSqlValidator} to provide desired SqlValidator
+   * and custom validation logic.</p>
+   */
+  public static class CalcitePreparingStmt extends Prepare
       implements RelOptTable.ViewExpander {
     protected final RelOptPlanner planner;
     protected final RexBuilder rexBuilder;
@@ -954,7 +971,12 @@ public class CalcitePrepareImpl implements CalcitePrepare {
     private int expansionDepth;
     private @Nullable SqlValidator sqlValidator;
 
-    CalcitePreparingStmt(CalcitePrepareImpl prepare,
+    /** Constructor.
+     *
+     * <p>Overload this constructor and {@link #createSqlValidator} to provide desired
+     *  SqlValidaor and custom validation logic.</p>
+     */
+    public CalcitePreparingStmt(CalcitePrepareImpl prepare,
         Context context,
         CatalogReader catalogReader,
         RelDataTypeFactory typeFactory,
diff --git a/core/src/main/java/org/apache/calcite/prepare/CalciteSqlValidator.java b/core/src/main/java/org/apache/calcite/prepare/CalciteSqlValidator.java
index b8a5623737..8033173882 100644
--- a/core/src/main/java/org/apache/calcite/prepare/CalciteSqlValidator.java
+++ b/core/src/main/java/org/apache/calcite/prepare/CalciteSqlValidator.java
@@ -22,10 +22,11 @@ import org.apache.calcite.sql.SqlInsert;
 import org.apache.calcite.sql.SqlOperatorTable;
 import org.apache.calcite.sql.validate.SqlValidatorImpl;
 
-/** Validator. */
-class CalciteSqlValidator extends SqlValidatorImpl {
+/** Validator.
+ */
+public class CalciteSqlValidator extends SqlValidatorImpl {
 
-  CalciteSqlValidator(SqlOperatorTable opTab,
+  public CalciteSqlValidator(SqlOperatorTable opTab,
       CalciteCatalogReader catalogReader, JavaTypeFactory typeFactory,
       Config config) {
     super(opTab, catalogReader, typeFactory, config);
diff --git a/core/src/test/java/org/apache/calcite/test/JdbcTest.java b/core/src/test/java/org/apache/calcite/test/JdbcTest.java
index 5f8590cde4..e62d455277 100644
--- a/core/src/test/java/org/apache/calcite/test/JdbcTest.java
+++ b/core/src/test/java/org/apache/calcite/test/JdbcTest.java
@@ -140,6 +140,7 @@ import java.util.Map;
 import java.util.Properties;
 import java.util.Set;
 import java.util.TimeZone;
+import java.util.concurrent.atomic.AtomicInteger;
 import java.util.function.Consumer;
 import java.util.regex.Pattern;
 import java.util.stream.Collectors;
@@ -163,6 +164,7 @@ import static org.junit.jupiter.api.Assertions.assertArrayEquals;
 import static org.junit.jupiter.api.Assertions.assertEquals;
 import static org.junit.jupiter.api.Assertions.assertFalse;
 import static org.junit.jupiter.api.Assertions.assertNotNull;
+import static org.junit.jupiter.api.Assertions.assertThrows;
 import static org.junit.jupiter.api.Assertions.assertTrue;
 import static org.junit.jupiter.api.Assertions.fail;
 
@@ -764,17 +766,71 @@ public class JdbcTest {
 
   /** Tests that a driver can be extended with its own parser and can execute
    * its own flavor of DDL. */
-  @Test void testMockDdl() throws Exception {
-    final MockDdlDriver driver = new MockDdlDriver();
+  @Test void testMockDdl() {
+    final AtomicInteger counter = new AtomicInteger();
+
+    // Raw MockDdlDriver does not implement commit.
+    checkMockDdl(counter, false, new MockDdlDriver());
+
+    // MockDdlDriver implements commit if we have supplied a
+    // prepare-factory to do so.
+    checkMockDdl(counter, true,
+        new MockDdlDriver()
+            .withPrepareFactory(() -> new CountingPrepare(counter)));
+
+    // Raw MockDdlDriver2 does not implement commit.
+    final MockDdlDriver2 driver2 = new MockDdlDriver2(counter);
+    checkMockDdl(counter, false, driver2);
+
+    // MockDdlDriver2 implements commit if we have supplied a
+    // prepare-factory to do so.
+    checkMockDdl(counter, true,
+        driver2.withPrepareFactory(() -> new CountingPrepare(counter)));
+
+    // MockDdlDriver2 implements commit if we override its createPrepareFactory
+    // method. The method is deprecated but override still needs to work.
+    checkMockDdl(counter, true,
+        new MockDdlDriver2(counter) {
+          @SuppressWarnings("deprecation")
+          @Override protected Function0<CalcitePrepare> createPrepareFactory() {
+            return () -> new CountingPrepare(counter);
+          }
+        });
+
+    // MockDdlDriver2 implements commit if we override its createPrepareFactory
+    // method.
+    checkMockDdl(counter, true,
+        new MockDdlDriver2(counter) {
+          @Override public CalcitePrepare createPrepare() {
+            return new CountingPrepare(counter);
+          }
+        });
+  }
+
+  static void checkMockDdl(AtomicInteger counter, boolean hasCommit,
+      Driver driver) {
     try (Connection connection =
              driver.connect("jdbc:calcite:", new Properties());
-        Statement statement = connection.createStatement()) {
-      assertThat(driver.counter, is(0));
-      statement.executeUpdate("COMMIT");
-      assertThat(driver.counter, is(1));
+         Statement statement = connection.createStatement()) {
+      final int original = counter.get();
+      if (hasCommit) {
+        int rowCount = statement.executeUpdate("COMMIT");
+        assertThat(rowCount, is(0));
+        assertThat(counter.get() - original, is(1));
+      } else {
+        assertThrows(SQLException.class,
+            () -> statement.executeUpdate("COMMIT"));
+      }
+    } catch (SQLException e) {
+      throw new AssertionError(e);
     }
   }
 
+  @Test void testCustomValidator() {
+    final Driver driver = new MockDdlDriver().withPrepareFactory(MockPrepareImpl::new);
+    assertThat(driver.createPrepare().getClass(), is(MockPrepareImpl.class));
+  }
+
   /**
    * The example in the README.
    */
@@ -8270,39 +8326,52 @@ public class JdbcTest {
 
   /** Mock driver that can execute a trivial DDL statement. */
   public static class MockDdlDriver extends org.apache.calcite.jdbc.Driver {
-    public int counter;
+  }
 
-    public MockDdlDriver() {
+  /** Mock driver that can execute a trivial DDL statement. */
+  public static class MockDdlDriver2 extends MockDdlDriver {
+    final AtomicInteger counter;
+
+    public MockDdlDriver2(AtomicInteger counter) {
+      this.counter = counter;
     }
+  }
 
-    @Override protected Function0<CalcitePrepare> createPrepareFactory() {
-      return new Function0<CalcitePrepare>() {
-        @Override public CalcitePrepare apply() {
-          return new CalcitePrepareImpl() {
-            @Override protected SqlParser.Config parserConfig() {
-              return super.parserConfig().withParserFactory(stream ->
-                  new SqlParserImpl(stream) {
-                    @Override public SqlNode parseSqlStmtEof() {
-                      return new SqlCall(SqlParserPos.ZERO) {
-                        @Override public SqlOperator getOperator() {
-                          return new SqlSpecialOperator("COMMIT",
-                              SqlKind.COMMIT);
-                        }
-
-                        @Override public List<SqlNode> getOperandList() {
-                          return ImmutableList.of();
-                        }
-                      };
-                    }
-                  });
-            }
+  /** Implementation of {@link CalcitePrepare} that counts how many DDL
+   * statements have been executed. */
+  private static class CountingPrepare extends CalcitePrepareImpl {
+    private final AtomicInteger counter;
+
+    CountingPrepare(AtomicInteger counter) {
+      this.counter = counter;
+    }
 
-            @Override public void executeDdl(Context context, SqlNode node) {
-              ++counter;
+    @Override protected SqlParser.Config parserConfig() {
+      return super.parserConfig().withParserFactory(stream ->
+          new SqlParserImpl(stream) {
+            @Override public SqlNode parseSqlStmtEof() {
+              return new SqlCall(SqlParserPos.ZERO) {
+                @Override public SqlOperator getOperator() {
+                  return new SqlSpecialOperator("COMMIT",
+                      SqlKind.COMMIT);
+                }
+
+                @Override public List<SqlNode> getOperandList() {
+                  return ImmutableList.of();
+                }
+              };
             }
-          };
-        }
-      };
+          });
+    }
+
+    @Override public void executeDdl(Context context, SqlNode node) {
+      counter.incrementAndGet();
+    }
+  }
+
+  /** Dummy subclass of CalcitePrepareImpl. */
+  public static class MockPrepareImpl extends CalcitePrepareImpl {
+    public MockPrepareImpl() {
     }
   }
 
diff --git a/testkit/src/main/java/org/apache/calcite/test/RelOptFixture.java b/testkit/src/main/java/org/apache/calcite/test/RelOptFixture.java
index 32ea34d027..7045d27435 100644
--- a/testkit/src/main/java/org/apache/calcite/test/RelOptFixture.java
+++ b/testkit/src/main/java/org/apache/calcite/test/RelOptFixture.java
@@ -77,7 +77,7 @@ import static java.util.Objects.requireNonNull;
  * set up (for example, the same SQL expression and set of planner rules), it is
  * safe to use the same fixture object as a starting point for both tests.
  */
-class RelOptFixture {
+public class RelOptFixture {
   static final RelOptFixture DEFAULT =
       new RelOptFixture(SqlToRelFixture.TESTER, SqlTestFactory.INSTANCE,
           null, RelSupplier.NONE, null, null,
diff --git a/testkit/src/main/java/org/apache/calcite/test/SqlValidatorFixture.java b/testkit/src/main/java/org/apache/calcite/test/SqlValidatorFixture.java
index e502eab956..834357f024 100644
--- a/testkit/src/main/java/org/apache/calcite/test/SqlValidatorFixture.java
+++ b/testkit/src/main/java/org/apache/calcite/test/SqlValidatorFixture.java
@@ -180,13 +180,13 @@ public class SqlValidatorFixture {
     return withValidatorConfig(c -> c.withLenientOperatorLookup(lenient));
   }
 
-  SqlValidatorFixture withWhole(boolean whole) {
+  public SqlValidatorFixture withWhole(boolean whole) {
     Preconditions.checkArgument(sap.cursor < 0);
     final StringAndPos sap = StringAndPos.of("^" + this.sap.sql + "^");
     return new SqlValidatorFixture(tester, factory, sap, expression, whole);
   }
 
-  SqlValidatorFixture ok() {
+  public SqlValidatorFixture ok() {
     tester.assertExceptionIsThrown(factory, toSql(false), null);
     return this;
   }
@@ -194,7 +194,7 @@ public class SqlValidatorFixture {
   /**
    * Checks that a SQL expression gives a particular error.
    */
-  SqlValidatorFixture fails(String expected) {
+  public SqlValidatorFixture fails(String expected) {
     requireNonNull(expected, "expected");
     tester.assertExceptionIsThrown(factory, toSql(true), expected);
     return this;
@@ -204,7 +204,7 @@ public class SqlValidatorFixture {
    * Checks that a SQL expression fails, giving an {@code expected} error,
    * if {@code b} is true, otherwise succeeds.
    */
-  SqlValidatorFixture failsIf(boolean b, String expected) {
+  public SqlValidatorFixture failsIf(boolean b, String expected) {
     if (b) {
       fails(expected);
     } else {