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 2020/01/16 09:33:28 UTC

[calcite] 02/02: Refactor MaterializationTest to use a fluent API

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

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

commit bfd6d558d3b7b56cc5a8ff13a4e6988f36ace03b
Author: Julian Hyde <jh...@apache.org>
AuthorDate: Tue Jan 14 16:38:10 2020 -0800

    Refactor MaterializationTest to use a fluent API
---
 .../apache/calcite/test/MaterializationTest.java   | 1813 ++++++++++----------
 1 file changed, 946 insertions(+), 867 deletions(-)

diff --git a/core/src/test/java/org/apache/calcite/test/MaterializationTest.java b/core/src/test/java/org/apache/calcite/test/MaterializationTest.java
index 2935359..23e94b5 100644
--- a/core/src/test/java/org/apache/calcite/test/MaterializationTest.java
+++ b/core/src/test/java/org/apache/calcite/test/MaterializationTest.java
@@ -50,6 +50,7 @@ import org.apache.calcite.test.JdbcTest.Event;
 import org.apache.calcite.test.JdbcTest.Location;
 import org.apache.calcite.tools.RuleSet;
 import org.apache.calcite.tools.RuleSets;
+import org.apache.calcite.util.ImmutableBeans;
 import org.apache.calcite.util.JsonBuilder;
 import org.apache.calcite.util.Smalls;
 import org.apache.calcite.util.TryThreadLocal;
@@ -126,6 +127,19 @@ public class MaterializationTest {
       new RexSimplify(rexBuilder, RelOptPredicateList.EMPTY, RexUtil.EXECUTOR)
           .withParanoid(true);
 
+  private static Sql sql() {
+    return ImmutableBeans.create(Sql.class)
+        .withModel(HR_FKUK_MODEL)
+        .withRuleSet(RuleSets.ofList(ImmutableList.of()))
+        .withChecker(CONTAINS_M0);
+  }
+
+  private static Sql sql(String materialize, String query) {
+    return sql()
+        .withMaterialize(materialize)
+        .withQuery(query);
+  }
+
   @Test public void testScan() {
     CalciteAssert.that()
         .withMaterializations(
@@ -172,7 +186,7 @@ public class MaterializationTest {
             + "union all select * from \"emps\" where \"empid\" < 200";
     String mv = "select *, \"empid\" * 2 from (" + union + ")";
     String query = "select * from (" + union + ") where (\"empid\" * 2) > 3";
-    checkMaterialize(mv, query);
+    sql(mv, query).ok();
   }
 
   @Test public void testFilterToProject1() {
@@ -181,7 +195,7 @@ public class MaterializationTest {
             + "from \"emps\" group by \"deptno\"";
     String mv = "select \"c\", \"s\", \"s\" from (" + agg + ")";
     String query = "select * from (" + agg + ") where (\"s\" * 0.8) > 10000";
-    checkNoMaterialize(mv, query, HR_FKUK_MODEL);
+    sql(mv, query).noMat();
   }
 
   @Test public void testFilterQueryOnProjectView() {
@@ -202,55 +216,7 @@ public class MaterializationTest {
 
   /** Checks that a given query can use a materialized view with a given
    * definition. */
-  private void checkMaterialize(String materialize, String query) {
-    checkMaterialize(materialize, query, HR_FKUK_MODEL, CONTAINS_M0,
-        RuleSets.ofList(ImmutableList.of()));
-  }
-
-  /** Checks that a given query can use a materialized view with a given
-   * definition. */
-  private void checkMaterialize(
-      String materialize, String query, boolean onlyBySubstitution) {
-    checkMaterialize(materialize, query, HR_FKUK_MODEL, CONTAINS_M0,
-        RuleSets.ofList(ImmutableList.of()), onlyBySubstitution);
-  }
-
-  /** Checks that a given query can use a materialized view with a given
-   * definition. */
-  private void checkMaterialize(String materialize, String query, String model,
-      Consumer<ResultSet> explainChecker) {
-    checkMaterialize(materialize, query, model, explainChecker,
-        RuleSets.ofList(ImmutableList.of()));
-  }
-
-  /** Checks that a given query can use a materialized view with a given
-   * definition. */
-  private void checkMaterialize(String materialize, String query, String model,
-      Consumer<ResultSet> explainChecker, final RuleSet rules) {
-    checkMaterialize(materialize, query, model, explainChecker, rules, false);
-  }
-
-  /** Checks that a given query can use a materialized view with a given
-   * definition. */
-  private void checkMaterialize(String materialize, String query, String model,
-      Consumer<ResultSet> explainChecker, final RuleSet rules,
-      boolean onlyBySubstitution) {
-    checkThatMaterialize(materialize, query, "m0", false, model, explainChecker,
-        rules, onlyBySubstitution).sameResultWithMaterializationsDisabled();
-  }
-
-  /** Checks that a given query can use a materialized view with a given
-   * definition. */
-  private CalciteAssert.AssertQuery checkThatMaterialize(String materialize,
-      String query, String name, boolean existing, String model,
-      Consumer<ResultSet> explainChecker, final RuleSet rules) {
-    return checkThatMaterialize(materialize, query, name, existing, model,
-        explainChecker, rules, false);
-  }
-
-  /** Checks that a given query can use a materialized view with a given
-   * definition. */
-  private CalciteAssert.AssertQuery checkThatMaterialize(String materialize,
+  static CalciteAssert.AssertQuery checkThatMaterialize_(String materialize,
       String query, String name, boolean existing, String model,
       Consumer<ResultSet> explainChecker, final RuleSet rules,
       boolean onlyBySubstitution) {
@@ -281,13 +247,7 @@ public class MaterializationTest {
 
   /** Checks that a given query CAN NOT use a materialized view with a given
    * definition. */
-  private void checkNoMaterialize(String materialize, String query,
-      String model) {
-    checkNoMaterialize(materialize, query, model, false);
-  }
-  /** Checks that a given query CAN NOT use a materialized view with a given
-   * definition. */
-  private void checkNoMaterialize(String materialize, String query,
+  private static void checkNoMaterialize_(String materialize, String query,
       String model, boolean onlyBySubstitution) {
     try (TryThreadLocal.Memo ignored = Prepare.THREAD_TRIM.push(true)) {
       MaterializationService.setThreadLocal();
@@ -309,72 +269,68 @@ public class MaterializationTest {
   /** Runs the same test as {@link #testFilterQueryOnProjectView()} but more
    * concisely. */
   @Test public void testFilterQueryOnProjectView0() {
-    checkMaterialize(
-        "select \"deptno\", \"empid\" from \"emps\"",
-        "select \"empid\" + 1 as x from \"emps\" where \"deptno\" = 10");
+    sql("select \"deptno\", \"empid\" from \"emps\"",
+        "select \"empid\" + 1 as x from \"emps\" where \"deptno\" = 10")
+        .ok();
   }
 
   /** As {@link #testFilterQueryOnProjectView()} but with extra column in
    * materialized view. */
   @Test public void testFilterQueryOnProjectView1() {
-    checkMaterialize(
-        "select \"deptno\", \"empid\", \"name\" from \"emps\"",
-        "select \"empid\" + 1 as x from \"emps\" where \"deptno\" = 10");
+    sql("select \"deptno\", \"empid\", \"name\" from \"emps\"",
+        "select \"empid\" + 1 as x from \"emps\" where \"deptno\" = 10")
+        .ok();
   }
 
   /** As {@link #testFilterQueryOnProjectView()} but with extra column in both
    * materialized view and query. */
   @Test public void testFilterQueryOnProjectView2() {
-    checkMaterialize(
-        "select \"deptno\", \"empid\", \"name\" from \"emps\"",
-        "select \"empid\" + 1 as x, \"name\" from \"emps\" where \"deptno\" = 10");
+    sql("select \"deptno\", \"empid\", \"name\" from \"emps\"",
+        "select \"empid\" + 1 as x, \"name\" from \"emps\" where \"deptno\" = 10")
+        .ok();
   }
 
   @Test public void testFilterQueryOnProjectView3() {
-    checkMaterialize(
-        "select \"deptno\" - 10 as \"x\", \"empid\" + 1, \"name\" from \"emps\"",
-        "select \"name\" from \"emps\" where \"deptno\" - 10 = 0");
+    sql("select \"deptno\" - 10 as \"x\", \"empid\" + 1, \"name\" from \"emps\"",
+        "select \"name\" from \"emps\" where \"deptno\" - 10 = 0")
+        .ok();
   }
 
   /** As {@link #testFilterQueryOnProjectView3()} but materialized view cannot
    * be used because it does not contain required expression. */
   @Test public void testFilterQueryOnProjectView4() {
-    checkNoMaterialize(
-        "select \"deptno\" - 10 as \"x\", \"empid\" + 1, \"name\" from \"emps\"",
-        "select \"name\" from \"emps\" where \"deptno\" + 10 = 20",
-        HR_FKUK_MODEL);
+    sql("select \"deptno\" - 10 as \"x\", \"empid\" + 1, \"name\" from \"emps\"",
+        "select \"name\" from \"emps\" where \"deptno\" + 10 = 20")
+        .noMat();
   }
 
   /** As {@link #testFilterQueryOnProjectView3()} but also contains an
    * expression column. */
   @Test public void testFilterQueryOnProjectView5() {
-    checkMaterialize(
-        "select \"deptno\" - 10 as \"x\", \"empid\" + 1 as ee, \"name\"\n"
+    sql("select \"deptno\" - 10 as \"x\", \"empid\" + 1 as ee, \"name\"\n"
             + "from \"emps\"",
         "select \"name\", \"empid\" + 1 as e\n"
-            + "from \"emps\" where \"deptno\" - 10 = 2",
-        HR_FKUK_MODEL,
-        CalciteAssert.checkResultContains(
+            + "from \"emps\" where \"deptno\" - 10 = 2")
+        .withResultContains(
             "EnumerableCalc(expr#0..2=[{inputs}], expr#3=[2], "
                 + "expr#4=[=($t0, $t3)], name=[$t2], E=[$t1], $condition=[$t4])\n"
-                + "  EnumerableTableScan(table=[[hr, m0]]"));
+                + "  EnumerableTableScan(table=[[hr, m0]]")
+        .ok();
   }
 
   /** Cannot materialize because "name" is not projected in the MV. */
   @Test public void testFilterQueryOnProjectView6() {
-    checkNoMaterialize(
-        "select \"deptno\" - 10 as \"x\", \"empid\"  from \"emps\"",
-        "select \"name\" from \"emps\" where \"deptno\" - 10 = 0",
-        HR_FKUK_MODEL);
+    sql("select \"deptno\" - 10 as \"x\", \"empid\"  from \"emps\"",
+        "select \"name\" from \"emps\" where \"deptno\" - 10 = 0")
+        .noMat();
   }
 
   /** As {@link #testFilterQueryOnProjectView3()} but also contains an
    * expression column. */
   @Test public void testFilterQueryOnProjectView7() {
-    checkNoMaterialize(
-        "select \"deptno\" - 10 as \"x\", \"empid\" + 1, \"name\" from \"emps\"",
-        "select \"name\", \"empid\" + 2 from \"emps\" where \"deptno\" - 10 = 0",
-        HR_FKUK_MODEL);
+    sql("select \"deptno\" - 10 as \"x\", \"empid\" + 1, \"name\" from \"emps\"",
+        "select \"name\", \"empid\" + 2 from \"emps\" where \"deptno\" - 10 = 0")
+        .noMat();
   }
 
   /** Test case for
@@ -428,137 +384,140 @@ public class MaterializationTest {
 
   @Tag("slow")
   @Test public void testFilterQueryOnFilterView() {
-    checkMaterialize(
-        "select \"deptno\", \"empid\", \"name\" from \"emps\" where \"deptno\" = 10",
-        "select \"empid\" + 1 as x, \"name\" from \"emps\" where \"deptno\" = 10");
+    sql("select \"deptno\", \"empid\", \"name\" from \"emps\" where \"deptno\" = 10",
+        "select \"empid\" + 1 as x, \"name\" from \"emps\" where \"deptno\" = 10")
+        .ok();
   }
 
   /** As {@link #testFilterQueryOnFilterView()} but condition is stronger in
    * query. */
   @Test public void testFilterQueryOnFilterView2() {
-    checkMaterialize(
-        "select \"deptno\", \"empid\", \"name\" from \"emps\" where \"deptno\" = 10",
-        "select \"empid\" + 1 as x, \"name\" from \"emps\" "
-            + "where \"deptno\" = 10 and \"empid\" < 150");
+    final String materialize = "select \"deptno\", \"empid\", \"name\"\n"
+        + "from \"emps\" where \"deptno\" = 10";
+    final String query = "select \"empid\" + 1 as x, \"name\"\n"
+        + "from \"emps\" where \"deptno\" = 10 and \"empid\" < 150";
+    sql(materialize, query).ok();
   }
 
   /** As {@link #testFilterQueryOnFilterView()} but condition is weaker in
    * view. */
   @Test public void testFilterQueryOnFilterView3() {
-    checkMaterialize(
-        "select \"deptno\", \"empid\", \"name\" from \"emps\" "
-            + "where \"deptno\" = 10 or \"deptno\" = 20 or \"empid\" < 160",
-        "select \"empid\" + 1 as x, \"name\" from \"emps\" where \"deptno\" = 10",
-        HR_FKUK_MODEL,
-        CalciteAssert.checkResultContains(
+    final String materialize = "select \"deptno\", \"empid\", \"name\"\n"
+        + "from \"emps\"\n"
+        + "where \"deptno\" = 10 or \"deptno\" = 20 or \"empid\" < 160";
+    final String query = "select \"empid\" + 1 as x, \"name\"\n"
+        + "from \"emps\"\n"
+        + "where \"deptno\" = 10";
+    sql(materialize, query)
+        .withResultContains(
             "EnumerableCalc(expr#0..2=[{inputs}], expr#3=[1], expr#4=[+($t1, $t3)], expr#5=[10], "
                 + "expr#6=[CAST($t0):INTEGER NOT NULL], expr#7=[=($t5, $t6)], X=[$t4], "
                 + "name=[$t2], $condition=[$t7])\n"
-                + "  EnumerableTableScan(table=[[hr, m0]])"));
+                + "  EnumerableTableScan(table=[[hr, m0]])")
+        .ok();
   }
 
   /** As {@link #testFilterQueryOnFilterView()} but condition is stronger in
    * query. */
   @Test public void testFilterQueryOnFilterView4() {
-    checkMaterialize(
-        "select * from \"emps\" where \"deptno\" > 10",
-        "select \"name\" from \"emps\" where \"deptno\" > 30");
+    sql("select * from \"emps\" where \"deptno\" > 10",
+        "select \"name\" from \"emps\" where \"deptno\" > 30")
+        .ok();
   }
 
   /** As {@link #testFilterQueryOnFilterView()} but condition is stronger in
    * query and columns selected are subset of columns in materialized view. */
   @Test public void testFilterQueryOnFilterView5() {
-    checkMaterialize(
-        "select \"name\", \"deptno\" from \"emps\" where \"deptno\" > 10",
-        "select \"name\" from \"emps\" where \"deptno\" > 30");
+    sql("select \"name\", \"deptno\" from \"emps\" where \"deptno\" > 10",
+        "select \"name\" from \"emps\" where \"deptno\" > 30")
+        .ok();
   }
 
   /** As {@link #testFilterQueryOnFilterView()} but condition is stronger in
    * query and columns selected are subset of columns in materialized view. */
   @Test public void testFilterQueryOnFilterView6() {
-    checkMaterialize(
-        "select \"name\", \"deptno\", \"salary\" from \"emps\" "
-            + "where \"salary\" > 2000.5",
-        "select \"name\" from \"emps\" where \"deptno\" > 30 and \"salary\" > 3000");
+    final String materialize = "select \"name\", \"deptno\", \"salary\"\n"
+        + "from \"emps\"\n"
+        + "where \"salary\" > 2000.5";
+    final String query = "select \"name\"\n"
+        + "from \"emps\"\n"
+        + "where \"deptno\" > 30 and \"salary\" > 3000";
+    sql(materialize, query).ok();
   }
 
   /** As {@link #testFilterQueryOnFilterView()} but condition is stronger in
    * query and columns selected are subset of columns in materialized view.
    * Condition here is complex. */
   @Test public void testFilterQueryOnFilterView7() {
-    checkMaterialize(
-        "select * from \"emps\" where "
-            + "((\"salary\" < 1111.9 and \"deptno\" > 10)"
-            + "or (\"empid\" > 400 and \"salary\" > 5000) "
-            + "or \"salary\" > 500)",
-        "select \"name\" from \"emps\" where (\"salary\" > 1000 "
-            + "or (\"deptno\" >= 30 and \"salary\" <= 500))");
+    final String materialize = "select * from \"emps\"\n"
+        + "where ((\"salary\" < 1111.9 and \"deptno\" > 10)\n"
+        + "    or (\"empid\" > 400 and \"salary\" > 5000)\n"
+        + "    or \"salary\" > 500)";
+    final String query = "select \"name\"\n"
+        + "from \"emps\"\n"
+        + "where (\"salary\" > 1000\n"
+        + "  or (\"deptno\" >= 30 and \"salary\" <= 500))";
+    sql(materialize, query).ok();
   }
 
   /** As {@link #testFilterQueryOnFilterView()} but condition is stronger in
    * query. However, columns selected are not present in columns of materialized
    * view, Hence should not use materialized view. */
   @Test public void testFilterQueryOnFilterView8() {
-    checkNoMaterialize(
-        "select \"name\", \"deptno\" from \"emps\" where \"deptno\" > 10",
-        "select \"name\", \"empid\" from \"emps\" where \"deptno\" > 30",
-        HR_FKUK_MODEL);
+    sql("select \"name\", \"deptno\" from \"emps\" where \"deptno\" > 10",
+        "select \"name\", \"empid\" from \"emps\" where \"deptno\" > 30")
+        .noMat();
   }
 
   /** As {@link #testFilterQueryOnFilterView()} but condition is weaker in
    * query. */
   @Test public void testFilterQueryOnFilterView9() {
-    checkNoMaterialize(
-        "select \"name\", \"deptno\" from \"emps\" where \"deptno\" > 10",
-        "select \"name\", \"empid\" from \"emps\" "
-            + "where \"deptno\" > 30 or \"empid\" > 10",
-        HR_FKUK_MODEL);
+    sql("select \"name\", \"deptno\" from \"emps\" where \"deptno\" > 10",
+        "select \"name\", \"empid\" from \"emps\"\n"
+            + "where \"deptno\" > 30 or \"empid\" > 10")
+        .noMat();
   }
 
   /** As {@link #testFilterQueryOnFilterView()} but condition currently
    * has unsupported type being checked on query. */
   @Test public void testFilterQueryOnFilterView10() {
-    checkNoMaterialize(
-        "select \"name\", \"deptno\" from \"emps\" where \"deptno\" > 10 "
+    sql("select \"name\", \"deptno\" from \"emps\" where \"deptno\" > 10 "
             + "and \"name\" = \'calcite\'",
         "select \"name\", \"empid\" from \"emps\" where \"deptno\" > 30 "
-            + "or \"empid\" > 10",
-        HR_FKUK_MODEL);
+            + "or \"empid\" > 10")
+        .noMat();
   }
 
   /** As {@link #testFilterQueryOnFilterView()} but condition is weaker in
    * query and columns selected are subset of columns in materialized view.
    * Condition here is complex. */
   @Test public void testFilterQueryOnFilterView11() {
-    checkNoMaterialize(
-        "select \"name\", \"deptno\" from \"emps\" where "
+    sql("select \"name\", \"deptno\" from \"emps\" where "
             + "(\"salary\" < 1111.9 and \"deptno\" > 10)"
             + "or (\"empid\" > 400 and \"salary\" > 5000)",
-        "select \"name\" from \"emps\" where \"deptno\" > 30 and \"salary\" > 3000",
-        HR_FKUK_MODEL);
+        "select \"name\" from \"emps\" where \"deptno\" > 30 and \"salary\" > 3000")
+        .noMat();
   }
 
   /** As {@link #testFilterQueryOnFilterView()} but condition of
    * query is stronger but is on the column not present in MV (salary).
    */
   @Test public void testFilterQueryOnFilterView12() {
-    checkNoMaterialize(
-        "select \"name\", \"deptno\" from \"emps\" where \"salary\" > 2000.5",
-        "select \"name\" from \"emps\" where \"deptno\" > 30 and \"salary\" > 3000",
-        HR_FKUK_MODEL);
+    sql("select \"name\", \"deptno\" from \"emps\" where \"salary\" > 2000.5",
+        "select \"name\" from \"emps\" where \"deptno\" > 30 and \"salary\" > 3000")
+        .noMat();
   }
 
   /** As {@link #testFilterQueryOnFilterView()} but condition is weaker in
    * query and columns selected are subset of columns in materialized view.
    * Condition here is complex. */
   @Test public void testFilterQueryOnFilterView13() {
-    checkNoMaterialize(
-        "select * from \"emps\" where "
+    sql("select * from \"emps\" where "
             + "(\"salary\" < 1111.9 and \"deptno\" > 10)"
             + "or (\"empid\" > 400 and \"salary\" > 5000)",
         "select \"name\" from \"emps\" where \"salary\" > 1000 "
-            + "or (\"deptno\" > 30 and \"salary\" > 3000)",
-        HR_FKUK_MODEL);
+            + "or (\"deptno\" > 30 and \"salary\" > 3000)")
+        .noMat();
   }
 
   /** As {@link #testFilterQueryOnFilterView7()} but columns in materialized
@@ -571,41 +530,41 @@ public class MaterializationTest {
         + "((\"salary\" < 1111.9 and \"deptno\" > 10)"
         + "or (\"empid\" > 400 and \"salary\" > 5000) "
         + "or \"salary\" > 500)";
-    checkMaterialize(m, q);
+    sql(m, q).ok();
   }
 
   /** As {@link #testFilterQueryOnFilterView13()} but using alias
    * and condition of query is stronger. */
   @Test public void testAlias() {
-    checkMaterialize(
-        "select * from \"emps\" as em where "
+    sql("select * from \"emps\" as em where "
             + "(em.\"salary\" < 1111.9 and em.\"deptno\" > 10)"
             + "or (em.\"empid\" > 400 and em.\"salary\" > 5000)",
         "select \"name\" as n from \"emps\" as e where "
-            + "(e.\"empid\" > 500 and e.\"salary\" > 6000)");
+            + "(e.\"empid\" > 500 and e.\"salary\" > 6000)")
+        .ok();
   }
 
   /** Aggregation query at same level of aggregation as aggregation
    * materialization. */
   @Test public void testAggregate0() {
-    checkMaterialize(
-        "select count(*) as c from \"emps\" group by \"empid\"",
-        "select count(*) + 1 as c from \"emps\" group by \"empid\"");
+    sql("select count(*) as c from \"emps\" group by \"empid\"",
+        "select count(*) + 1 as c from \"emps\" group by \"empid\"")
+        .ok();
   }
 
   /**
    * Aggregation query at same level of aggregation as aggregation
    * materialization but with different row types. */
   @Test public void testAggregate1() {
-    checkMaterialize(
-        "select count(*) as c0 from \"emps\" group by \"empid\"",
-        "select count(*) as c1 from \"emps\" group by \"empid\"");
+    sql("select count(*) as c0 from \"emps\" group by \"empid\"",
+        "select count(*) as c1 from \"emps\" group by \"empid\"")
+        .ok();
   }
 
   @Test public void testAggregate2() {
-    checkMaterialize(
-        "select \"deptno\", count(*) as c, sum(\"empid\") as s from \"emps\" group by \"deptno\"",
-        "select count(*) + 1 as c, \"deptno\" from \"emps\" group by \"deptno\"");
+    sql("select \"deptno\", count(*) as c, sum(\"empid\") as s from \"emps\" group by \"deptno\"",
+        "select count(*) + 1 as c, \"deptno\" from \"emps\" group by \"deptno\"")
+        .ok();
   }
 
   @Test public void testAggregate3() {
@@ -625,7 +584,7 @@ public class MaterializationTest {
             + "  (select \"deptno\", \"salary\", 100 as \"k\"\n"
             + "  from " + deduplicated + ")\n"
             + "group by \"deptno\"";
-    checkMaterialize(mv, query);
+    sql(mv, query).ok();
   }
 
   @Test public void testAggregate4() {
@@ -638,7 +597,7 @@ public class MaterializationTest {
         + "from \"emps\"\n"
         + "where \"commission\" = 100\n"
         + "group by \"deptno\"";
-    checkMaterialize(mv, query, true);
+    sql(mv, query).withOnlyBySubstitution(true).ok();
   }
 
   @Test public void testAggregate5() {
@@ -651,7 +610,7 @@ public class MaterializationTest {
         + "from \"emps\"\n"
         + "where \"commission\" * (\"deptno\" + \"commission\") = 100\n"
         + "group by \"commission\"";
-    checkMaterialize(mv, query, true);
+    sql(mv, query).withOnlyBySubstitution(true).ok();
   }
 
   /**
@@ -672,7 +631,7 @@ public class MaterializationTest {
         + "where \"salary\" > 1000\n"
         + "group by \"deptno\")\n"
         + "where \"sum_salary\" > 10";
-    checkNoMaterialize(mv, query, HR_FKUK_MODEL, true);
+    sql(mv, query).withOnlyBySubstitution(true).noMat();
   }
 
   @Test public void testAggregate7() {
@@ -710,7 +669,7 @@ public class MaterializationTest {
         + "from \"emps\"\n"
         + "group by \"deptno\")\n"
         + "where \"sum_salary\" > 10";
-    checkMaterialize(mv, query, true);
+    sql(mv, query).withOnlyBySubstitution(true).ok();
   }
 
   /**
@@ -731,7 +690,7 @@ public class MaterializationTest {
         + "where \"deptno\" >=20\n"
         + "group by \"deptno\")\n"
         + "where \"sum_salary\" > 10";
-    checkMaterialize(mv, query, true);
+    sql(mv, query).withOnlyBySubstitution(true).ok();
   }
 
   /**
@@ -753,83 +712,106 @@ public class MaterializationTest {
         + "where \"deptno\" >= 20\n"
         + "group by \"deptno\")\n"
         + "where \"sum_salary\" > 20";
-    checkMaterialize(mv, query, true);
+    sql(mv, query).withOnlyBySubstitution(true).ok();
   }
 
   /** Aggregation query at same level of aggregation as aggregation
    * materialization with grouping sets. */
   @Test public void testAggregateGroupSets1() {
-    checkMaterialize(
-        "select \"empid\", \"deptno\", count(*) as c, sum(\"salary\") as s from \"emps\" group by cube(\"empid\",\"deptno\")",
-        "select count(*) + 1 as c, \"deptno\" from \"emps\" group by cube(\"empid\",\"deptno\")");
+    final String materialize = ""
+        + "select \"empid\", \"deptno\", count(*) as c, sum(\"salary\") as s\n"
+        + "from \"emps\"\n"
+        + "group by cube(\"empid\",\"deptno\")";
+    final String query = "select count(*) + 1 as c, \"deptno\"\n"
+        + "from \"emps\"\n"
+        + "group by cube(\"empid\",\"deptno\")";
+    sql(materialize, query).ok();
   }
 
   /** Aggregation query with different grouping sets, should not
    * do materialization. */
   @Test public void testAggregateGroupSets2() {
-    checkNoMaterialize(
-        "select \"empid\", \"deptno\", count(*) as c, sum(\"salary\") as s from \"emps\" group by cube(\"empid\",\"deptno\")",
-        "select count(*) + 1 as c, \"deptno\" from \"emps\" group by rollup(\"empid\",\"deptno\")",
-        HR_FKUK_MODEL);
+    final String materialize = "select \"empid\", \"deptno\",\n"
+        + "  count(*) as c, sum(\"salary\") as s\n"
+        + "from \"emps\"\n"
+        + "group by cube(\"empid\",\"deptno\")";
+    final String query = "select count(*) + 1 as c, \"deptno\"\n"
+        + "from \"emps\"\n"
+        + "group by rollup(\"empid\",\"deptno\")";
+    sql(materialize, query).noMat();
   }
 
   /** Aggregation query at coarser level of aggregation than aggregation
    * materialization. Requires an additional aggregate to roll up. Note that
    * COUNT is rolled up using SUM0. */
   @Test public void testAggregateRollUp() {
-    checkMaterialize(
-        "select \"empid\", \"deptno\", count(*) as c, sum(\"empid\") as s from \"emps\" "
-            + "group by \"empid\", \"deptno\"",
-        "select count(*) + 1 as c, \"deptno\" from \"emps\" group by \"deptno\"",
-        HR_FKUK_MODEL,
-        CalciteAssert.checkResultContains(
+    final String materialize = "select \"empid\", \"deptno\", count(*) as c,\n"
+        + "  sum(\"empid\") as s\n"
+        + "from \"emps\"\n"
+        + "group by \"empid\", \"deptno\"";
+    final String query = "select count(*) + 1 as c, \"deptno\"\n"
+        + "from \"emps\"\n"
+        + "group by \"deptno\"";
+    sql(materialize, query)
+        .withResultContains(
             "EnumerableCalc(expr#0..1=[{inputs}], expr#2=[1], "
                 + "expr#3=[+($t1, $t2)], C=[$t3], deptno=[$t0])\n"
                 + "  EnumerableAggregate(group=[{1}], agg#0=[$SUM0($2)])\n"
-                + "    EnumerableTableScan(table=[[hr, m0]])"));
+                + "    EnumerableTableScan(table=[[hr, m0]])")
+        .ok();
   }
 
   /** Aggregation query with groupSets at coarser level of aggregation than
    * aggregation materialization. Requires an additional aggregate to roll up.
    * Note that COUNT is rolled up using SUM0. */
   @Test public void testAggregateGroupSetsRollUp() {
-    checkMaterialize(
-        "select \"empid\", \"deptno\", count(*) as c, sum(\"salary\") as s from \"emps\" "
-            + "group by \"empid\", \"deptno\"",
-        "select count(*) + 1 as c,  \"deptno\" from \"emps\" group by cube(\"empid\",\"deptno\")",
-        HR_FKUK_MODEL,
-        CalciteAssert.checkResultContains(
-            "EnumerableCalc(expr#0..2=[{inputs}], expr#3=[1], "
-                + "expr#4=[+($t2, $t3)], C=[$t4], deptno=[$t1])\n"
-                + "  EnumerableAggregate(group=[{0, 1}], groups=[[{0, 1}, {0}, {1}, {}]], agg#0=[$SUM0($2)])\n"
-                + "    EnumerableTableScan(table=[[hr, m0]])"));
+    final String materialize = "select \"empid\", \"deptno\", count(*) as c,\n"
+        + "  sum(\"salary\") as s\n"
+        + "from \"emps\"\n"
+        + "group by \"empid\", \"deptno\"";
+    final String query = "select count(*) + 1 as c,  \"deptno\"\n"
+        + "from \"emps\"\n"
+        + "group by cube(\"empid\",\"deptno\")";
+    final String expected = "EnumerableCalc(expr#0..2=[{inputs}], expr#3=[1], "
+        + "expr#4=[+($t2, $t3)], C=[$t4], deptno=[$t1])\n"
+        + "  EnumerableAggregate(group=[{0, 1}], "
+        + "groups=[[{0, 1}, {0}, {1}, {}]], agg#0=[$SUM0($2)])\n"
+        + "    EnumerableTableScan(table=[[hr, m0]])";
+    sql(materialize, query).withResultContains(expected).ok();
   }
 
   @Test public void testAggregateGroupSetsRollUp2() {
-    checkMaterialize(
-        "select \"empid\", \"deptno\", count(*) as c, sum(\"empid\") as s from \"emps\" "
-            + "group by \"empid\", \"deptno\"",
-        "select count(*) + 1 as c,  \"deptno\" from \"emps\" group by cube(\"empid\",\"deptno\")",
-        HR_FKUK_MODEL,
-        CalciteAssert.checkResultContains(
-            "EnumerableCalc(expr#0..2=[{inputs}], expr#3=[1], "
-                + "expr#4=[+($t2, $t3)], C=[$t4], deptno=[$t1])\n"
-                + "  EnumerableAggregate(group=[{0, 1}], groups=[[{0, 1}, {0}, {1}, {}]], agg#0=[$SUM0($2)])\n"
-                + "    EnumerableTableScan(table=[[hr, m0]])"));
+    final String materialize = "select \"empid\", \"deptno\", count(*) as c,\n"
+        + "  sum(\"empid\") as s\n"
+        + "from \"emps\"\n"
+        + "group by \"empid\", \"deptno\"";
+    final String query = "select count(*) + 1 as c,  \"deptno\"\n"
+        + "from \"emps\"\n"
+        + "group by cube(\"empid\",\"deptno\")";
+    final String expected = "EnumerableCalc(expr#0..2=[{inputs}], expr#3=[1], "
+        + "expr#4=[+($t2, $t3)], C=[$t4], deptno=[$t1])\n"
+        + "  EnumerableAggregate(group=[{0, 1}], "
+        + "groups=[[{0, 1}, {0}, {1}, {}]], agg#0=[$SUM0($2)])\n"
+        + "    EnumerableTableScan(table=[[hr, m0]])";
+    sql(materialize, query).withResultContains(expected).ok();
   }
 
   /** Aggregation materialization with a project. */
   @Test public void testAggregateProject() {
     // Note that materialization does not start with the GROUP BY columns.
     // Not a smart way to design a materialization, but people may do it.
-    checkMaterialize(
-        "select \"deptno\", count(*) as c, \"empid\" + 2, sum(\"empid\") as s from \"emps\" group by \"empid\", \"deptno\"",
-        "select count(*) + 1 as c, \"deptno\" from \"emps\" group by \"deptno\"",
-        HR_FKUK_MODEL,
-        CalciteAssert.checkResultContains(
-            "EnumerableCalc(expr#0..1=[{inputs}], expr#2=[1], expr#3=[+($t1, $t2)], C=[$t3], deptno=[$t0])\n"
-                + "  EnumerableAggregate(group=[{0}], agg#0=[$SUM0($1)])\n"
-                + "    EnumerableTableScan(table=[[hr, m0]])"));
+    final String materialize = "select \"deptno\", count(*) as c,\n"
+        + "  \"empid\" + 2, sum(\"empid\") as s\n"
+        + "from \"emps\"\n"
+        + "group by \"empid\", \"deptno\"";
+    final String query = "select count(*) + 1 as c, \"deptno\"\n"
+        + "from \"emps\"\n"
+        + "group by \"deptno\"";
+    final String expected = "EnumerableCalc(expr#0..1=[{inputs}], expr#2=[1], "
+        + "expr#3=[+($t1, $t2)], C=[$t3], deptno=[$t0])\n"
+        + "  EnumerableAggregate(group=[{0}], agg#0=[$SUM0($1)])\n"
+        + "    EnumerableTableScan(table=[[hr, m0]])";
+    sql(materialize, query).withResultContains(expected).ok();
   }
 
   /** Test case for
@@ -837,49 +819,61 @@ public class MaterializationTest {
    * AggregateOnProjectToAggregateUnifyRule ignores Project incorrectly when its
    * Mapping breaks ordering</a>. */
   @Test public void testAggregateOnProject1() {
-    checkMaterialize(
-        "select \"empid\", \"deptno\", count(*) as c, sum(\"empid\") as s from \"emps\" "
-            + "group by \"empid\", \"deptno\"",
-        "select count(*) + 1 as c, \"deptno\" from \"emps\" group by \"deptno\", \"empid\"");
+    final String materialize = "select \"empid\", \"deptno\", count(*) as c,\n"
+        + "  sum(\"empid\") as s\n"
+        + "from \"emps\"\n"
+        + "group by \"empid\", \"deptno\"";
+    final String query = "select count(*) + 1 as c, \"deptno\"\n"
+        + "from \"emps\"\n"
+        + "group by \"deptno\", \"empid\"";
+    sql(materialize, query).ok();
   }
 
   @Test public void testAggregateOnProject2() {
-    checkMaterialize(
-        "select \"empid\", \"deptno\", count(*) as c, sum(\"salary\") as s from \"emps\" "
-            + "group by \"empid\", \"deptno\"",
-        "select count(*) + 1 as c,  \"deptno\" from \"emps\" group by cube(\"deptno\", \"empid\")",
-        HR_FKUK_MODEL,
-        CalciteAssert.checkResultContains(
-            "EnumerableCalc(expr#0..2=[{inputs}], expr#3=[1], "
-                + "expr#4=[+($t2, $t3)], C=[$t4], deptno=[$t1])\n"
-                + "  EnumerableAggregate(group=[{0, 1}], groups=[[{0, 1}, {0}, {1}, {}]], agg#0=[$SUM0($2)])\n"
-                + "    EnumerableTableScan(table=[[hr, m0]])"));
+    final String materialize = "select \"empid\", \"deptno\", count(*) as c,\n"
+        + "  sum(\"salary\") as s\n"
+        + "from \"emps\"\n"
+        + "group by \"empid\", \"deptno\"";
+    final String query = "select count(*) + 1 as c,  \"deptno\"\n"
+        + "from \"emps\"\n"
+        + "group by cube(\"deptno\", \"empid\")";
+    final String expected = "EnumerableCalc(expr#0..2=[{inputs}], expr#3=[1], "
+        + "expr#4=[+($t2, $t3)], C=[$t4], deptno=[$t1])\n"
+        + "  EnumerableAggregate(group=[{0, 1}], "
+        + "groups=[[{0, 1}, {0}, {1}, {}]], agg#0=[$SUM0($2)])\n"
+        + "    EnumerableTableScan(table=[[hr, m0]])";
+    sql(materialize, query).withResultContains(expected).ok();
   }
 
   @Test public void testAggregateOnProject3() {
-    checkMaterialize(
-        "select \"empid\", \"deptno\", count(*) as c, sum(\"salary\") as s from \"emps\" "
-            + "group by \"empid\", \"deptno\"",
-        "select count(*) + 1 as c,  \"deptno\" from \"emps\" group by rollup(\"deptno\", \"empid\")",
-        HR_FKUK_MODEL,
-        CalciteAssert.checkResultContains(
-            "EnumerableCalc(expr#0..2=[{inputs}], expr#3=[1], "
-                + "expr#4=[+($t2, $t3)], C=[$t4], deptno=[$t1])\n"
-                + "  EnumerableAggregate(group=[{0, 1}], groups=[[{0, 1}, {1}, {}]], agg#0=[$SUM0($2)])\n"
-                + "    EnumerableTableScan(table=[[hr, m0]])"));
+    final String materialize = "select \"empid\", \"deptno\", count(*) as c,\n"
+        + "  sum(\"salary\") as s\n"
+        + "from \"emps\"\n"
+        + "group by \"empid\", \"deptno\"";
+    final String query = "select count(*) + 1 as c, \"deptno\"\n"
+        + "from \"emps\"\n"
+        + "group by rollup(\"deptno\", \"empid\")";
+    final String expected = "EnumerableCalc(expr#0..2=[{inputs}], expr#3=[1], "
+        + "expr#4=[+($t2, $t3)], C=[$t4], deptno=[$t1])\n"
+        + "  EnumerableAggregate(group=[{0, 1}], groups=[[{0, 1}, {1}, {}]], agg#0=[$SUM0($2)])\n"
+        + "    EnumerableTableScan(table=[[hr, m0]])";
+    sql(materialize, query).withResultContains(expected).ok();
   }
 
   @Test public void testAggregateOnProject4() {
-    checkMaterialize(
-        "select \"salary\", \"empid\", \"deptno\", count(*) as c, sum(\"commission\") as s from \"emps\" "
-            + "group by \"salary\", \"empid\", \"deptno\"",
-        "select count(*) + 1 as c,  \"deptno\" from \"emps\" group by rollup(\"empid\", \"deptno\", \"salary\")",
-        HR_FKUK_MODEL,
-        CalciteAssert.checkResultContains(
-            "EnumerableCalc(expr#0..3=[{inputs}], expr#4=[1], "
-                + "expr#5=[+($t3, $t4)], C=[$t5], deptno=[$t2])\n"
-                + "  EnumerableAggregate(group=[{0, 1, 2}], groups=[[{0, 1, 2}, {1, 2}, {1}, {}]], agg#0=[$SUM0($3)])\n"
-                + "    EnumerableTableScan(table=[[hr, m0]])"));
+    final String materialize = "select \"salary\", \"empid\", \"deptno\",\n"
+        + "  count(*) as c, sum(\"commission\") as s\n"
+        + "from \"emps\"\n"
+        + "group by \"salary\", \"empid\", \"deptno\"";
+    final String query = "select count(*) + 1 as c,  \"deptno\"\n"
+        + "from \"emps\"\n"
+        + "group by rollup(\"empid\", \"deptno\", \"salary\")";
+    final String expected = "EnumerableCalc(expr#0..3=[{inputs}], expr#4=[1], "
+        + "expr#5=[+($t3, $t4)], C=[$t5], deptno=[$t2])\n"
+        + "  EnumerableAggregate(group=[{0, 1, 2}], "
+        + "groups=[[{0, 1, 2}, {1, 2}, {1}, {}]], agg#0=[$SUM0($3)])\n"
+        + "    EnumerableTableScan(table=[[hr, m0]])";
+    sql(materialize, query).withResultContains(expected).ok();
   }
 
   /** Test case for
@@ -887,15 +881,14 @@ public class MaterializationTest {
    * AggregateOnCalcToAggregateUnifyRule ignores Project incorrectly when
    * there's missing grouping or mapping breaks ordering</a>. */
   @Test public void testAggregateOnProject5() {
-    checkMaterialize(
-        "select \"empid\", \"deptno\", \"name\", count(*) from \"emps\"\n"
+    sql("select \"empid\", \"deptno\", \"name\", count(*) from \"emps\"\n"
             + "group by \"empid\", \"deptno\", \"name\"",
-        "select \"name\", \"empid\", count(*) from \"emps\" group by \"name\", \"empid\"",
-        HR_FKUK_MODEL,
-        CalciteAssert.checkResultContains(""
+        "select \"name\", \"empid\", count(*) from \"emps\" group by \"name\", \"empid\"")
+        .withResultContains(""
             + "EnumerableCalc(expr#0..2=[{inputs}], name=[$t1], empid=[$t0], EXPR$2=[$t2])\n"
             + "  EnumerableAggregate(group=[{0, 2}], EXPR$2=[$SUM0($3)])\n"
-            + "    EnumerableTableScan(table=[[hr, m0]])"));
+            + "    EnumerableTableScan(table=[[hr, m0]])")
+        .ok();
   }
 
   @Test public void testAggregateOnProjectAndFilter() {
@@ -908,7 +901,7 @@ public class MaterializationTest {
         + "from \"emps\"\n"
         + "where \"deptno\" = 10\n"
         + "group by \"deptno\"";
-    checkMaterialize(mv, query, true);
+    sql(mv, query).withOnlyBySubstitution(true).ok();
   }
 
   @Test public void testProjectOnProject() {
@@ -920,16 +913,20 @@ public class MaterializationTest {
         + "select \"deptno\", sum(\"salary\") + 2\n"
         + "from \"emps\"\n"
         + "group by \"deptno\"";
-    checkMaterialize(mv, query, true);
+    sql(mv, query).withOnlyBySubstitution(true).ok();
   }
 
   @Test public void testPermutationError() {
-    checkMaterialize(
-        "select min(\"salary\"), count(*), max(\"salary\"), sum(\"salary\"), \"empid\" "
-            + "from \"emps\" group by \"empid\"",
-        "select count(*), \"empid\" from \"emps\" group by \"empid\"",
-        HR_FKUK_MODEL,
-        CalciteAssert.checkResultContains("EnumerableTableScan(table=[[hr, m0]])"));
+    final String materialize = "select min(\"salary\"), count(*),\n"
+        + "  max(\"salary\"), sum(\"salary\"), \"empid\"\n"
+        + "from \"emps\"\n"
+        + "group by \"empid\"";
+    final String query = "select count(*), \"empid\"\n"
+        + "from \"emps\"\n"
+        + "group by \"empid\"";
+    sql(materialize, query)
+        .withResultContains("EnumerableTableScan(table=[[hr, m0]])")
+        .ok();
   }
 
   @Test public void testJoinOnLeftProjectToJoin() {
@@ -953,7 +950,7 @@ public class MaterializationTest {
         + "  from \"depts\"\n"
         + "  group by \"deptno\") \"B\"\n"
         + "  on \"A\".\"deptno\" = \"B\".\"deptno\"";
-    checkMaterialize(mv, query, true);
+    sql(mv, query).withOnlyBySubstitution(true).ok();
   }
 
   @Test public void testJoinOnRightProjectToJoin() {
@@ -977,7 +974,7 @@ public class MaterializationTest {
         + "  from \"depts\"\n"
         + "  group by \"deptno\") \"B\"\n"
         + "  on \"A\".\"deptno\" = \"B\".\"deptno\"";
-    checkMaterialize(mv, query, true);
+    sql(mv, query).withOnlyBySubstitution(true).ok();
   }
 
   @Test public void testJoinOnProjectsToJoin() {
@@ -1001,7 +998,7 @@ public class MaterializationTest {
         + "  from \"depts\"\n"
         + "  group by \"deptno\") \"B\"\n"
         + "  on \"A\".\"deptno\" = \"B\".\"deptno\"";
-    checkMaterialize(mv, query, true);
+    sql(mv, query).withOnlyBySubstitution(true).ok();
   }
 
   @Test public void testJoinOnCalcToJoin0() {
@@ -1014,7 +1011,7 @@ public class MaterializationTest {
         + " (select \"empid\", \"deptno\" from \"emps\" where \"deptno\" > 10) A"
         + " join \"depts\"\n"
         + "on \"A\".\"deptno\" = \"depts\".\"deptno\"";
-    checkMaterialize(mv, query, true);
+    sql(mv, query).withOnlyBySubstitution(true).ok();
   }
 
   @Test public void testJoinOnCalcToJoin1() {
@@ -1027,7 +1024,7 @@ public class MaterializationTest {
         + "\"emps\" join\n"
         + "(select \"deptno\" from \"depts\" where \"deptno\" > 10) B\n"
         + "on \"emps\".\"deptno\" = \"B\".\"deptno\"";
-    checkMaterialize(mv, query, true);
+    sql(mv, query).withOnlyBySubstitution(true).ok();
   }
 
   @Test public void testJoinOnCalcToJoin2() {
@@ -1041,7 +1038,7 @@ public class MaterializationTest {
         + "join\n"
         + "(select \"deptno\" from \"depts\" where \"deptno\" > 10) B\n"
         + "on \"A\".\"deptno\" = \"B\".\"deptno\"";
-    checkMaterialize(mv, query, true);
+    sql(mv, query).withOnlyBySubstitution(true).ok();
   }
 
   @Test public void testJoinOnCalcToJoin3() {
@@ -1056,45 +1053,50 @@ public class MaterializationTest {
         + "(select \"deptno\" from \"depts\" where \"deptno\" > 10) B\n"
         + "on \"A\".\"deptno\" = \"B\".\"deptno\"";
     // Match failure because join condition references non-mapping projects.
-    checkNoMaterialize(mv, query, HR_FKUK_MODEL, true);
+    sql(mv, query).withOnlyBySubstitution(true).noMat();
   }
 
   @Test public void testJoinOnCalcToJoin4() {
-    String mv = ""
-        + "select \"emps\".\"empid\", \"emps\".\"deptno\", \"depts\".\"deptno\" from\n"
-        + "\"emps\" join \"depts\"\n"
-        + "on \"emps\".\"deptno\" = \"depts\".\"deptno\"";
-    String query = ""
-        + "select * from\n"
+    String mv = "select \"emps\".\"empid\", \"emps\".\"deptno\",\n"
+        + "  \"depts\".\"deptno\"\n"
+        + "from \"emps\"\n"
+        + "join \"depts\" on \"emps\".\"deptno\" = \"depts\".\"deptno\"";
+    String query = "select *\n"
+        + "from\n"
         + "(select \"empid\", \"deptno\" from \"emps\" where \"empid\" is not null) A\n"
         + "full join\n"
         + "(select \"deptno\" from \"depts\" where \"deptno\" is not null) B\n"
         + "on \"A\".\"deptno\" = \"B\".\"deptno\"";
     // Match failure because of outer join type but filtering condition in Calc is not empty.
-    checkNoMaterialize(mv, query, HR_FKUK_MODEL, true);
+    sql(mv, query).withOnlyBySubstitution(true).noMat();
   }
 
   @Tag("slow")
   @Test public void testSwapJoin() {
-    checkMaterialize(
-        "select count(*) as c from \"foodmart\".\"sales_fact_1997\" as s join \"foodmart\".\"time_by_day\" as t on s.\"time_id\" = t.\"time_id\"",
-        "select count(*) as c from \"foodmart\".\"time_by_day\" as t join \"foodmart\".\"sales_fact_1997\" as s on t.\"time_id\" = s.\"time_id\"",
-        JdbcTest.FOODMART_MODEL,
-        CalciteAssert.checkResultContains("EnumerableTableScan(table=[[mat, m0]])"));
+    final String materialize = "select count(*) as c\n"
+        + "from \"foodmart\".\"sales_fact_1997\" as s\n"
+        + "join \"foodmart\".\"time_by_day\" as t on s.\"time_id\" = t.\"time_id\"";
+    final String query = "select count(*) as c\n"
+        + "from \"foodmart\".\"time_by_day\" as t\n"
+        + "join \"foodmart\".\"sales_fact_1997\" as s on t.\"time_id\" = s.\"time_id\"";
+    sql(materialize, query)
+        .withModel(JdbcTest.FOODMART_MODEL)
+        .withResultContains("EnumerableTableScan(table=[[mat, m0]])")
+        .ok();
   }
 
   @Disabled
   @Test public void testOrderByQueryOnProjectView() {
-    checkMaterialize(
-        "select \"deptno\", \"empid\" from \"emps\"",
-        "select \"empid\" from \"emps\" order by \"deptno\"");
+    sql("select \"deptno\", \"empid\" from \"emps\"",
+        "select \"empid\" from \"emps\" order by \"deptno\"")
+        .ok();
   }
 
   @Disabled
   @Test public void testOrderByQueryOnOrderByView() {
-    checkMaterialize(
-        "select \"deptno\", \"empid\" from \"emps\" order by \"deptno\"",
-        "select \"empid\" from \"emps\" order by \"deptno\"");
+    sql("select \"deptno\", \"empid\" from \"emps\" order by \"deptno\"",
+        "select \"empid\" from \"emps\" order by \"deptno\"")
+        .ok();
   }
 
   @Disabled
@@ -1434,7 +1436,7 @@ public class MaterializationTest {
    */
   @Disabled
   @Test public void testFilterGroupQueryOnStar() {
-    checkMaterialize("select p.\"product_name\", t.\"the_year\",\n"
+    sql("select p.\"product_name\", t.\"the_year\",\n"
             + "  sum(f.\"unit_sales\") as \"sum_unit_sales\", count(*) as \"c\"\n"
             + "from \"foodmart\".\"sales_fact_1997\" as f\n"
             + "join (\n"
@@ -1458,9 +1460,9 @@ public class MaterializationTest {
             + "where t.\"the_year\" = 1997\n"
             + "and t.\"time_id\" = f.\"time_id\"\n"
             + "group by t.\"the_year\",\n"
-            + " t.\"the_month\"\n",
-        JdbcTest.FOODMART_MODEL,
-        CONTAINS_M0);
+            + " t.\"the_month\"\n")
+        .withModel(JdbcTest.FOODMART_MODEL)
+        .ok();
   }
 
   /** Simpler than {@link #testFilterGroupQueryOnStar()}, tests a query on a
@@ -1472,9 +1474,9 @@ public class MaterializationTest {
         + "join \"foodmart\".\"time_by_day\" as t on f.\"time_id\" = t.\"time_id\"\n"
         + "join \"foodmart\".\"product\" as p on f.\"product_id\" = p.\"product_id\"\n"
         + "join \"foodmart\".\"product_class\" as pc on p.\"product_class_id\" = pc.\"product_class_id\"\n";
-    checkMaterialize(
-        q, q + "where t.\"month_of_year\" = 10", JdbcTest.FOODMART_MODEL,
-        CONTAINS_M0);
+    sql(q, q + "where t.\"month_of_year\" = 10")
+        .withModel(JdbcTest.FOODMART_MODEL)
+        .ok();
   }
 
   /** A materialization that is a join of a union cannot at present be converted
@@ -1485,14 +1487,14 @@ public class MaterializationTest {
     String q = "select *\n"
         + "from (select * from \"emps\" union all select * from \"emps\")\n"
         + "join \"depts\" using (\"deptno\")";
-    checkNoMaterialize(q, q, HR_FKUK_MODEL);
+    sql(q, q).noMat();
   }
 
   @Test public void testJoinMaterialization() {
     String q = "select *\n"
         + "from (select * from \"emps\" where \"empid\" < 300)\n"
         + "join \"depts\" using (\"deptno\")";
-    checkMaterialize("select * from \"emps\" where \"empid\" < 500", q);
+    sql("select * from \"emps\" where \"empid\" < 500", q).ok();
   }
 
   /** Test case for
@@ -1505,7 +1507,7 @@ public class MaterializationTest {
         + "join \"depts\" using (\"deptno\")";
     final String m = "select \"deptno\", \"empid\", \"name\",\n"
         + "\"salary\", \"commission\" from \"emps\"";
-    checkMaterialize(m, q);
+    sql(m, q).ok();
   }
 
   @Test public void testJoinMaterialization3() {
@@ -1513,548 +1515,589 @@ public class MaterializationTest {
         + "join \"depts\" using (\"deptno\") where \"empid\" = 1";
     final String m = "select \"empid\" \"deptno\" from \"emps\"\n"
         + "join \"depts\" using (\"deptno\")";
-    checkMaterialize(m, q);
+    sql(m, q).ok();
   }
 
   @Test public void testUnionAll() {
     String q = "select * from \"emps\" where \"empid\" > 300\n"
         + "union all select * from \"emps\" where \"empid\" < 200";
     String m = "select * from \"emps\" where \"empid\" < 500";
-    checkMaterialize(m, q, HR_FKUK_MODEL,
+    sql(m, q).withChecker(
         CalciteAssert.checkResultContains(
-            "EnumerableTableScan(table=[[hr, m0]])", 1));
+            "EnumerableTableScan(table=[[hr, m0]])", 1))
+        .ok();
   }
 
   @Test public void testAggregateMaterializationNoAggregateFuncs1() {
-    checkMaterialize(
-        "select \"empid\", \"deptno\" from \"emps\" group by \"empid\", \"deptno\"",
-        "select \"empid\", \"deptno\" from \"emps\" group by \"empid\", \"deptno\"",
-        HR_FKUK_MODEL,
-        CalciteAssert.checkResultContains(
-            "EnumerableTableScan(table=[[hr, m0]])"));
+    sql("select \"empid\", \"deptno\" from \"emps\" group by \"empid\", \"deptno\"",
+        "select \"empid\", \"deptno\" from \"emps\" group by \"empid\", \"deptno\"")
+        .withResultContains(
+            "EnumerableTableScan(table=[[hr, m0]])")
+        .ok();
   }
 
   @Test public void testAggregateMaterializationNoAggregateFuncs2() {
-    checkMaterialize(
-        "select \"empid\", \"deptno\" from \"emps\" group by \"empid\", \"deptno\"",
-        "select \"deptno\" from \"emps\" group by \"deptno\"",
-        HR_FKUK_MODEL,
-        CalciteAssert.checkResultContains(
+    sql("select \"empid\", \"deptno\" from \"emps\" group by \"empid\", \"deptno\"",
+        "select \"deptno\" from \"emps\" group by \"deptno\"")
+        .withResultContains(
             "EnumerableAggregate(group=[{1}])\n"
-                + "  EnumerableTableScan(table=[[hr, m0]])"));
+                + "  EnumerableTableScan(table=[[hr, m0]])")
+        .ok();
   }
 
   @Test public void testAggregateMaterializationNoAggregateFuncs3() {
-    checkNoMaterialize(
-        "select \"deptno\" from \"emps\" group by \"deptno\"",
-        "select \"empid\", \"deptno\" from \"emps\" group by \"empid\", \"deptno\"",
-        HR_FKUK_MODEL);
+    sql("select \"deptno\" from \"emps\" group by \"deptno\"",
+        "select \"empid\", \"deptno\" from \"emps\" group by \"empid\", \"deptno\"")
+        .noMat();
   }
 
   @Test public void testAggregateMaterializationNoAggregateFuncs4() {
-    checkMaterialize(
-        "select \"empid\", \"deptno\" from \"emps\" where \"deptno\" = 10 group by \"empid\", \"deptno\"",
-        "select \"deptno\" from \"emps\" where \"deptno\" = 10 group by \"deptno\"",
-        HR_FKUK_MODEL,
-        CalciteAssert.checkResultContains(
-            "EnumerableAggregate(group=[{1}])\n"
-                + "  EnumerableTableScan(table=[[hr, m0]])"));
+    final String materialize = "select \"empid\", \"deptno\"\n"
+        + "from \"emps\"\n"
+        + "where \"deptno\" = 10\n"
+        + "group by \"empid\", \"deptno\"";
+    final String query = "select \"deptno\"\n"
+        + "from \"emps\"\n"
+        + "where \"deptno\" = 10\n"
+        + "group by \"deptno\"";
+    final String expected = "EnumerableAggregate(group=[{1}])\n"
+        + "  EnumerableTableScan(table=[[hr, m0]])";
+    sql(materialize, query).withResultContains(expected).ok();
   }
 
   @Test public void testAggregateMaterializationNoAggregateFuncs5() {
-    checkNoMaterialize(
-        "select \"empid\", \"deptno\" from \"emps\" where \"deptno\" = 5 group by \"empid\", \"deptno\"",
-        "select \"deptno\" from \"emps\" where \"deptno\" = 10 group by \"deptno\"",
-        HR_FKUK_MODEL);
+    final String materialize = "select \"empid\", \"deptno\"\n"
+        + "from \"emps\"\n"
+        + "where \"deptno\" = 5\n"
+        + "group by \"empid\", \"deptno\"";
+    final String query = "select \"deptno\"\n"
+        + "from \"emps\"\n"
+        + "where \"deptno\" = 10\n"
+        + "group by \"deptno\"";
+    sql(materialize, query).noMat();
   }
 
   @Test public void testAggregateMaterializationNoAggregateFuncs6() {
-    checkMaterialize(
-        "select \"empid\", \"deptno\" from \"emps\" where \"deptno\" > 5 group by \"empid\", \"deptno\"",
-        "select \"deptno\" from \"emps\" where \"deptno\" > 10 group by \"deptno\"",
-        HR_FKUK_MODEL,
-        CalciteAssert.checkResultContains(
-            "EnumerableAggregate(group=[{1}])\n"
-                + "  EnumerableCalc(expr#0..1=[{inputs}], expr#2=[10], expr#3=[<($t2, $t1)], "
-                + "proj#0..1=[{exprs}], $condition=[$t3])\n"
-                + "    EnumerableTableScan(table=[[hr, m0]])"));
+    final String materialize = "select \"empid\", \"deptno\"\n"
+        + "from \"emps\"\n"
+        + "where \"deptno\" > 5\n"
+        + "group by \"empid\", \"deptno\"";
+    final String query = "select \"deptno\"\n"
+        + "from \"emps\"\n"
+        + "where \"deptno\" > 10\n"
+        + "group by \"deptno\"";
+    final String expected = "EnumerableAggregate(group=[{1}])\n"
+        + "  EnumerableCalc(expr#0..1=[{inputs}], expr#2=[10], expr#3=[<($t2, $t1)], "
+        + "proj#0..1=[{exprs}], $condition=[$t3])\n"
+        + "    EnumerableTableScan(table=[[hr, m0]])";
+    sql(materialize, query).withResultContains(expected).ok();
   }
 
   @Test public void testAggregateMaterializationNoAggregateFuncs7() {
-    checkNoMaterialize(
-        "select \"empid\", \"deptno\" from \"emps\" where \"deptno\" > 5 group by \"empid\", \"deptno\"",
-        "select \"deptno\" from \"emps\" where \"deptno\" < 10 group by \"deptno\"",
-        HR_FKUK_MODEL);
+    final String materialize = "select \"empid\", \"deptno\"\n"
+        + "from \"emps\"\n"
+        + "where \"deptno\" > 5\n"
+        + "group by \"empid\", \"deptno\"";
+    final String query = "select \"deptno\"\n"
+        + "from \"emps\"\n"
+        + "where \"deptno\" < 10\n"
+        + "group by \"deptno\"";
+    sql(materialize, query).noMat();
   }
 
   @Test public void testAggregateMaterializationNoAggregateFuncs8() {
-    checkNoMaterialize(
-        "select \"empid\" from \"emps\" group by \"empid\", \"deptno\"",
-        "select \"deptno\" from \"emps\" group by \"deptno\"",
-        HR_FKUK_MODEL);
+    sql("select \"empid\" from \"emps\" group by \"empid\", \"deptno\"",
+        "select \"deptno\" from \"emps\" group by \"deptno\"")
+        .noMat();
   }
 
   @Test public void testAggregateMaterializationNoAggregateFuncs9() {
-    checkNoMaterialize(
-        "select \"empid\", \"deptno\" from \"emps\"\n"
+    sql("select \"empid\", \"deptno\" from \"emps\"\n"
             + "where \"salary\" > 1000 group by \"name\", \"empid\", \"deptno\"",
         "select \"empid\" from \"emps\"\n"
-            + "where \"salary\" > 2000 group by \"name\", \"empid\"",
-        HR_FKUK_MODEL);
+            + "where \"salary\" > 2000 group by \"name\", \"empid\"")
+        .noMat();
   }
 
   @Test public void testAggregateMaterializationAggregateFuncs1() {
-    checkMaterialize(
-        "select \"empid\", \"deptno\", count(*) as c, sum(\"empid\") as s\n"
+    sql("select \"empid\", \"deptno\", count(*) as c, sum(\"empid\") as s\n"
             + "from \"emps\" group by \"empid\", \"deptno\"",
-        "select \"deptno\" from \"emps\" group by \"deptno\"",
-        HR_FKUK_MODEL,
-        CalciteAssert.checkResultContains(
+        "select \"deptno\" from \"emps\" group by \"deptno\"")
+        .withResultContains(
             "EnumerableAggregate(group=[{1}])\n"
-                + "  EnumerableTableScan(table=[[hr, m0]])"));
+                + "  EnumerableTableScan(table=[[hr, m0]])")
+        .ok();
   }
 
   @Test public void testAggregateMaterializationAggregateFuncs2() {
-    checkMaterialize(
-        "select \"empid\", \"deptno\", count(*) as c, sum(\"empid\") as s\n"
+    sql("select \"empid\", \"deptno\", count(*) as c, sum(\"empid\") as s\n"
             + "from \"emps\" group by \"empid\", \"deptno\"",
         "select \"deptno\", count(*) as c, sum(\"empid\") as s\n"
-            + "from \"emps\" group by \"deptno\"",
-        HR_FKUK_MODEL,
-        CalciteAssert.checkResultContains(
+            + "from \"emps\" group by \"deptno\"")
+        .withResultContains(
             "EnumerableAggregate(group=[{1}], C=[$SUM0($2)], S=[$SUM0($3)])\n"
-                + "  EnumerableTableScan(table=[[hr, m0]])"));
+                + "  EnumerableTableScan(table=[[hr, m0]])")
+        .ok();
   }
 
   @Test public void testAggregateMaterializationAggregateFuncs3() {
-    checkMaterialize(
-        "select \"empid\", \"deptno\", count(*) as c, sum(\"empid\") as s\n"
+    sql("select \"empid\", \"deptno\", count(*) as c, sum(\"empid\") as s\n"
             + "from \"emps\" group by \"empid\", \"deptno\"",
         "select \"deptno\", \"empid\", sum(\"empid\") as s, count(*) as c\n"
-            + "from \"emps\" group by \"empid\", \"deptno\"",
-        HR_FKUK_MODEL,
-        CalciteAssert.checkResultContains(
+            + "from \"emps\" group by \"empid\", \"deptno\"")
+        .withResultContains(
             "EnumerableCalc(expr#0..3=[{inputs}], deptno=[$t1], empid=[$t0], "
                 + "S=[$t3], C=[$t2])\n"
-                + "  EnumerableTableScan(table=[[hr, m0]])"));
+                + "  EnumerableTableScan(table=[[hr, m0]])")
+        .ok();
   }
 
   @Test public void testAggregateMaterializationAggregateFuncs4() {
-    checkMaterialize(
-        "select \"empid\", \"deptno\", count(*) as c, sum(\"empid\") as s\n"
+    sql("select \"empid\", \"deptno\", count(*) as c, sum(\"empid\") as s\n"
             + "from \"emps\" where \"deptno\" >= 10 group by \"empid\", \"deptno\"",
         "select \"deptno\", sum(\"empid\") as s\n"
-            + "from \"emps\" where \"deptno\" > 10 group by \"deptno\"",
-        HR_FKUK_MODEL,
-        CalciteAssert.checkResultContains(
+            + "from \"emps\" where \"deptno\" > 10 group by \"deptno\"")
+        .withResultContains(
             "EnumerableAggregate(group=[{1}], S=[$SUM0($3)])\n"
                 + "  EnumerableCalc(expr#0..3=[{inputs}], expr#4=[10], expr#5=[<($t4, $t1)], "
                 + "proj#0..3=[{exprs}], $condition=[$t5])\n"
-                + "    EnumerableTableScan(table=[[hr, m0]])"));
+                + "    EnumerableTableScan(table=[[hr, m0]])")
+        .ok();
   }
 
   @Test public void testAggregateMaterializationAggregateFuncs5() {
-    checkMaterialize(
-        "select \"empid\", \"deptno\", count(*) + 1 as c, sum(\"empid\") as s\n"
+    sql("select \"empid\", \"deptno\", count(*) + 1 as c, sum(\"empid\") as s\n"
             + "from \"emps\" where \"deptno\" >= 10 group by \"empid\", \"deptno\"",
         "select \"deptno\", sum(\"empid\") + 1 as s\n"
-            + "from \"emps\" where \"deptno\" > 10 group by \"deptno\"",
-        HR_FKUK_MODEL,
-        CalciteAssert.checkResultContains(
+            + "from \"emps\" where \"deptno\" > 10 group by \"deptno\"")
+        .withResultContains(
             "EnumerableCalc(expr#0..1=[{inputs}], expr#2=[1], expr#3=[+($t1, $t2)],"
                 + " deptno=[$t0], S=[$t3])\n"
                 + "  EnumerableAggregate(group=[{1}], agg#0=[$SUM0($3)])\n"
                 + "    EnumerableCalc(expr#0..3=[{inputs}], expr#4=[10], expr#5=[<($t4, $t1)], "
                 + "proj#0..3=[{exprs}], $condition=[$t5])\n"
-                + "      EnumerableTableScan(table=[[hr, m0]])"));
+                + "      EnumerableTableScan(table=[[hr, m0]])")
+        .ok();
   }
 
   @Test public void testAggregateMaterializationAggregateFuncs6() {
-    checkNoMaterialize(
-        "select \"empid\", \"deptno\", count(*) + 1 as c, sum(\"empid\") + 2 as s\n"
+    sql("select \"empid\", \"deptno\", count(*) + 1 as c, sum(\"empid\") + 2 as s\n"
             + "from \"emps\" where \"deptno\" >= 10 group by \"empid\", \"deptno\"",
         "select \"deptno\", sum(\"empid\") + 1 as s\n"
-            + "from \"emps\" where \"deptno\" > 10 group by \"deptno\"",
-        HR_FKUK_MODEL);
+            + "from \"emps\" where \"deptno\" > 10 group by \"deptno\"")
+        .noMat();
   }
 
   @Test public void testAggregateMaterializationAggregateFuncs7() {
-    checkMaterialize(
-        "select \"empid\", \"deptno\", count(*) + 1 as c, sum(\"empid\") as s\n"
+    sql("select \"empid\", \"deptno\", count(*) + 1 as c, sum(\"empid\") as s\n"
             + "from \"emps\" where \"deptno\" >= 10 group by \"empid\", \"deptno\"",
         "select \"deptno\" + 1, sum(\"empid\") + 1 as s\n"
-            + "from \"emps\" where \"deptno\" > 10 group by \"deptno\"",
-        HR_FKUK_MODEL,
-        CalciteAssert.checkResultContains(
+            + "from \"emps\" where \"deptno\" > 10 group by \"deptno\"")
+        .withResultContains(
             "EnumerableCalc(expr#0..1=[{inputs}], expr#2=[1], expr#3=[+($t0, $t2)], "
                 + "expr#4=[+($t1, $t2)], EXPR$0=[$t3], S=[$t4])\n"
                 + "  EnumerableAggregate(group=[{1}], agg#0=[$SUM0($3)])\n"
                 + "    EnumerableCalc(expr#0..3=[{inputs}], expr#4=[10], expr#5=[<($t4, $t1)], "
                 + "proj#0..3=[{exprs}], $condition=[$t5])\n"
-                + "      EnumerableTableScan(table=[[hr, m0]])"));
+                + "      EnumerableTableScan(table=[[hr, m0]])")
+        .ok();
   }
 
   @Disabled
   @Test public void testAggregateMaterializationAggregateFuncs8() {
     // TODO: It should work, but top project in the query is not matched by the planner.
     // It needs further checking.
-    checkMaterialize(
-        "select \"empid\", \"deptno\" + 1, count(*) + 1 as c, sum(\"empid\") as s\n"
+    sql("select \"empid\", \"deptno\" + 1, count(*) + 1 as c, sum(\"empid\") as s\n"
             + "from \"emps\" where \"deptno\" >= 10 group by \"empid\", \"deptno\"",
         "select \"deptno\" + 1, sum(\"empid\") + 1 as s\n"
-            + "from \"emps\" where \"deptno\" > 10 group by \"deptno\"");
+            + "from \"emps\" where \"deptno\" > 10 group by \"deptno\"")
+        .ok();
   }
 
   @Test public void testAggregateMaterializationAggregateFuncs9() {
-    checkMaterialize(
-        "select \"empid\", floor(cast('1997-01-20 12:34:56' as timestamp) to month), count(*) + 1 as c, sum(\"empid\") as s\n"
-            + "from \"emps\" group by \"empid\", floor(cast('1997-01-20 12:34:56' as timestamp) to month)",
-        "select floor(cast('1997-01-20 12:34:56' as timestamp) to year), sum(\"empid\") as s\n"
-            + "from \"emps\" group by floor(cast('1997-01-20 12:34:56' as timestamp) to year)");
+    final String materialize = "select \"empid\",\n"
+        + "  floor(cast('1997-01-20 12:34:56' as timestamp) to month),\n"
+        + "  count(*) + 1 as c, sum(\"empid\") as s\n"
+        + "from \"emps\"\n"
+        + "group by \"empid\",\n"
+        + "  floor(cast('1997-01-20 12:34:56' as timestamp) to month)";
+    final String query = "select\n"
+        + "  floor(cast('1997-01-20 12:34:56' as timestamp) to year),\n"
+        + "  sum(\"empid\") as s\n"
+        + "from \"emps\"\n"
+        + "group by floor(cast('1997-01-20 12:34:56' as timestamp) to year)";
+    sql(materialize, query).ok();
   }
 
   @Test public void testAggregateMaterializationAggregateFuncs10() {
-    checkMaterialize(
-        "select \"empid\", floor(cast('1997-01-20 12:34:56' as timestamp) to month), count(*) + 1 as c, sum(\"empid\") as s\n"
-            + "from \"emps\" group by \"empid\", floor(cast('1997-01-20 12:34:56' as timestamp) to month)",
-        "select floor(cast('1997-01-20 12:34:56' as timestamp) to year), sum(\"empid\") + 1 as s\n"
-            + "from \"emps\" group by floor(cast('1997-01-20 12:34:56' as timestamp) to year)");
+    final String materialize = "select \"empid\",\n"
+        + "  floor(cast('1997-01-20 12:34:56' as timestamp) to month),\n"
+        + "  count(*) + 1 as c, sum(\"empid\") as s\n"
+        + "from \"emps\"\n"
+        + "group by \"empid\",\n"
+        + "  floor(cast('1997-01-20 12:34:56' as timestamp) to month)";
+    final String query = "select\n"
+        + "  floor(cast('1997-01-20 12:34:56' as timestamp) to year),\n"
+        + " sum(\"empid\") + 1 as s\n"
+        + "from \"emps\"\n"
+        + "group by floor(cast('1997-01-20 12:34:56' as timestamp) to year)";
+    sql(materialize, query).ok();
   }
 
   @Test public void testAggregateMaterializationAggregateFuncs11() {
-    checkMaterialize(
-        "select \"empid\", floor(cast('1997-01-20 12:34:56' as timestamp) to second), count(*) + 1 as c, sum(\"empid\") as s\n"
-            + "from \"emps\" group by \"empid\", floor(cast('1997-01-20 12:34:56' as timestamp) to second)",
-        "select floor(cast('1997-01-20 12:34:56' as timestamp) to minute), sum(\"empid\") as s\n"
-            + "from \"emps\" group by floor(cast('1997-01-20 12:34:56' as timestamp) to minute)");
+    final String materialize = "select \"empid\",\n"
+        + "  floor(cast('1997-01-20 12:34:56' as timestamp) to second),\n"
+        + "  count(*) + 1 as c, sum(\"empid\") as s\nfrom \"emps\"\n"
+        + "group by \"empid\",\n"
+        + "  floor(cast('1997-01-20 12:34:56' as timestamp) to second)";
+    final String query = "select\n"
+        + "  floor(cast('1997-01-20 12:34:56' as timestamp) to minute),\n"
+        + "  sum(\"empid\") as s\n"
+        + "from \"emps\"\n"
+        + "group by floor(cast('1997-01-20 12:34:56' as timestamp) to minute)";
+    sql(materialize, query).ok();
   }
 
   @Test public void testAggregateMaterializationAggregateFuncs12() {
-    checkMaterialize(
-        "select \"empid\", floor(cast('1997-01-20 12:34:56' as timestamp) to second), count(*) + 1 as c, sum(\"empid\") as s\n"
-            + "from \"emps\" group by \"empid\", floor(cast('1997-01-20 12:34:56' as timestamp) to second)",
-        "select floor(cast('1997-01-20 12:34:56' as timestamp) to month), sum(\"empid\") as s\n"
-            + "from \"emps\" group by floor(cast('1997-01-20 12:34:56' as timestamp) to month)");
+    final String materialize = "select \"empid\",\n"
+        + "  floor(cast('1997-01-20 12:34:56' as timestamp) to second),\n"
+        + "  count(*) + 1 as c, sum(\"empid\") as s\n"
+        + "from \"emps\"\n"
+        + "group by \"empid\",\n"
+        + "  floor(cast('1997-01-20 12:34:56' as timestamp) to second)";
+    final String query = "select\n"
+        + "  floor(cast('1997-01-20 12:34:56' as timestamp) to month),\n"
+        + "  sum(\"empid\") as s\n"
+        + "from \"emps\"\n"
+        + "group by floor(cast('1997-01-20 12:34:56' as timestamp) to month)";
+    sql(materialize, query).ok();
   }
 
   @Test public void testAggregateMaterializationAggregateFuncs13() {
-    checkMaterialize(
-        "select \"empid\", cast('1997-01-20 12:34:56' as timestamp), count(*) + 1 as c, sum(\"empid\") as s\n"
-            + "from \"emps\" group by \"empid\", cast('1997-01-20 12:34:56' as timestamp)",
-        "select floor(cast('1997-01-20 12:34:56' as timestamp) to year), sum(\"empid\") as s\n"
-            + "from \"emps\" group by floor(cast('1997-01-20 12:34:56' as timestamp) to year)");
+    final String materialize = "select \"empid\",\n"
+        + "  cast('1997-01-20 12:34:56' as timestamp),\n"
+        + "  count(*) + 1 as c, sum(\"empid\") as s\n"
+        + "from \"emps\"\n"
+        + "group by \"empid\", cast('1997-01-20 12:34:56' as timestamp)";
+    final String query = "select\n"
+        + "  floor(cast('1997-01-20 12:34:56' as timestamp) to year),\n"
+        + "  sum(\"empid\") as s\n"
+        + "from \"emps\"\n"
+        + "group by floor(cast('1997-01-20 12:34:56' as timestamp) to year)";
+    sql(materialize, query).ok();
   }
 
   @Test public void testAggregateMaterializationAggregateFuncs14() {
-    checkMaterialize(
-        "select \"empid\", floor(cast('1997-01-20 12:34:56' as timestamp) to month), count(*) + 1 as c, sum(\"empid\") as s\n"
-            + "from \"emps\" group by \"empid\", floor(cast('1997-01-20 12:34:56' as timestamp) to month)",
-        "select floor(cast('1997-01-20 12:34:56' as timestamp) to hour), sum(\"empid\") as s\n"
-            + "from \"emps\" group by floor(cast('1997-01-20 12:34:56' as timestamp) to hour)");
+    final String materialize = "select \"empid\",\n"
+        + "  floor(cast('1997-01-20 12:34:56' as timestamp) to month),\n"
+        + "  count(*) + 1 as c, sum(\"empid\") as s\n"
+        + "from \"emps\"\n"
+        + "group by \"empid\",\n"
+        + "  floor(cast('1997-01-20 12:34:56' as timestamp) to month)";
+    final String query = "select\n"
+        + "  floor(cast('1997-01-20 12:34:56' as timestamp) to hour),\n"
+        + "  sum(\"empid\") as s\n"
+        + "from \"emps\"\n"
+        + "group by floor(cast('1997-01-20 12:34:56' as timestamp) to hour)";
+    sql(materialize, query).ok();
   }
 
   @Test public void testAggregateMaterializationAggregateFuncs15() {
-    checkMaterialize(
-        "select \"eventid\", floor(cast(\"ts\" as timestamp) to second), count(*) + 1 as c, sum(\"eventid\") as s\n"
-            + "from \"events\" group by \"eventid\", floor(cast(\"ts\" as timestamp) to second)",
-        "select floor(cast(\"ts\" as timestamp) to minute), sum(\"eventid\") as s\n"
-            + "from \"events\" group by floor(cast(\"ts\" as timestamp) to minute)");
+    final String materialize = "select \"eventid\",\n"
+        + "  floor(cast(\"ts\" as timestamp) to second), count(*) + 1 as c,\n"
+        + "  sum(\"eventid\") as s\n"
+        + "from \"events\"\n"
+        + "group by \"eventid\", floor(cast(\"ts\" as timestamp) to second)";
+    final String query = "select floor(cast(\"ts\" as timestamp) to minute),\n"
+        + "  sum(\"eventid\") as s\n"
+        + "from \"events\"\n"
+        + "group by floor(cast(\"ts\" as timestamp) to minute)";
+    sql(materialize, query).ok();
   }
 
   @Test public void testAggregateMaterializationAggregateFuncs16() {
-    checkMaterialize(
-        "select \"eventid\", cast(\"ts\" as timestamp), count(*) + 1 as c, sum(\"eventid\") as s\n"
+    sql("select \"eventid\", cast(\"ts\" as timestamp), count(*) + 1 as c, sum(\"eventid\") as s\n"
             + "from \"events\" group by \"eventid\", cast(\"ts\" as timestamp)",
         "select floor(cast(\"ts\" as timestamp) to year), sum(\"eventid\") as s\n"
-            + "from \"events\" group by floor(cast(\"ts\" as timestamp) to year)");
+            + "from \"events\" group by floor(cast(\"ts\" as timestamp) to year)")
+        .ok();
   }
 
   @Test public void testAggregateMaterializationAggregateFuncs17() {
-    checkMaterialize(
-        "select \"eventid\", floor(cast(\"ts\" as timestamp) to month), count(*) + 1 as c, sum(\"eventid\") as s\n"
-            + "from \"events\" group by \"eventid\", floor(cast(\"ts\" as timestamp) to month)",
-        "select floor(cast(\"ts\" as timestamp) to hour), sum(\"eventid\") as s\n"
-            + "from \"events\" group by floor(cast(\"ts\" as timestamp) to hour)",
-        HR_FKUK_MODEL,
-        CalciteAssert.checkResultContains(
-            "EnumerableTableScan(table=[[hr, events]])"));
+    final String materialize = "select \"eventid\",\n"
+        + "  floor(cast(\"ts\" as timestamp) to month), count(*) + 1 as c,\n"
+        + "  sum(\"eventid\") as s\n"
+        + "from \"events\"\n"
+        + "group by \"eventid\", floor(cast(\"ts\" as timestamp) to month)";
+    final String query = "select floor(cast(\"ts\" as timestamp) to hour),\n"
+        + "  sum(\"eventid\") as s\n"
+        + "from \"events\"\n"
+        + "group by floor(cast(\"ts\" as timestamp) to hour)";
+    final String expected = "EnumerableTableScan(table=[[hr, events]])";
+    sql(materialize, query).withResultContains(expected).ok();
   }
 
   @Test public void testAggregateMaterializationAggregateFuncs18() {
-    checkMaterialize(
-        "select \"empid\", \"deptno\", count(*) + 1 as c, sum(\"empid\") as s\n"
+    sql("select \"empid\", \"deptno\", count(*) + 1 as c, sum(\"empid\") as s\n"
             + "from \"emps\" group by \"empid\", \"deptno\"",
         "select \"empid\"*\"deptno\", sum(\"empid\") as s\n"
-            + "from \"emps\" group by \"empid\"*\"deptno\"");
+            + "from \"emps\" group by \"empid\"*\"deptno\"")
+        .ok();
   }
 
   @Test public void testAggregateMaterializationAggregateFuncs19() {
-    checkMaterialize(
-        "select \"empid\", \"deptno\", count(*) as c, sum(\"empid\") as s\n"
+    sql("select \"empid\", \"deptno\", count(*) as c, sum(\"empid\") as s\n"
             + "from \"emps\" group by \"empid\", \"deptno\"",
         "select \"empid\" + 10, count(*) + 1 as c\n"
-            + "from \"emps\" group by \"empid\" + 10");
+            + "from \"emps\" group by \"empid\" + 10")
+        .ok();
   }
 
   @Test public void testJoinAggregateMaterializationNoAggregateFuncs1() {
-    checkMaterialize(
-        "select \"empid\", \"depts\".\"deptno\" from \"emps\"\n"
+    sql("select \"empid\", \"depts\".\"deptno\" from \"emps\"\n"
             + "join \"depts\" using (\"deptno\") where \"depts\".\"deptno\" > 10\n"
             + "group by \"empid\", \"depts\".\"deptno\"",
         "select \"empid\" from \"emps\"\n"
             + "join \"depts\" using (\"deptno\") where \"depts\".\"deptno\" > 20\n"
-            + "group by \"empid\", \"depts\".\"deptno\"",
-        HR_FKUK_MODEL,
-        CalciteAssert.checkResultContains(
+            + "group by \"empid\", \"depts\".\"deptno\"")
+        .withResultContains(
             "EnumerableCalc(expr#0..1=[{inputs}], expr#2=[20], expr#3=[>($t1, $t2)], "
                 + "empid=[$t0], $condition=[$t3])\n"
-                + "  EnumerableTableScan(table=[[hr, m0]])"));
+                + "  EnumerableTableScan(table=[[hr, m0]])")
+        .ok();
   }
 
   @Test public void testJoinAggregateMaterializationNoAggregateFuncs2() {
-    checkMaterialize(
-        "select \"depts\".\"deptno\", \"empid\" from \"depts\"\n"
+    sql("select \"depts\".\"deptno\", \"empid\" from \"depts\"\n"
             + "join \"emps\" using (\"deptno\") where \"depts\".\"deptno\" > 10\n"
             + "group by \"empid\", \"depts\".\"deptno\"",
         "select \"empid\" from \"emps\"\n"
             + "join \"depts\" using (\"deptno\") where \"depts\".\"deptno\" > 20\n"
-            + "group by \"empid\", \"depts\".\"deptno\"",
-        HR_FKUK_MODEL,
-        CalciteAssert.checkResultContains(
+            + "group by \"empid\", \"depts\".\"deptno\"")
+        .withResultContains(
             "EnumerableCalc(expr#0..1=[{inputs}], expr#2=[20], expr#3=[<($t2, $t0)], "
                 + "empid=[$t1], $condition=[$t3])\n"
-                + "  EnumerableTableScan(table=[[hr, m0]])"));
+                + "  EnumerableTableScan(table=[[hr, m0]])")
+        .ok();
   }
 
   @Test public void testJoinAggregateMaterializationNoAggregateFuncs3() {
     // It does not match, Project on top of query
-    checkNoMaterialize(
-        "select \"empid\" from \"emps\"\n"
+    sql("select \"empid\" from \"emps\"\n"
             + "join \"depts\" using (\"deptno\") where \"depts\".\"deptno\" > 10\n"
             + "group by \"empid\", \"depts\".\"deptno\"",
         "select \"empid\" from \"emps\"\n"
             + "join \"depts\" using (\"deptno\") where \"depts\".\"deptno\" > 20\n"
-            + "group by \"empid\", \"depts\".\"deptno\"",
-        HR_FKUK_MODEL);
+            + "group by \"empid\", \"depts\".\"deptno\"")
+        .noMat();
   }
 
   @Test public void testJoinAggregateMaterializationNoAggregateFuncs4() {
-    checkMaterialize(
-        "select \"empid\", \"depts\".\"deptno\" from \"emps\"\n"
+    sql("select \"empid\", \"depts\".\"deptno\" from \"emps\"\n"
             + "join \"depts\" using (\"deptno\") where \"emps\".\"deptno\" > 10\n"
             + "group by \"empid\", \"depts\".\"deptno\"",
         "select \"empid\" from \"emps\"\n"
             + "join \"depts\" using (\"deptno\") where \"depts\".\"deptno\" > 20\n"
-            + "group by \"empid\", \"depts\".\"deptno\"",
-        HR_FKUK_MODEL,
-        CalciteAssert.checkResultContains(
+            + "group by \"empid\", \"depts\".\"deptno\"")
+        .withResultContains(
             "EnumerableCalc(expr#0..1=[{inputs}], expr#2=[20], expr#3=[<($t2, $t1)], "
                 + "empid=[$t0], $condition=[$t3])\n"
-                + "  EnumerableTableScan(table=[[hr, m0]])"));
+                + "  EnumerableTableScan(table=[[hr, m0]])")
+        .ok();
   }
 
   @Test public void testJoinAggregateMaterializationNoAggregateFuncs5() {
-    checkMaterialize(
-        "select \"depts\".\"deptno\", \"emps\".\"empid\" from \"depts\"\n"
-            + "join \"emps\" using (\"deptno\") where \"emps\".\"empid\" > 10\n"
-            + "group by \"depts\".\"deptno\", \"emps\".\"empid\"",
-        "select \"depts\".\"deptno\" from \"depts\"\n"
-            + "join \"emps\" using (\"deptno\") where \"emps\".\"empid\" > 15\n"
-            + "group by \"depts\".\"deptno\", \"emps\".\"empid\"",
-        HR_FKUK_MODEL,
-        CalciteAssert.checkResultContains(
-            "EnumerableCalc(expr#0..1=[{inputs}], expr#2=[15], expr#3=[>($t1, $t2)], "
-                + "deptno=[$t0], $condition=[$t3])\n"
-                + "  EnumerableTableScan(table=[[hr, m0]])"));
+    final String materialize = "select \"depts\".\"deptno\", \"emps\".\"empid\" from \"depts\"\n"
+        + "join \"emps\" using (\"deptno\") where \"emps\".\"empid\" > 10\n"
+        + "group by \"depts\".\"deptno\", \"emps\".\"empid\"";
+    final String query = "select \"depts\".\"deptno\" from \"depts\"\n"
+        + "join \"emps\" using (\"deptno\") where \"emps\".\"empid\" > 15\n"
+        + "group by \"depts\".\"deptno\", \"emps\".\"empid\"";
+    final String expected = ""
+        + "EnumerableCalc(expr#0..1=[{inputs}], expr#2=[15], expr#3=[>($t1, $t2)], "
+        + "deptno=[$t0], $condition=[$t3])\n"
+        + "  EnumerableTableScan(table=[[hr, m0]])";
+    sql(materialize, query).withResultContains(expected).ok();
   }
 
   @Test public void testJoinAggregateMaterializationNoAggregateFuncs6() {
-    checkMaterialize(
-        "select \"depts\".\"deptno\", \"emps\".\"empid\" from \"depts\"\n"
-            + "join \"emps\" using (\"deptno\") where \"emps\".\"empid\" > 10\n"
-            + "group by \"depts\".\"deptno\", \"emps\".\"empid\"",
-        "select \"depts\".\"deptno\" from \"depts\"\n"
-            + "join \"emps\" using (\"deptno\") where \"emps\".\"empid\" > 15\n"
-            + "group by \"depts\".\"deptno\"",
-        HR_FKUK_MODEL,
-        CalciteAssert.checkResultContains(
-            "EnumerableAggregate(group=[{0}])\n"
-                + "  EnumerableCalc(expr#0..1=[{inputs}], expr#2=[15], expr#3=[<($t2, $t1)], "
-                + "proj#0..1=[{exprs}], $condition=[$t3])\n"
-                + "    EnumerableTableScan(table=[[hr, m0]])"));
+    final String materialize = "select \"depts\".\"deptno\", \"emps\".\"empid\" from \"depts\"\n"
+        + "join \"emps\" using (\"deptno\") where \"emps\".\"empid\" > 10\n"
+        + "group by \"depts\".\"deptno\", \"emps\".\"empid\"";
+    final String query = "select \"depts\".\"deptno\" from \"depts\"\n"
+        + "join \"emps\" using (\"deptno\") where \"emps\".\"empid\" > 15\n"
+        + "group by \"depts\".\"deptno\"";
+    final String expected = "EnumerableAggregate(group=[{0}])\n"
+        + "  EnumerableCalc(expr#0..1=[{inputs}], expr#2=[15], expr#3=[<($t2, $t1)], "
+        + "proj#0..1=[{exprs}], $condition=[$t3])\n"
+        + "    EnumerableTableScan(table=[[hr, m0]])";
+    sql(materialize, query).withResultContains(expected).ok();
   }
 
   @Tag("slow")
   @Test public void testJoinAggregateMaterializationNoAggregateFuncs7() {
-    checkMaterialize(
-        "select \"depts\".\"deptno\", \"dependents\".\"empid\"\n"
-            + "from \"depts\"\n"
-            + "join \"dependents\" on (\"depts\".\"name\" = \"dependents\".\"name\")\n"
-            + "join \"locations\" on (\"locations\".\"name\" = \"dependents\".\"name\")\n"
-            + "join \"emps\" on (\"emps\".\"deptno\" = \"depts\".\"deptno\")\n"
-            + "where \"depts\".\"deptno\" > 11\n"
-            + "group by \"depts\".\"deptno\", \"dependents\".\"empid\"",
-        "select \"dependents\".\"empid\"\n"
-            + "from \"depts\"\n"
-            + "join \"dependents\" on (\"depts\".\"name\" = \"dependents\".\"name\")\n"
-            + "join \"locations\" on (\"locations\".\"name\" = \"dependents\".\"name\")\n"
-            + "join \"emps\" on (\"emps\".\"deptno\" = \"depts\".\"deptno\")\n"
-            + "where \"depts\".\"deptno\" > 10\n"
-            + "group by \"dependents\".\"empid\"",
-        HR_FKUK_MODEL,
-        CalciteAssert.checkResultContains(
-            "EnumerableAggregate(group=[{0}])",
-            "EnumerableUnion(all=[true])",
-            "EnumerableAggregate(group=[{2}])",
-            "EnumerableTableScan(table=[[hr, m0]])",
-            "expr#5=[10], expr#6=[>($t0, $t5)], expr#7=[11], expr#8=[>=($t7, $t0)]"));
+    final String materialize = "select \"depts\".\"deptno\",\n"
+        + " \"dependents\".\"empid\"\n"
+        + "from \"depts\"\n"
+        + "join \"dependents\" on (\"depts\".\"name\" = \"dependents\".\"name\")\n"
+        + "join \"locations\" on (\"locations\".\"name\" = \"dependents\".\"name\")\n"
+        + "join \"emps\" on (\"emps\".\"deptno\" = \"depts\".\"deptno\")\n"
+        + "where \"depts\".\"deptno\" > 11\n"
+        + "group by \"depts\".\"deptno\", \"dependents\".\"empid\"";
+    final String query = "select \"dependents\".\"empid\"\n"
+        + "from \"depts\"\n"
+        + "join \"dependents\" on (\"depts\".\"name\" = \"dependents\".\"name\")\n"
+        + "join \"locations\" on (\"locations\".\"name\" = \"dependents\".\"name\")\n"
+        + "join \"emps\" on (\"emps\".\"deptno\" = \"depts\".\"deptno\")\n"
+        + "where \"depts\".\"deptno\" > 10\n"
+        + "group by \"dependents\".\"empid\"";
+    final String[] expecteds = {
+        "EnumerableAggregate(group=[{0}])",
+        "EnumerableUnion(all=[true])",
+        "EnumerableAggregate(group=[{2}])",
+        "EnumerableTableScan(table=[[hr, m0]])",
+        "expr#5=[10], expr#6=[>($t0, $t5)], expr#7=[11], expr#8=[>=($t7, $t0)]"};
+    sql(materialize, query).withResultContains(expecteds).ok();
   }
 
   @Test public void testJoinAggregateMaterializationNoAggregateFuncs8() {
-    checkNoMaterialize(
-        "select \"depts\".\"deptno\", \"dependents\".\"empid\"\n"
-            + "from \"depts\"\n"
-            + "join \"dependents\" on (\"depts\".\"name\" = \"dependents\".\"name\")\n"
-            + "join \"locations\" on (\"locations\".\"name\" = \"dependents\".\"name\")\n"
-            + "join \"emps\" on (\"emps\".\"deptno\" = \"depts\".\"deptno\")\n"
-            + "where \"depts\".\"deptno\" > 20\n"
-            + "group by \"depts\".\"deptno\", \"dependents\".\"empid\"",
-        "select \"dependents\".\"empid\"\n"
-            + "from \"depts\"\n"
-            + "join \"dependents\" on (\"depts\".\"name\" = \"dependents\".\"name\")\n"
-            + "join \"locations\" on (\"locations\".\"name\" = \"dependents\".\"name\")\n"
-            + "join \"emps\" on (\"emps\".\"deptno\" = \"depts\".\"deptno\")\n"
-            + "where \"depts\".\"deptno\" > 10 and \"depts\".\"deptno\" < 20\n"
-            + "group by \"dependents\".\"empid\"",
-        HR_FKUK_MODEL);
+    final String materialize = "select \"depts\".\"deptno\",\n"
+        + " \"dependents\".\"empid\"\n"
+        + "from \"depts\"\n"
+        + "join \"dependents\" on (\"depts\".\"name\" = \"dependents\".\"name\")\n"
+        + "join \"locations\" on (\"locations\".\"name\" = \"dependents\".\"name\")\n"
+        + "join \"emps\" on (\"emps\".\"deptno\" = \"depts\".\"deptno\")\n"
+        + "where \"depts\".\"deptno\" > 20\n"
+        + "group by \"depts\".\"deptno\", \"dependents\".\"empid\"";
+    final String query = "select \"dependents\".\"empid\"\n"
+        + "from \"depts\"\n"
+        + "join \"dependents\" on (\"depts\".\"name\" = \"dependents\".\"name\")\n"
+        + "join \"locations\" on (\"locations\".\"name\" = \"dependents\".\"name\")\n"
+        + "join \"emps\" on (\"emps\".\"deptno\" = \"depts\".\"deptno\")\n"
+        + "where \"depts\".\"deptno\" > 10 and \"depts\".\"deptno\" < 20\n"
+        + "group by \"dependents\".\"empid\"";
+    sql(materialize, query).noMat();
   }
 
   @Test public void testJoinAggregateMaterializationNoAggregateFuncs9() {
-    checkMaterialize(
-        "select \"depts\".\"deptno\", \"dependents\".\"empid\"\n"
-            + "from \"depts\"\n"
-            + "join \"dependents\" on (\"depts\".\"name\" = \"dependents\".\"name\")\n"
-            + "join \"locations\" on (\"locations\".\"name\" = \"dependents\".\"name\")\n"
-            + "join \"emps\" on (\"emps\".\"deptno\" = \"depts\".\"deptno\")\n"
-            + "where \"depts\".\"deptno\" > 11 and \"depts\".\"deptno\" < 19\n"
-            + "group by \"depts\".\"deptno\", \"dependents\".\"empid\"",
-        "select \"dependents\".\"empid\"\n"
-            + "from \"depts\"\n"
-            + "join \"dependents\" on (\"depts\".\"name\" = \"dependents\".\"name\")\n"
-            + "join \"locations\" on (\"locations\".\"name\" = \"dependents\".\"name\")\n"
-            + "join \"emps\" on (\"emps\".\"deptno\" = \"depts\".\"deptno\")\n"
-            + "where \"depts\".\"deptno\" > 10 and \"depts\".\"deptno\" < 20\n"
-            + "group by \"dependents\".\"empid\"",
-        HR_FKUK_MODEL,
-        CalciteAssert.checkResultContains(
-            "EnumerableAggregate(group=[{0}])",
-            "EnumerableUnion(all=[true])",
-            "EnumerableAggregate(group=[{2}])",
-            "EnumerableTableScan(table=[[hr, m0]])",
-            "expr#13=[OR($t10, $t12)], expr#14=[AND($t6, $t8, $t13)]"));
+    final String materialize = "select \"depts\".\"deptno\",\n"
+        + " \"dependents\".\"empid\"\n"
+        + "from \"depts\"\n"
+        + "join \"dependents\" on (\"depts\".\"name\" = \"dependents\".\"name\")\n"
+        + "join \"locations\" on (\"locations\".\"name\" = \"dependents\".\"name\")\n"
+        + "join \"emps\" on (\"emps\".\"deptno\" = \"depts\".\"deptno\")\n"
+        + "where \"depts\".\"deptno\" > 11 and \"depts\".\"deptno\" < 19\n"
+        + "group by \"depts\".\"deptno\", \"dependents\".\"empid\"";
+    final String query = "select \"dependents\".\"empid\"\n"
+        + "from \"depts\"\n"
+        + "join \"dependents\" on (\"depts\".\"name\" = \"dependents\".\"name\")\n"
+        + "join \"locations\" on (\"locations\".\"name\" = \"dependents\".\"name\")\n"
+        + "join \"emps\" on (\"emps\".\"deptno\" = \"depts\".\"deptno\")\n"
+        + "where \"depts\".\"deptno\" > 10 and \"depts\".\"deptno\" < 20\n"
+        + "group by \"dependents\".\"empid\"";
+    final String[] expecteds = {
+        "EnumerableAggregate(group=[{0}])",
+        "EnumerableUnion(all=[true])",
+        "EnumerableAggregate(group=[{2}])",
+        "EnumerableTableScan(table=[[hr, m0]])",
+        "expr#13=[OR($t10, $t12)], expr#14=[AND($t6, $t8, $t13)]"};
+    sql(materialize, query).withResultContains(expecteds).ok();
   }
 
   @Tag("slow")
   @Test public void testJoinAggregateMaterializationNoAggregateFuncs10() {
-    checkMaterialize(
-        "select \"depts\".\"name\", \"dependents\".\"name\" as \"name2\", "
-            + "\"emps\".\"deptno\", \"depts\".\"deptno\" as \"deptno2\", "
-            + "\"dependents\".\"empid\"\n"
-            + "from \"depts\", \"dependents\", \"emps\"\n"
-            + "where \"depts\".\"deptno\" > 10\n"
-            + "group by \"depts\".\"name\", \"dependents\".\"name\", "
-            + "\"emps\".\"deptno\", \"depts\".\"deptno\", "
-            + "\"dependents\".\"empid\"",
-        "select \"dependents\".\"empid\"\n"
-            + "from \"depts\"\n"
-            + "join \"dependents\" on (\"depts\".\"name\" = \"dependents\".\"name\")\n"
-            + "join \"emps\" on (\"emps\".\"deptno\" = \"depts\".\"deptno\")\n"
-            + "where \"depts\".\"deptno\" > 10\n"
-            + "group by \"dependents\".\"empid\"",
-        HR_FKUK_MODEL,
-        CalciteAssert.checkResultContains(
-            "EnumerableAggregate(group=[{4}])\n"
-                + "  EnumerableCalc(expr#0..4=[{inputs}], expr#5=[=($t2, $t3)], "
-                + "expr#6=[CAST($t1):VARCHAR], "
-                + "expr#7=[CAST($t0):VARCHAR], "
-                + "expr#8=[=($t6, $t7)], expr#9=[AND($t5, $t8)], proj#0..4=[{exprs}], $condition=[$t9])\n"
-                + "    EnumerableTableScan(table=[[hr, m0]])"));
+    final String materialize = "select \"depts\".\"name\", \"dependents\".\"name\" as \"name2\", "
+        + "\"emps\".\"deptno\", \"depts\".\"deptno\" as \"deptno2\", "
+        + "\"dependents\".\"empid\"\n"
+        + "from \"depts\", \"dependents\", \"emps\"\n"
+        + "where \"depts\".\"deptno\" > 10\n"
+        + "group by \"depts\".\"name\", \"dependents\".\"name\", "
+        + "\"emps\".\"deptno\", \"depts\".\"deptno\", "
+        + "\"dependents\".\"empid\"";
+    final String query = "select \"dependents\".\"empid\"\n"
+        + "from \"depts\"\n"
+        + "join \"dependents\" on (\"depts\".\"name\" = \"dependents\".\"name\")\n"
+        + "join \"emps\" on (\"emps\".\"deptno\" = \"depts\".\"deptno\")\n"
+        + "where \"depts\".\"deptno\" > 10\n"
+        + "group by \"dependents\".\"empid\"";
+    final String expected = "EnumerableAggregate(group=[{4}])\n"
+        + "  EnumerableCalc(expr#0..4=[{inputs}], expr#5=[=($t2, $t3)], "
+        + "expr#6=[CAST($t1):VARCHAR], expr#7=[CAST($t0):VARCHAR], "
+        + "expr#8=[=($t6, $t7)], expr#9=[AND($t5, $t8)], proj#0..4=[{exprs}], "
+        + "$condition=[$t9])\n"
+        + "    EnumerableTableScan(table=[[hr, m0]])";
+    sql(materialize, query).withResultContains(expected).ok();
   }
 
   @Test public void testJoinAggregateMaterializationAggregateFuncs1() {
     // This test relies on FK-UK relationship
-    checkMaterialize(
+    final String materialize =
         "select \"empid\", \"depts\".\"deptno\", count(*) as c, sum(\"empid\") as s\n"
             + "from \"emps\" join \"depts\" using (\"deptno\")\n"
-            + "group by \"empid\", \"depts\".\"deptno\"",
-        "select \"deptno\" from \"emps\" group by \"deptno\"",
-        HR_FKUK_MODEL,
-        CalciteAssert.checkResultContains(
-            "EnumerableAggregate(group=[{1}])\n"
-                + "  EnumerableTableScan(table=[[hr, m0]])"));
+            + "group by \"empid\", \"depts\".\"deptno\"";
+    final String query = "select \"deptno\" from \"emps\" group by \"deptno\"";
+    final String expected = "EnumerableAggregate(group=[{1}])\n"
+        + "  EnumerableTableScan(table=[[hr, m0]])";
+    sql(materialize, query).withResultContains(expected).ok();
   }
 
   @Test public void testJoinAggregateMaterializationAggregateFuncs2() {
-    checkMaterialize(
+    final String materialize =
         "select \"empid\", \"emps\".\"deptno\", count(*) as c, sum(\"empid\") as s\n"
             + "from \"emps\" join \"depts\" using (\"deptno\")\n"
-            + "group by \"empid\", \"emps\".\"deptno\"",
-        "select \"depts\".\"deptno\", count(*) as c, sum(\"empid\") as s\n"
-            + "from \"emps\" join \"depts\" using (\"deptno\")\n"
-            + "group by \"depts\".\"deptno\"",
-        HR_FKUK_MODEL,
-        CalciteAssert.checkResultContains(
-            "EnumerableAggregate(group=[{1}], C=[$SUM0($2)], S=[$SUM0($3)])\n"
-                + "  EnumerableTableScan(table=[[hr, m0]])"));
+            + "group by \"empid\", \"emps\".\"deptno\"";
+    final String query = "select \"depts\".\"deptno\", count(*) as c, sum(\"empid\") as s\n"
+        + "from \"emps\" join \"depts\" using (\"deptno\")\n"
+        + "group by \"depts\".\"deptno\"";
+    final String expected = "EnumerableAggregate(group=[{1}], C=[$SUM0($2)], S=[$SUM0($3)])\n"
+        + "  EnumerableTableScan(table=[[hr, m0]])";
+    sql(materialize, query).withResultContains(expected).ok();
   }
 
   @Test public void testJoinAggregateMaterializationAggregateFuncs3() {
     // This test relies on FK-UK relationship
-    checkMaterialize(
+    final String materialize =
         "select \"empid\", \"depts\".\"deptno\", count(*) as c, sum(\"empid\") as s\n"
             + "from \"emps\" join \"depts\" using (\"deptno\")\n"
-            + "group by \"empid\", \"depts\".\"deptno\"",
-        "select \"deptno\", \"empid\", sum(\"empid\") as s, count(*) as c\n"
-            + "from \"emps\" group by \"empid\", \"deptno\"",
-        HR_FKUK_MODEL,
-        CalciteAssert.checkResultContains(
-            "EnumerableCalc(expr#0..3=[{inputs}], deptno=[$t1], empid=[$t0], "
-                + "S=[$t3], C=[$t2])\n"
-                + "  EnumerableTableScan(table=[[hr, m0]])"));
+            + "group by \"empid\", \"depts\".\"deptno\"";
+    final String query = "select \"deptno\", \"empid\", sum(\"empid\") as s, count(*) as c\n"
+        + "from \"emps\" group by \"empid\", \"deptno\"";
+    final String expected = "EnumerableCalc(expr#0..3=[{inputs}], "
+        + "deptno=[$t1], empid=[$t0], S=[$t3], C=[$t2])\n"
+            + "  EnumerableTableScan(table=[[hr, m0]])";
+    sql(materialize, query).withResultContains(expected).ok();
   }
 
   @Test public void testJoinAggregateMaterializationAggregateFuncs4() {
-    checkMaterialize(
+    final String materialize =
         "select \"empid\", \"emps\".\"deptno\", count(*) as c, sum(\"empid\") as s\n"
             + "from \"emps\" join \"depts\" using (\"deptno\")\n"
-            + "where \"emps\".\"deptno\" >= 10 group by \"empid\", \"emps\".\"deptno\"",
-        "select \"depts\".\"deptno\", sum(\"empid\") as s\n"
-            + "from \"emps\" join \"depts\" using (\"deptno\")\n"
-            + "where \"emps\".\"deptno\" > 10 group by \"depts\".\"deptno\"",
-        HR_FKUK_MODEL,
-        CalciteAssert.checkResultContains(
-            "EnumerableAggregate(group=[{1}], S=[$SUM0($3)])\n"
-                + "  EnumerableCalc(expr#0..3=[{inputs}], expr#4=[10], expr#5=[<($t4, $t1)], "
-                + "proj#0..3=[{exprs}], $condition=[$t5])\n"
-                + "    EnumerableTableScan(table=[[hr, m0]])"));
+            + "where \"emps\".\"deptno\" >= 10 group by \"empid\", \"emps\".\"deptno\"";
+    final String query = "select \"depts\".\"deptno\", sum(\"empid\") as s\n"
+        + "from \"emps\" join \"depts\" using (\"deptno\")\n"
+        + "where \"emps\".\"deptno\" > 10 group by \"depts\".\"deptno\"";
+    final String expected = "EnumerableAggregate(group=[{1}], S=[$SUM0($3)])\n"
+        + "  EnumerableCalc(expr#0..3=[{inputs}], expr#4=[10], expr#5=[<($t4, $t1)], "
+        + "proj#0..3=[{exprs}], $condition=[$t5])\n"
+        + "    EnumerableTableScan(table=[[hr, m0]])";
+    sql(materialize, query).withResultContains(expected).ok();
   }
 
   @Test public void testJoinAggregateMaterializationAggregateFuncs5() {
-    checkMaterialize(
+    final String materialize =
         "select \"empid\", \"depts\".\"deptno\", count(*) + 1 as c, sum(\"empid\") as s\n"
             + "from \"emps\" join \"depts\" using (\"deptno\")\n"
-            + "where \"depts\".\"deptno\" >= 10 group by \"empid\", \"depts\".\"deptno\"",
-        "select \"depts\".\"deptno\", sum(\"empid\") + 1 as s\n"
-            + "from \"emps\" join \"depts\" using (\"deptno\")\n"
-            + "where \"depts\".\"deptno\" > 10 group by \"depts\".\"deptno\"",
-        HR_FKUK_MODEL,
-        CalciteAssert.checkResultContains(
-            "EnumerableCalc(expr#0..1=[{inputs}], expr#2=[1], expr#3=[+($t1, $t2)], "
-                + "deptno=[$t0], S=[$t3])\n"
-                + "  EnumerableAggregate(group=[{1}], agg#0=[$SUM0($3)])\n"
-                + "    EnumerableCalc(expr#0..3=[{inputs}], expr#4=[10], expr#5=[<($t4, $t1)], "
-                + "proj#0..3=[{exprs}], $condition=[$t5])\n"
-                + "      EnumerableTableScan(table=[[hr, m0]])"));
+            + "where \"depts\".\"deptno\" >= 10 group by \"empid\", \"depts\".\"deptno\"";
+    final String query = "select \"depts\".\"deptno\", sum(\"empid\") + 1 as s\n"
+        + "from \"emps\" join \"depts\" using (\"deptno\")\n"
+        + "where \"depts\".\"deptno\" > 10 group by \"depts\".\"deptno\"";
+    final String expected = "EnumerableCalc(expr#0..1=[{inputs}], expr#2=[1], "
+        + "expr#3=[+($t1, $t2)], deptno=[$t0], S=[$t3])\n"
+        + "  EnumerableAggregate(group=[{1}], agg#0=[$SUM0($3)])\n"
+        + "    EnumerableCalc(expr#0..3=[{inputs}], expr#4=[10], expr#5=[<($t4, $t1)], "
+        + "proj#0..3=[{exprs}], $condition=[$t5])\n"
+        + "      EnumerableTableScan(table=[[hr, m0]])";
+    sql(materialize, query).withResultContains(expected).ok();
   }
 
   @Disabled
@@ -2076,97 +2119,100 @@ public class MaterializationTest {
         + "join \"depts\" on (\"emps\".\"deptno\" = \"depts\".\"deptno\")\n"
         + "join \"dependents\" on (\"depts\".\"name\" = \"dependents\".\"name\")\n"
         + "group by \"dependents\".\"empid\"";
-    checkMaterialize(m, q);
+    sql(m, q).ok();
   }
 
   @Test public void testJoinAggregateMaterializationAggregateFuncs7() {
-    checkMaterialize(
-        "select \"dependents\".\"empid\", \"emps\".\"deptno\", sum(\"salary\") as s\n"
-            + "from \"emps\"\n"
-            + "join \"dependents\" on (\"emps\".\"empid\" = \"dependents\".\"empid\")\n"
-            + "group by \"dependents\".\"empid\", \"emps\".\"deptno\"",
-        "select \"dependents\".\"empid\", sum(\"salary\") as s\n"
-            + "from \"emps\"\n"
-            + "join \"depts\" on (\"emps\".\"deptno\" = \"depts\".\"deptno\")\n"
-            + "join \"dependents\" on (\"emps\".\"empid\" = \"dependents\".\"empid\")\n"
-            + "group by \"dependents\".\"empid\"",
-        HR_FKUK_MODEL,
-        CalciteAssert.checkResultContains(
-            "EnumerableAggregate(group=[{4}], S=[$SUM0($6)])\n"
-                + "  EnumerableCalc(expr#0..6=[{inputs}], expr#7=[=($t5, $t0)], proj#0..6=[{exprs}], $condition=[$t7])\n"
-                + "    EnumerableNestedLoopJoin(condition=[true], joinType=[inner])\n"
-                + "      EnumerableTableScan(table=[[hr, depts]])\n"
-                + "      EnumerableTableScan(table=[[hr, m0]])"));
+    final String materialize = "select \"dependents\".\"empid\",\n"
+        + "  \"emps\".\"deptno\", sum(\"salary\") as s\n"
+        + "from \"emps\"\n"
+        + "join \"dependents\" on (\"emps\".\"empid\" = \"dependents\".\"empid\")\n"
+        + "group by \"dependents\".\"empid\", \"emps\".\"deptno\"";
+    final String query = "select \"dependents\".\"empid\", sum(\"salary\") as s\n"
+        + "from \"emps\"\n"
+        + "join \"depts\" on (\"emps\".\"deptno\" = \"depts\".\"deptno\")\n"
+        + "join \"dependents\" on (\"emps\".\"empid\" = \"dependents\".\"empid\")\n"
+        + "group by \"dependents\".\"empid\"";
+    final String expected = "EnumerableAggregate(group=[{4}], S=[$SUM0($6)])\n"
+        + "  EnumerableCalc(expr#0..6=[{inputs}], expr#7=[=($t5, $t0)], "
+        + "proj#0..6=[{exprs}], $condition=[$t7])\n"
+        + "    EnumerableNestedLoopJoin(condition=[true], joinType=[inner])\n"
+        + "      EnumerableTableScan(table=[[hr, depts]])\n"
+        + "      EnumerableTableScan(table=[[hr, m0]])";
+    sql(materialize, query).withResultContains(expected).ok();
   }
 
   @Test public void testJoinAggregateMaterializationAggregateFuncs8() {
-    checkMaterialize(
-        "select \"dependents\".\"empid\", \"emps\".\"deptno\", sum(\"salary\") as s\n"
-            + "from \"emps\"\n"
-            + "join \"dependents\" on (\"emps\".\"empid\" = \"dependents\".\"empid\")\n"
-            + "group by \"dependents\".\"empid\", \"emps\".\"deptno\"",
-        "select \"depts\".\"name\", sum(\"salary\") as s\n"
-            + "from \"emps\"\n"
-            + "join \"depts\" on (\"emps\".\"deptno\" = \"depts\".\"deptno\")\n"
-            + "join \"dependents\" on (\"emps\".\"empid\" = \"dependents\".\"empid\")\n"
-            + "group by \"depts\".\"name\"",
-        HR_FKUK_MODEL,
-        CalciteAssert.checkResultContains(
-            "EnumerableAggregate(group=[{1}], S=[$SUM0($6)])\n"
-                + "  EnumerableCalc(expr#0..6=[{inputs}], expr#7=[=($t5, $t0)], proj#0..6=[{exprs}], $condition=[$t7])\n"
-                + "    EnumerableNestedLoopJoin(condition=[true], joinType=[inner])\n"
-                + "      EnumerableTableScan(table=[[hr, depts]])\n"
-                + "      EnumerableTableScan(table=[[hr, m0]])"));
+    final String materialize = "select \"dependents\".\"empid\",\n"
+        + "  \"emps\".\"deptno\", sum(\"salary\") as s\n"
+        + "from \"emps\"\n"
+        + "join \"dependents\" on (\"emps\".\"empid\" = \"dependents\".\"empid\")\n"
+        + "group by \"dependents\".\"empid\", \"emps\".\"deptno\"";
+    final String query = "select \"depts\".\"name\", sum(\"salary\") as s\n"
+        + "from \"emps\"\n"
+        + "join \"depts\" on (\"emps\".\"deptno\" = \"depts\".\"deptno\")\n"
+        + "join \"dependents\" on (\"emps\".\"empid\" = \"dependents\".\"empid\")\n"
+        + "group by \"depts\".\"name\"";
+    final String expected = "EnumerableAggregate(group=[{1}], S=[$SUM0($6)])\n"
+        + "  EnumerableCalc(expr#0..6=[{inputs}], expr#7=[=($t5, $t0)], "
+        + "proj#0..6=[{exprs}], $condition=[$t7])\n"
+        + "    EnumerableNestedLoopJoin(condition=[true], joinType=[inner])\n"
+        + "      EnumerableTableScan(table=[[hr, depts]])\n"
+        + "      EnumerableTableScan(table=[[hr, m0]])";
+    sql(materialize, query).withResultContains(expected).ok();
   }
 
   @Test public void testJoinAggregateMaterializationAggregateFuncs9() {
-    checkMaterialize(
-        "select \"dependents\".\"empid\", \"emps\".\"deptno\", count(distinct \"salary\") as s\n"
-            + "from \"emps\"\n"
-            + "join \"dependents\" on (\"emps\".\"empid\" = \"dependents\".\"empid\")\n"
-            + "group by \"dependents\".\"empid\", \"emps\".\"deptno\"",
-        "select \"emps\".\"deptno\", count(distinct \"salary\") as s\n"
-            + "from \"emps\"\n"
-            + "join \"dependents\" on (\"emps\".\"empid\" = \"dependents\".\"empid\")\n"
-            + "group by \"dependents\".\"empid\", \"emps\".\"deptno\"",
-        HR_FKUK_MODEL,
-        CalciteAssert.checkResultContains(
-            "EnumerableCalc(expr#0..2=[{inputs}], deptno=[$t1], S=[$t2])\n"
-                + "  EnumerableTableScan(table=[[hr, m0]])"));
+    final String materialize = "select \"dependents\".\"empid\",\n"
+        + "  \"emps\".\"deptno\", count(distinct \"salary\") as s\n"
+        + "from \"emps\"\n"
+        + "join \"dependents\" on (\"emps\".\"empid\" = \"dependents\".\"empid\")\n"
+        + "group by \"dependents\".\"empid\", \"emps\".\"deptno\"";
+    final String query = "select \"emps\".\"deptno\",\n"
+        + "  count(distinct \"salary\") as s\n"
+        + "from \"emps\"\n"
+        + "join \"dependents\" on (\"emps\".\"empid\" = \"dependents\".\"empid\")\n"
+        + "group by \"dependents\".\"empid\", \"emps\".\"deptno\"";
+    final String expected = "EnumerableCalc(expr#0..2=[{inputs}], "
+        + "deptno=[$t1], S=[$t2])\n"
+        + "  EnumerableTableScan(table=[[hr, m0]])";
+    sql(materialize, query).withResultContains(expected).ok();
   }
 
   @Test public void testJoinAggregateMaterializationAggregateFuncs10() {
-    checkNoMaterialize(
-        "select \"dependents\".\"empid\", \"emps\".\"deptno\", count(distinct \"salary\") as s\n"
-            + "from \"emps\"\n"
-            + "join \"dependents\" on (\"emps\".\"empid\" = \"dependents\".\"empid\")\n"
-            + "group by \"dependents\".\"empid\", \"emps\".\"deptno\"",
-        "select \"emps\".\"deptno\", count(distinct \"salary\") as s\n"
-            + "from \"emps\"\n"
-            + "join \"dependents\" on (\"emps\".\"empid\" = \"dependents\".\"empid\")\n"
-            + "group by \"emps\".\"deptno\"",
-        HR_FKUK_MODEL);
+    final String materialize = "select \"dependents\".\"empid\",\n"
+        + "  \"emps\".\"deptno\", count(distinct \"salary\") as s\n"
+        + "from \"emps\"\n"
+        + "join \"dependents\" on (\"emps\".\"empid\" = \"dependents\".\"empid\")\n"
+        + "group by \"dependents\".\"empid\", \"emps\".\"deptno\"";
+    final String query = "select \"emps\".\"deptno\",\n"
+        + "  count(distinct \"salary\") as s\n"
+        + "from \"emps\"\n"
+        + "join \"dependents\" on (\"emps\".\"empid\" = \"dependents\".\"empid\")\n"
+        + "group by \"emps\".\"deptno\"";
+    sql(materialize, query).noMat();
   }
 
   @Tag("slow")
   @Test public void testJoinAggregateMaterializationAggregateFuncs11() {
-    checkMaterialize(
-        "select \"depts\".\"deptno\", \"dependents\".\"empid\", count(\"emps\".\"salary\") as s\n"
-            + "from \"depts\"\n"
-            + "join \"dependents\" on (\"depts\".\"name\" = \"dependents\".\"name\")\n"
-            + "join \"locations\" on (\"locations\".\"name\" = \"dependents\".\"name\")\n"
-            + "join \"emps\" on (\"emps\".\"deptno\" = \"depts\".\"deptno\")\n"
-            + "where \"depts\".\"deptno\" > 11 and \"depts\".\"deptno\" < 19\n"
-            + "group by \"depts\".\"deptno\", \"dependents\".\"empid\"",
-        "select \"dependents\".\"empid\", count(\"emps\".\"salary\") + 1\n"
-            + "from \"depts\"\n"
-            + "join \"dependents\" on (\"depts\".\"name\" = \"dependents\".\"name\")\n"
-            + "join \"locations\" on (\"locations\".\"name\" = \"dependents\".\"name\")\n"
-            + "join \"emps\" on (\"emps\".\"deptno\" = \"depts\".\"deptno\")\n"
-            + "where \"depts\".\"deptno\" > 10 and \"depts\".\"deptno\" < 20\n"
-            + "group by \"dependents\".\"empid\"",
-        HR_FKUK_MODEL,
-        CalciteAssert.checkResultContains(
+    final String materialize = "select \"depts\".\"deptno\",\n"
+        + "  \"dependents\".\"empid\", count(\"emps\".\"salary\") as s\n"
+        + "from \"depts\"\n"
+        + "join \"dependents\" on (\"depts\".\"name\" = \"dependents\".\"name\")\n"
+        + "join \"locations\" on (\"locations\".\"name\" = \"dependents\".\"name\")\n"
+        + "join \"emps\" on (\"emps\".\"deptno\" = \"depts\".\"deptno\")\n"
+        + "where \"depts\".\"deptno\" > 11 and \"depts\".\"deptno\" < 19\n"
+        + "group by \"depts\".\"deptno\", \"dependents\".\"empid\"";
+    final String query = "select \"dependents\".\"empid\",\n"
+        + "  count(\"emps\".\"salary\") + 1\n"
+        + "from \"depts\"\n"
+        + "join \"dependents\" on (\"depts\".\"name\" = \"dependents\".\"name\")\n"
+        + "join \"locations\" on (\"locations\".\"name\" = \"dependents\".\"name\")\n"
+        + "join \"emps\" on (\"emps\".\"deptno\" = \"depts\".\"deptno\")\n"
+        + "where \"depts\".\"deptno\" > 10 and \"depts\".\"deptno\" < 20\n"
+        + "group by \"dependents\".\"empid\"";
+    sql(materialize, query)
+        .withResultContains(
             "PLAN=EnumerableCalc(expr#0..1=[{inputs}], expr#2=[1], expr#3=[+($t1, $t2)], "
                 + "empid=[$t0], EXPR$1=[$t3])\n"
                 + "  EnumerableAggregate(group=[{0}], agg#0=[$SUM0($1)])",
@@ -2174,44 +2220,46 @@ public class MaterializationTest {
             "EnumerableAggregate(group=[{2}], agg#0=[COUNT()])",
             "EnumerableAggregate(group=[{1}], agg#0=[$SUM0($2)])",
             "EnumerableTableScan(table=[[hr, m0]])",
-            "expr#13=[OR($t10, $t12)], expr#14=[AND($t6, $t8, $t13)]"));
+            "expr#13=[OR($t10, $t12)], expr#14=[AND($t6, $t8, $t13)]")
+        .ok();
   }
 
   @Test public void testJoinAggregateMaterializationAggregateFuncs12() {
-    checkNoMaterialize(
-        "select \"depts\".\"deptno\", \"dependents\".\"empid\", count(distinct \"emps\".\"salary\") as s\n"
-            + "from \"depts\"\n"
-            + "join \"dependents\" on (\"depts\".\"name\" = \"dependents\".\"name\")\n"
-            + "join \"locations\" on (\"locations\".\"name\" = \"dependents\".\"name\")\n"
-            + "join \"emps\" on (\"emps\".\"deptno\" = \"depts\".\"deptno\")\n"
-            + "where \"depts\".\"deptno\" > 11 and \"depts\".\"deptno\" < 19\n"
-            + "group by \"depts\".\"deptno\", \"dependents\".\"empid\"",
-        "select \"dependents\".\"empid\", count(distinct \"emps\".\"salary\") + 1\n"
-            + "from \"depts\"\n"
-            + "join \"dependents\" on (\"depts\".\"name\" = \"dependents\".\"name\")\n"
-            + "join \"locations\" on (\"locations\".\"name\" = \"dependents\".\"name\")\n"
-            + "join \"emps\" on (\"emps\".\"deptno\" = \"depts\".\"deptno\")\n"
-            + "where \"depts\".\"deptno\" > 10 and \"depts\".\"deptno\" < 20\n"
-            + "group by \"dependents\".\"empid\"",
-        HR_FKUK_MODEL);
+    final String materialize = "select \"depts\".\"deptno\",\n"
+        + "  \"dependents\".\"empid\",\n"
+        + "  count(distinct \"emps\".\"salary\") as s\n"
+        + "from \"depts\"\n"
+        + "join \"dependents\" on (\"depts\".\"name\" = \"dependents\".\"name\")\n"
+        + "join \"locations\" on (\"locations\".\"name\" = \"dependents\".\"name\")\n"
+        + "join \"emps\" on (\"emps\".\"deptno\" = \"depts\".\"deptno\")\n"
+        + "where \"depts\".\"deptno\" > 11 and \"depts\".\"deptno\" < 19\n"
+        + "group by \"depts\".\"deptno\", \"dependents\".\"empid\"";
+    final String query = "select \"dependents\".\"empid\",\n"
+        + "  count(distinct \"emps\".\"salary\") + 1\n"
+        + "from \"depts\"\n"
+        + "join \"dependents\" on (\"depts\".\"name\" = \"dependents\".\"name\")\n"
+        + "join \"locations\" on (\"locations\".\"name\" = \"dependents\".\"name\")\n"
+        + "join \"emps\" on (\"emps\".\"deptno\" = \"depts\".\"deptno\")\n"
+        + "where \"depts\".\"deptno\" > 10 and \"depts\".\"deptno\" < 20\n"
+        + "group by \"dependents\".\"empid\"";
+    sql(materialize, query).noMat();
   }
 
   @Test public void testJoinAggregateMaterializationAggregateFuncs13() {
-    checkNoMaterialize(
-        "select \"dependents\".\"empid\", \"emps\".\"deptno\", count(distinct \"salary\") as s\n"
-            + "from \"emps\"\n"
-            + "join \"dependents\" on (\"emps\".\"empid\" = \"dependents\".\"empid\")\n"
-            + "group by \"dependents\".\"empid\", \"emps\".\"deptno\"",
-        "select \"emps\".\"deptno\", count(\"salary\") as s\n"
-            + "from \"emps\"\n"
-            + "join \"dependents\" on (\"emps\".\"empid\" = \"dependents\".\"empid\")\n"
-            + "group by \"dependents\".\"empid\", \"emps\".\"deptno\"",
-        HR_FKUK_MODEL);
+    final String materialize = "select \"dependents\".\"empid\",\n"
+        + "  \"emps\".\"deptno\", count(distinct \"salary\") as s\n"
+        + "from \"emps\"\n"
+        + "join \"dependents\" on (\"emps\".\"empid\" = \"dependents\".\"empid\")\n"
+        + "group by \"dependents\".\"empid\", \"emps\".\"deptno\"";
+    final String query = "select \"emps\".\"deptno\", count(\"salary\") as s\n"
+        + "from \"emps\"\n"
+        + "join \"dependents\" on (\"emps\".\"empid\" = \"dependents\".\"empid\")\n"
+        + "group by \"dependents\".\"empid\", \"emps\".\"deptno\"";
+    sql(materialize, query).noMat();
   }
 
   @Test public void testJoinAggregateMaterializationAggregateFuncs14() {
-    checkMaterialize(
-        "select \"empid\", \"emps\".\"name\", \"emps\".\"deptno\", \"depts\".\"name\", "
+    sql("select \"empid\", \"emps\".\"name\", \"emps\".\"deptno\", \"depts\".\"name\", "
             + "count(*) as c, sum(\"empid\") as s\n"
             + "from \"emps\" join \"depts\" using (\"deptno\")\n"
             + "where (\"depts\".\"name\" is not null and \"emps\".\"name\" = 'a') or "
@@ -2220,105 +2268,96 @@ public class MaterializationTest {
         "select \"depts\".\"deptno\", sum(\"empid\") as s\n"
             + "from \"emps\" join \"depts\" using (\"deptno\")\n"
             + "where \"depts\".\"name\" is not null and \"emps\".\"name\" = 'a'\n"
-            + "group by \"depts\".\"deptno\"",
-        HR_FKUK_MODEL,
-        CONTAINS_M0);
+            + "group by \"depts\".\"deptno\"")
+        .ok();
   }
 
   @Test public void testJoinMaterialization4() {
-    checkMaterialize(
-        "select \"empid\" \"deptno\" from \"emps\"\n"
+    sql("select \"empid\" \"deptno\" from \"emps\"\n"
             + "join \"depts\" using (\"deptno\")",
         "select \"empid\" \"deptno\" from \"emps\"\n"
-            + "join \"depts\" using (\"deptno\") where \"empid\" = 1",
-        HR_FKUK_MODEL,
-        CalciteAssert.checkResultContains(
+            + "join \"depts\" using (\"deptno\") where \"empid\" = 1")
+        .withResultContains(
             "EnumerableCalc(expr#0=[{inputs}], expr#1=[CAST($t0):INTEGER NOT NULL], expr#2=[1], "
                 + "expr#3=[=($t1, $t2)], deptno=[$t0], $condition=[$t3])\n"
-                + "  EnumerableTableScan(table=[[hr, m0]])"));
+                + "  EnumerableTableScan(table=[[hr, m0]])")
+        .ok();
   }
 
   @Test public void testJoinMaterialization5() {
-    checkMaterialize(
-        "select cast(\"empid\" as BIGINT) from \"emps\"\n"
+    sql("select cast(\"empid\" as BIGINT) from \"emps\"\n"
             + "join \"depts\" using (\"deptno\")",
         "select \"empid\" \"deptno\" from \"emps\"\n"
-            + "join \"depts\" using (\"deptno\") where \"empid\" > 1",
-        HR_FKUK_MODEL,
-        CalciteAssert.checkResultContains(
+            + "join \"depts\" using (\"deptno\") where \"empid\" > 1")
+        .withResultContains(
             "EnumerableCalc(expr#0=[{inputs}], expr#1=[CAST($t0):JavaType(int) NOT NULL], "
                 + "expr#2=[1], expr#3=[>($t1, $t2)], EXPR$0=[$t1], $condition=[$t3])\n"
-                + "  EnumerableTableScan(table=[[hr, m0]])"));
+                + "  EnumerableTableScan(table=[[hr, m0]])")
+        .ok();
   }
 
   @Test public void testJoinMaterialization6() {
-    checkMaterialize(
-        "select cast(\"empid\" as BIGINT) from \"emps\"\n"
+    sql("select cast(\"empid\" as BIGINT) from \"emps\"\n"
             + "join \"depts\" using (\"deptno\")",
         "select \"empid\" \"deptno\" from \"emps\"\n"
-            + "join \"depts\" using (\"deptno\") where \"empid\" = 1",
-        HR_FKUK_MODEL,
-        CalciteAssert.checkResultContains(
+            + "join \"depts\" using (\"deptno\") where \"empid\" = 1")
+        .withResultContains(
             "EnumerableCalc(expr#0=[{inputs}], expr#1=[CAST($t0):JavaType(int) NOT NULL], "
                 + "expr#2=[CAST($t1):INTEGER NOT NULL], expr#3=[1], expr#4=[=($t2, $t3)], "
                 + "EXPR$0=[$t1], $condition=[$t4])\n"
-                + "  EnumerableTableScan(table=[[hr, m0]])"));
+                + "  EnumerableTableScan(table=[[hr, m0]])")
+        .ok();
   }
 
   @Test public void testJoinMaterialization7() {
-    checkMaterialize(
-        "select \"depts\".\"name\"\n"
+    sql("select \"depts\".\"name\"\n"
             + "from \"emps\"\n"
             + "join \"depts\" on (\"emps\".\"deptno\" = \"depts\".\"deptno\")",
         "select \"dependents\".\"empid\"\n"
             + "from \"emps\"\n"
             + "join \"depts\" on (\"emps\".\"deptno\" = \"depts\".\"deptno\")\n"
-            + "join \"dependents\" on (\"depts\".\"name\" = \"dependents\".\"name\")",
-        HR_FKUK_MODEL,
-        CalciteAssert.checkResultContains(
+            + "join \"dependents\" on (\"depts\".\"name\" = \"dependents\".\"name\")")
+        .withResultContains(
             "EnumerableCalc(expr#0..2=[{inputs}], empid=[$t1])\n"
                 + "  EnumerableHashJoin(condition=[=($0, $2)], joinType=[inner])\n"
                 + "    EnumerableCalc(expr#0=[{inputs}], expr#1=[CAST($t0):VARCHAR], name00=[$t1])\n"
                 + "      EnumerableTableScan(table=[[hr, m0]])\n"
                 + "    EnumerableCalc(expr#0..1=[{inputs}], expr#2=[CAST($t1):VARCHAR], empid=[$t0], name0=[$t2])\n"
-                + "      EnumerableTableScan(table=[[hr, dependents]])"));
+                + "      EnumerableTableScan(table=[[hr, dependents]])")
+        .ok();
   }
 
   @Test public void testJoinMaterialization8() {
-    checkMaterialize(
-        "select \"depts\".\"name\"\n"
+    sql("select \"depts\".\"name\"\n"
             + "from \"emps\"\n"
             + "join \"depts\" on (\"emps\".\"deptno\" = \"depts\".\"deptno\")",
         "select \"dependents\".\"empid\"\n"
             + "from \"depts\"\n"
             + "join \"dependents\" on (\"depts\".\"name\" = \"dependents\".\"name\")\n"
-            + "join \"emps\" on (\"emps\".\"deptno\" = \"depts\".\"deptno\")",
-        HR_FKUK_MODEL,
-        CalciteAssert.checkResultContains(
+            + "join \"emps\" on (\"emps\".\"deptno\" = \"depts\".\"deptno\")")
+        .withResultContains(
             "EnumerableCalc(expr#0..2=[{inputs}], empid=[$t0])\n"
                 + "  EnumerableNestedLoopJoin(condition=[=(CAST($1):VARCHAR, CAST($2):VARCHAR)], joinType=[inner])\n"
                 + "    EnumerableTableScan(table=[[hr, dependents]])\n"
-                + "    EnumerableTableScan(table=[[hr, m0]])"));
+                + "    EnumerableTableScan(table=[[hr, m0]])")
+        .ok();
   }
 
   @Test public void testJoinMaterialization9() {
-    checkMaterialize(
-        "select \"depts\".\"name\"\n"
+    sql("select \"depts\".\"name\"\n"
             + "from \"emps\"\n"
             + "join \"depts\" on (\"emps\".\"deptno\" = \"depts\".\"deptno\")",
         "select \"dependents\".\"empid\"\n"
             + "from \"depts\"\n"
             + "join \"dependents\" on (\"depts\".\"name\" = \"dependents\".\"name\")\n"
             + "join \"locations\" on (\"locations\".\"name\" = \"dependents\".\"name\")\n"
-            + "join \"emps\" on (\"emps\".\"deptno\" = \"depts\".\"deptno\")",
-        HR_FKUK_MODEL,
-        CONTAINS_M0);
+            + "join \"emps\" on (\"emps\".\"deptno\" = \"depts\".\"deptno\")")
+        .ok();
   }
 
   @Tag("slow")
   @Test public void testJoinMaterialization10() {
-    checkMaterialize(
-        "select \"depts\".\"deptno\", \"dependents\".\"empid\"\n"
+    sql("select \"depts\".\"deptno\", \"dependents\".\"empid\"\n"
             + "from \"depts\"\n"
             + "join \"dependents\" on (\"depts\".\"name\" = \"dependents\".\"name\")\n"
             + "join \"emps\" on (\"emps\".\"deptno\" = \"depts\".\"deptno\")\n"
@@ -2327,29 +2366,27 @@ public class MaterializationTest {
             + "from \"depts\"\n"
             + "join \"dependents\" on (\"depts\".\"name\" = \"dependents\".\"name\")\n"
             + "join \"emps\" on (\"emps\".\"deptno\" = \"depts\".\"deptno\")\n"
-            + "where \"depts\".\"deptno\" > 10",
-        HR_FKUK_MODEL,
-        CalciteAssert.checkResultContains(
+            + "where \"depts\".\"deptno\" > 10")
+        .withResultContains(
             "EnumerableUnion(all=[true])",
             "EnumerableTableScan(table=[[hr, m0]])",
-            "expr#5=[10], expr#6=[>($t0, $t5)], expr#7=[30], expr#8=[>=($t7, $t0)]"));
+            "expr#5=[10], expr#6=[>($t0, $t5)], expr#7=[30], expr#8=[>=($t7, $t0)]")
+        .ok();
   }
 
   @Test public void testJoinMaterialization11() {
-    checkMaterialize(
-        "select \"empid\" from \"emps\"\n"
+    sql("select \"empid\" from \"emps\"\n"
             + "join \"depts\" using (\"deptno\")",
         "select \"empid\" from \"emps\"\n"
-            + "where \"deptno\" in (select \"deptno\" from \"depts\")",
-        HR_FKUK_MODEL,
-        CalciteAssert.checkResultContains(
-            "PLAN=EnumerableTableScan(table=[[hr, m0]])"));
+            + "where \"deptno\" in (select \"deptno\" from \"depts\")")
+        .withResultContains(
+            "PLAN=EnumerableTableScan(table=[[hr, m0]])")
+        .ok();
   }
 
   @Tag("slow")
   @Test public void testJoinMaterialization12() {
-    checkMaterialize(
-        "select \"empid\", \"emps\".\"name\", \"emps\".\"deptno\", \"depts\".\"name\"\n"
+    sql("select \"empid\", \"emps\".\"name\", \"emps\".\"deptno\", \"depts\".\"name\"\n"
             + "from \"emps\" join \"depts\" using (\"deptno\")\n"
             + "where (\"depts\".\"name\" is not null and \"emps\".\"name\" = 'a') or "
             + "(\"depts\".\"name\" is not null and \"emps\".\"name\" = 'b') or "
@@ -2357,151 +2394,139 @@ public class MaterializationTest {
         "select \"depts\".\"deptno\", \"depts\".\"name\"\n"
             + "from \"emps\" join \"depts\" using (\"deptno\")\n"
             + "where (\"depts\".\"name\" is not null and \"emps\".\"name\" = 'a') or "
-            + "(\"depts\".\"name\" is not null and \"emps\".\"name\" = 'b')",
-        HR_FKUK_MODEL,
-        CONTAINS_M0);
+            + "(\"depts\".\"name\" is not null and \"emps\".\"name\" = 'b')")
+        .ok();
   }
 
   @Test public void testJoinMaterializationUKFK1() {
-    checkMaterialize(
-        "select \"a\".\"empid\" \"deptno\" from\n"
+    sql("select \"a\".\"empid\" \"deptno\" from\n"
             + "(select * from \"emps\" where \"empid\" = 1) \"a\"\n"
             + "join \"depts\" using (\"deptno\")\n"
             + "join \"dependents\" using (\"empid\")",
         "select \"a\".\"empid\" from \n"
             + "(select * from \"emps\" where \"empid\" = 1) \"a\"\n"
-            + "join \"dependents\" using (\"empid\")\n",
-        HR_FKUK_MODEL,
-        CalciteAssert.checkResultContains(
-            "PLAN=EnumerableTableScan(table=[[hr, m0]])"));
+            + "join \"dependents\" using (\"empid\")\n")
+        .withResultContains(
+            "PLAN=EnumerableTableScan(table=[[hr, m0]])")
+        .ok();
   }
 
   @Test public void testJoinMaterializationUKFK2() {
-    checkMaterialize(
-        "select \"a\".\"empid\", \"a\".\"deptno\" from\n"
+    sql("select \"a\".\"empid\", \"a\".\"deptno\" from\n"
             + "(select * from \"emps\" where \"empid\" = 1) \"a\"\n"
             + "join \"depts\" using (\"deptno\")\n"
             + "join \"dependents\" using (\"empid\")",
         "select \"a\".\"empid\" from \n"
             + "(select * from \"emps\" where \"empid\" = 1) \"a\"\n"
-            + "join \"dependents\" using (\"empid\")\n",
-        HR_FKUK_MODEL,
-        CalciteAssert.checkResultContains(
+            + "join \"dependents\" using (\"empid\")\n")
+        .withResultContains(
             "EnumerableCalc(expr#0..1=[{inputs}], empid=[$t0])\n"
-                + "  EnumerableTableScan(table=[[hr, m0]])"));
+                + "  EnumerableTableScan(table=[[hr, m0]])")
+        .ok();
   }
 
   @Test public void testJoinMaterializationUKFK3() {
-    checkNoMaterialize(
-        "select \"a\".\"empid\", \"a\".\"deptno\" from\n"
+    sql("select \"a\".\"empid\", \"a\".\"deptno\" from\n"
             + "(select * from \"emps\" where \"empid\" = 1) \"a\"\n"
             + "join \"depts\" using (\"deptno\")\n"
             + "join \"dependents\" using (\"empid\")",
         "select \"a\".\"name\" from \n"
             + "(select * from \"emps\" where \"empid\" = 1) \"a\"\n"
-            + "join \"dependents\" using (\"empid\")\n",
-        HR_FKUK_MODEL);
+            + "join \"dependents\" using (\"empid\")\n")
+        .noMat();
   }
 
   @Test public void testJoinMaterializationUKFK4() {
-    checkMaterialize(
-        "select \"empid\" \"deptno\" from\n"
+    sql("select \"empid\" \"deptno\" from\n"
             + "(select * from \"emps\" where \"empid\" = 1)\n"
             + "join \"depts\" using (\"deptno\")",
-        "select \"empid\" from \"emps\" where \"empid\" = 1\n",
-        HR_FKUK_MODEL,
-        CalciteAssert.checkResultContains(
-            "PLAN=EnumerableTableScan(table=[[hr, m0]])"));
+        "select \"empid\" from \"emps\" where \"empid\" = 1\n")
+        .withResultContains(
+            "PLAN=EnumerableTableScan(table=[[hr, m0]])")
+        .ok();
   }
 
   @Tag("slow")
   @Test public void testJoinMaterializationUKFK5() {
-    checkMaterialize(
-        "select \"emps\".\"empid\", \"emps\".\"deptno\" from \"emps\"\n"
+    sql("select \"emps\".\"empid\", \"emps\".\"deptno\" from \"emps\"\n"
             + "join \"depts\" using (\"deptno\")\n"
             + "join \"dependents\" using (\"empid\")"
             + "where \"emps\".\"empid\" = 1",
         "select \"emps\".\"empid\" from \"emps\"\n"
             + "join \"dependents\" using (\"empid\")\n"
-            + "where \"emps\".\"empid\" = 1",
-        HR_FKUK_MODEL,
-        CalciteAssert.checkResultContains(
+            + "where \"emps\".\"empid\" = 1")
+        .withResultContains(
             "EnumerableCalc(expr#0..1=[{inputs}], empid=[$t0])\n"
-                + "  EnumerableTableScan(table=[[hr, m0]])"));
+                + "  EnumerableTableScan(table=[[hr, m0]])")
+        .ok();
   }
 
   @Tag("slow")
   @Test public void testJoinMaterializationUKFK6() {
-    checkMaterialize(
-        "select \"emps\".\"empid\", \"emps\".\"deptno\" from \"emps\"\n"
+    sql("select \"emps\".\"empid\", \"emps\".\"deptno\" from \"emps\"\n"
             + "join \"depts\" \"a\" on (\"emps\".\"deptno\"=\"a\".\"deptno\")\n"
             + "join \"depts\" \"b\" on (\"emps\".\"deptno\"=\"b\".\"deptno\")\n"
             + "join \"dependents\" using (\"empid\")"
             + "where \"emps\".\"empid\" = 1",
         "select \"emps\".\"empid\" from \"emps\"\n"
             + "join \"dependents\" using (\"empid\")\n"
-            + "where \"emps\".\"empid\" = 1",
-        HR_FKUK_MODEL,
-        CalciteAssert.checkResultContains(
+            + "where \"emps\".\"empid\" = 1")
+        .withResultContains(
             "EnumerableCalc(expr#0..1=[{inputs}], empid=[$t0])\n"
-                + "  EnumerableTableScan(table=[[hr, m0]])"));
+                + "  EnumerableTableScan(table=[[hr, m0]])")
+        .ok();
   }
 
   @Test public void testJoinMaterializationUKFK7() {
-    checkNoMaterialize(
-        "select \"emps\".\"empid\", \"emps\".\"deptno\" from \"emps\"\n"
+    sql("select \"emps\".\"empid\", \"emps\".\"deptno\" from \"emps\"\n"
             + "join \"depts\" \"a\" on (\"emps\".\"name\"=\"a\".\"name\")\n"
             + "join \"depts\" \"b\" on (\"emps\".\"name\"=\"b\".\"name\")\n"
             + "join \"dependents\" using (\"empid\")"
             + "where \"emps\".\"empid\" = 1",
         "select \"emps\".\"empid\" from \"emps\"\n"
             + "join \"dependents\" using (\"empid\")\n"
-            + "where \"emps\".\"empid\" = 1",
-        HR_FKUK_MODEL);
+            + "where \"emps\".\"empid\" = 1")
+        .noMat();
   }
 
   @Test public void testJoinMaterializationUKFK8() {
-    checkNoMaterialize(
-        "select \"emps\".\"empid\", \"emps\".\"deptno\" from \"emps\"\n"
+    sql("select \"emps\".\"empid\", \"emps\".\"deptno\" from \"emps\"\n"
             + "join \"depts\" \"a\" on (\"emps\".\"deptno\"=\"a\".\"deptno\")\n"
             + "join \"depts\" \"b\" on (\"emps\".\"name\"=\"b\".\"name\")\n"
             + "join \"dependents\" using (\"empid\")"
             + "where \"emps\".\"empid\" = 1",
         "select \"emps\".\"empid\" from \"emps\"\n"
             + "join \"dependents\" using (\"empid\")\n"
-            + "where \"emps\".\"empid\" = 1",
-        HR_FKUK_MODEL);
+            + "where \"emps\".\"empid\" = 1")
+        .noMat();
   }
 
   @Tag("slow")
   @Test public void testJoinMaterializationUKFK9() {
-    checkMaterialize(
-        "select * from \"emps\"\n"
+    sql("select * from \"emps\"\n"
             + "join \"dependents\" using (\"empid\")",
         "select \"emps\".\"empid\", \"dependents\".\"empid\", \"emps\".\"deptno\"\n"
             + "from \"emps\"\n"
             + "join \"dependents\" using (\"empid\")"
             + "join \"depts\" \"a\" on (\"emps\".\"deptno\"=\"a\".\"deptno\")\n"
-            + "where \"emps\".\"name\" = 'Bill'",
-        HR_FKUK_MODEL,
-        CalciteAssert.checkResultContains(
-            "EnumerableTableScan(table=[[hr, m0]])"));
+            + "where \"emps\".\"name\" = 'Bill'")
+        .withResultContains(
+            "EnumerableTableScan(table=[[hr, m0]])")
+        .ok();
   }
 
   @Test public void testViewMaterialization() {
-    checkThatMaterialize(
-        "select \"depts\".\"name\"\n"
+    sql("select \"depts\".\"name\"\n"
             + "from \"emps\"\n"
             + "join \"depts\" on (\"emps\".\"deptno\" = \"depts\".\"deptno\")",
         "select \"depts\".\"name\"\n"
             + "from \"depts\"\n"
-            + "join \"emps\" on (\"emps\".\"deptno\" = \"depts\".\"deptno\")",
-        "matview",
-        true,
-        HR_FKUK_MODEL,
-        CalciteAssert.checkResultContains(
-            "EnumerableValues(tuples=[[{ 'noname' }]])"),
-        RuleSets.ofList(ImmutableList.of()))
+            + "join \"emps\" on (\"emps\".\"deptno\" = \"depts\".\"deptno\")")
+        .withViewName("matview")
+        .withExisting(true)
+        .withResultContains(
+            "EnumerableValues(tuples=[[{ 'noname' }]])")
+        .that()
         .returnsValue("noname");
   }
 
@@ -2511,9 +2536,10 @@ public class MaterializationTest {
         + "  select max(\"empid\") from \"emps\"\n"
         + "  where \"deptno\" = e1.\"deptno\")";
     final String m = "select \"empid\", \"deptno\" from \"emps\"\n";
-    checkMaterialize(m, q, HR_FKUK_MODEL,
+    sql(m, q).withChecker(
         CalciteAssert.checkResultContains(
-            "EnumerableTableScan(table=[[hr, m0]])", 1));
+            "EnumerableTableScan(table=[[hr, m0]])", 1))
+        .ok();
   }
 
   @Test public void testTableModify() {
@@ -2610,9 +2636,10 @@ public class MaterializationTest {
         + "from (select * from \"emps\" where \"empid\" < 300)\n"
         + "join (select * from \"emps\" where \"empid\" < 200) using (\"empid\")";
     String m = "select * from \"emps\" where \"empid\" < 500";
-    checkMaterialize(m, q, HR_FKUK_MODEL,
+    sql(m, q).withChecker(
         CalciteAssert.checkResultContains(
-            "EnumerableTableScan(table=[[hr, m0]])", 2));
+            "EnumerableTableScan(table=[[hr, m0]])", 2))
+        .ok();
   }
 
   @Test public void testMultiMaterializationMultiUsage() {
@@ -2672,75 +2699,71 @@ public class MaterializationTest {
   @Test public void testAggregateMaterializationOnCountDistinctQuery1() {
     // The column empid is already unique, thus DISTINCT is not
     // in the COUNT of the resulting rewriting
-    checkMaterialize(
-        "select \"deptno\", \"empid\", \"salary\"\n"
+    sql("select \"deptno\", \"empid\", \"salary\"\n"
             + "from \"emps\"\n"
             + "group by \"deptno\", \"empid\", \"salary\"",
         "select \"deptno\", count(distinct \"empid\") as c from (\n"
             + "select \"deptno\", \"empid\"\n"
             + "from \"emps\"\n"
             + "group by \"deptno\", \"empid\")\n"
-            + "group by \"deptno\"",
-        HR_FKUK_MODEL,
-        CalciteAssert.checkResultContains(
+            + "group by \"deptno\"")
+        .withResultContains(
             "EnumerableAggregate(group=[{0}], C=[COUNT($1)])\n"
-                + "  EnumerableTableScan(table=[[hr, m0]]"));
+                + "  EnumerableTableScan(table=[[hr, m0]]")
+        .ok();
   }
 
   @Test public void testAggregateMaterializationOnCountDistinctQuery2() {
     // The column empid is already unique, thus DISTINCT is not
     // in the COUNT of the resulting rewriting
-    checkMaterialize(
-        "select \"deptno\", \"salary\", \"empid\"\n"
+    sql("select \"deptno\", \"salary\", \"empid\"\n"
             + "from \"emps\"\n"
             + "group by \"deptno\", \"salary\", \"empid\"",
         "select \"deptno\", count(distinct \"empid\") as c from (\n"
             + "select \"deptno\", \"empid\"\n"
             + "from \"emps\"\n"
             + "group by \"deptno\", \"empid\")\n"
-            + "group by \"deptno\"",
-        HR_FKUK_MODEL,
-        CalciteAssert.checkResultContains(
+            + "group by \"deptno\"")
+        .withResultContains(
             "EnumerableAggregate(group=[{0}], C=[COUNT($2)])\n"
-                + "  EnumerableTableScan(table=[[hr, m0]]"));
+                + "  EnumerableTableScan(table=[[hr, m0]]")
+        .ok();
   }
 
   @Test public void testAggregateMaterializationOnCountDistinctQuery3() {
     // The column salary is not unique, thus we end up with
     // a different rewriting
-    checkMaterialize(
-        "select \"deptno\", \"empid\", \"salary\"\n"
+    sql("select \"deptno\", \"empid\", \"salary\"\n"
             + "from \"emps\"\n"
             + "group by \"deptno\", \"empid\", \"salary\"",
         "select \"deptno\", count(distinct \"salary\") from (\n"
             + "select \"deptno\", \"salary\"\n"
             + "from \"emps\"\n"
             + "group by \"deptno\", \"salary\")\n"
-            + "group by \"deptno\"",
-        HR_FKUK_MODEL,
-        CalciteAssert.checkResultContains(
+            + "group by \"deptno\"")
+        .withResultContains(
             "EnumerableAggregate(group=[{0}], EXPR$1=[COUNT($1)])\n"
                 + "  EnumerableAggregate(group=[{0, 2}])\n"
-                + "    EnumerableTableScan(table=[[hr, m0]]"));
+                + "    EnumerableTableScan(table=[[hr, m0]]")
+        .ok();
   }
 
   @Test public void testAggregateMaterializationOnCountDistinctQuery4() {
     // Although there is no DISTINCT in the COUNT, this is
     // equivalent to previous query
-    checkMaterialize(
-        "select \"deptno\", \"salary\", \"empid\"\n"
+    sql("select \"deptno\", \"salary\", \"empid\"\n"
             + "from \"emps\"\n"
             + "group by \"deptno\", \"salary\", \"empid\"",
         "select \"deptno\", count(\"salary\") from (\n"
             + "select \"deptno\", \"salary\"\n"
             + "from \"emps\"\n"
             + "group by \"deptno\", \"salary\")\n"
-            + "group by \"deptno\"",
-        HR_FKUK_MODEL,
-        CalciteAssert.checkResultContains(
+            + "group by \"deptno\"")
+        .withResultContains(
             "EnumerableAggregate(group=[{0}], EXPR$1=[COUNT()])\n"
                 + "  EnumerableAggregate(group=[{0, 1}])\n"
-                + "    EnumerableTableScan(table=[[hr, m0]]"));
+                + "    EnumerableTableScan(table=[[hr, m0]]")
+        .ok();
   }
 
   @Test public void testMaterializationSubstitution() {
@@ -2825,26 +2848,25 @@ public class MaterializationTest {
             + "  join\n"
             + "  \"depts\" \"y\"\n"
             + "  on \"x\".\"deptno\"=\"y\".\"deptno\"\n";
-    checkMaterialize(sql, sql);
+    sql(sql, sql).ok();
   }
 
   @Test public void testUnionAllToUnionAll() {
     String sql0 = "select * from \"emps\" where \"empid\" < 300";
     String sql1 = "select * from \"emps\" where \"empid\" > 200";
-    checkMaterialize(sql0 + " union all " + sql1, sql1 + " union all " + sql0);
+    sql(sql0 + " union all " + sql1, sql1 + " union all " + sql0).ok();
   }
 
   @Test public void testUnionDistinctToUnionDistinct() {
     String sql0 = "select * from \"emps\" where \"empid\" < 300";
     String sql1 = "select * from \"emps\" where \"empid\" > 200";
-    checkMaterialize(sql0 + " union " + sql1, sql1 + " union " + sql0);
+    sql(sql0 + " union " + sql1, sql1 + " union " + sql0).ok();
   }
 
   @Test public void testUnionDistinctToUnionAll() {
     String sql0 = "select * from \"emps\" where \"empid\" < 300";
     String sql1 = "select * from \"emps\" where \"empid\" > 200";
-    checkNoMaterialize(sql0 + " union " + sql1, sql0 + " union all " + sql1,
-        HR_FKUK_MODEL);
+    sql(sql0 + " union " + sql1, sql0 + " union all " + sql1).noMat();
   }
 
   @Test public void testUnionOnCalcsToUnion() {
@@ -2864,7 +2886,7 @@ public class MaterializationTest {
         + "select \"deptno\", \"salary\" * 2\n"
         + "from \"emps\"\n"
         + "where \"empid\" < 100 and \"salary\" > 100";
-    checkMaterialize(mv, query);
+    sql(mv, query).ok();
   }
 
   @Test public void testIntersectToIntersect0() {
@@ -2876,7 +2898,7 @@ public class MaterializationTest {
         + "select \"deptno\" from \"depts\"\n"
         + "intersect\n"
         + "select \"deptno\" from \"emps\"";
-    checkMaterialize(mv, query, true);
+    sql(mv, query).withOnlyBySubstitution(true).ok();
   }
 
   @Test public void testIntersectToIntersect1() {
@@ -2888,7 +2910,7 @@ public class MaterializationTest {
         + "select \"deptno\" from \"depts\"\n"
         + "intersect all\n"
         + "select \"deptno\" from \"emps\"";
-    checkMaterialize(mv, query, true);
+    sql(mv, query).withOnlyBySubstitution(true).ok();
   }
 
   @Test public void testIntersectToCalcOnIntersect() {
@@ -2902,7 +2924,7 @@ public class MaterializationTest {
         + "select \"name\",\"deptno\" from \"depts\"\n"
         + "intersect all\n"
         + "select \"name\",\"deptno\" from \"emps\"";
-    checkMaterialize(mv, query, true);
+    sql(mv, query).withOnlyBySubstitution(true).ok();
   }
 
   private static <E> List<List<List<E>>> list3(E[][][] as) {
@@ -2996,4 +3018,61 @@ public class MaterializationTest {
       return Smalls.strView("noname");
     }
   }
+
+  /** Fluent class that contains information necessary to run a test. */
+  private interface Sql {
+    default void ok() {
+      that().sameResultWithMaterializationsDisabled();
+    }
+
+    default CalciteAssert.AssertQuery that() {
+      return checkThatMaterialize_(getMaterialize(), getQuery(), getViewName(),
+          isExisting(), getModel(), getChecker(), getRuleSet(),
+          isOnlyBySubstitution());
+    }
+
+    @ImmutableBeans.Property
+    @ImmutableBeans.BooleanDefault(false)
+    boolean isExisting();
+    Sql withExisting(boolean existing);
+
+    default void noMat() {
+      checkNoMaterialize_(getMaterialize(), getQuery(),
+          getModel(), isOnlyBySubstitution());
+    }
+
+    default Sql withResultContains(String... expected) {
+      return withChecker(CalciteAssert.checkResultContains(expected));
+    }
+
+    @ImmutableBeans.Property
+    String getMaterialize();
+    Sql withMaterialize(String materialize);
+
+    @ImmutableBeans.Property
+    String getQuery();
+    Sql withQuery(String query);
+
+    @ImmutableBeans.Property
+    @ImmutableBeans.BooleanDefault(false)
+    boolean isOnlyBySubstitution();
+    Sql withOnlyBySubstitution(boolean onlyBySubstitution);
+
+    @ImmutableBeans.Property
+    String getModel();
+    Sql withModel(String model);
+
+    @ImmutableBeans.Property
+    Consumer<ResultSet> getChecker();
+    Sql withChecker(Consumer<ResultSet> explainChecker);
+
+    @ImmutableBeans.Property
+    RuleSet getRuleSet();
+    Sql withRuleSet(RuleSet ruleSet);
+
+    @ImmutableBeans.Property
+    @ImmutableBeans.StringDefault("m0")
+    String getViewName();
+    Sql withViewName(String viewName);
+  }
 }