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 2014/10/28 18:50:55 UTC

[7/9] Various lattice improvements.

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/41215c2f/core/src/test/java/net/hydromatic/optiq/test/LatticeTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/net/hydromatic/optiq/test/LatticeTest.java b/core/src/test/java/net/hydromatic/optiq/test/LatticeTest.java
index 0d7f8f2..e9c1ac4 100644
--- a/core/src/test/java/net/hydromatic/optiq/test/LatticeTest.java
+++ b/core/src/test/java/net/hydromatic/optiq/test/LatticeTest.java
@@ -16,6 +16,7 @@
  */
 package net.hydromatic.optiq.test;
 
+import net.hydromatic.optiq.materialize.MaterializationService;
 import net.hydromatic.optiq.runtime.Hook;
 
 import org.eigenbase.rel.RelNode;
@@ -25,11 +26,16 @@ import org.eigenbase.util.Util;
 
 import com.google.common.base.Function;
 import com.google.common.collect.ImmutableList;
+import com.google.common.collect.Lists;
 
 import org.junit.Ignore;
 import org.junit.Test;
 
 import java.io.IOException;
+import java.sql.Connection;
+import java.sql.DriverManager;
+import java.sql.ResultSet;
+import java.sql.SQLException;
 import java.util.Arrays;
 import java.util.List;
 import java.util.concurrent.atomic.AtomicInteger;
@@ -246,8 +252,7 @@ public class LatticeTest {
 
   /** Tests a model with pre-defined tiles. */
   @Test public void testLatticeWithPreDefinedTiles() {
-    foodmartModel(
-        " auto: false,\n"
+    foodmartModel(" auto: false,\n"
         + "  defaultMeasures: [ {\n"
         + "    agg: 'count'\n"
         + "  } ],\n"
@@ -255,8 +260,7 @@ public class LatticeTest {
         + "    dimensions: [ 'the_year', ['t', 'quarter'] ],\n"
         + "    measures: [ ]\n"
         + "  } ]\n")
-        .query(
-            "select distinct t.\"the_year\", t.\"quarter\"\n"
+        .query("select distinct t.\"the_year\", t.\"quarter\"\n"
             + "from \"foodmart\".\"sales_fact_1997\" as s\n"
             + "join \"foodmart\".\"time_by_day\" as t using (\"time_id\")\n")
       .enableMaterializations(true)
@@ -268,25 +272,8 @@ public class LatticeTest {
   /** A query that uses a pre-defined aggregate table, at the same
    *  granularity but fewer calls to aggregate functions. */
   @Test public void testLatticeWithPreDefinedTilesFewerMeasures() {
-    foodmartModel(
-        " auto: false,\n"
-        + "  defaultMeasures: [ {\n"
-        + "    agg: 'count'\n"
-        + "  } ],\n"
-        + "  tiles: [ {\n"
-        + "    dimensions: [ 'the_year', ['t', 'quarter'] ],\n"
-        + "    measures: [ {\n"
-        + "      agg: 'sum',\n"
-        + "      args: 'unit_sales'\n"
-        + "    }, {\n"
-        + "      agg: 'sum',\n"
-        + "      args: 'store_sales'\n"
-        + "    }, {\n"
-        + "      agg: 'count'\n"
-        + "    } ]\n"
-        + "  } ]\n")
-        .query(
-            "select t.\"the_year\", t.\"quarter\", count(*) as c\n"
+    foodmartModelWithOneTile()
+        .query("select t.\"the_year\", t.\"quarter\", count(*) as c\n"
             + "from \"foodmart\".\"sales_fact_1997\" as s\n"
             + "join \"foodmart\".\"time_by_day\" as t using (\"time_id\")\n"
             + "group by t.\"the_year\", t.\"quarter\"")
@@ -305,25 +292,8 @@ public class LatticeTest {
    * granularity. Includes a measure computed from a grouping column, a measure
    * based on COUNT rolled up using SUM, and an expression on a measure. */
   @Test public void testLatticeWithPreDefinedTilesRollUp() {
-    foodmartModel(
-        " auto: false,\n"
-        + "  defaultMeasures: [ {\n"
-        + "    agg: 'count'\n"
-        + "  } ],\n"
-        + "  tiles: [ {\n"
-        + "    dimensions: [ 'the_year', ['t', 'quarter'] ],\n"
-        + "    measures: [ {\n"
-        + "      agg: 'sum',\n"
-        + "      args: 'unit_sales'\n"
-        + "    }, {\n"
-        + "      agg: 'sum',\n"
-        + "      args: 'store_sales'\n"
-        + "    }, {\n"
-        + "      agg: 'count'\n"
-        + "    } ]\n"
-        + "  } ]\n")
-        .query(
-            "select t.\"the_year\",\n"
+    foodmartModelWithOneTile()
+        .query("select t.\"the_year\",\n"
             + "  count(*) as c,\n"
             + "  min(\"quarter\") as q,\n"
             + "  sum(\"unit_sales\") * 10 as us\n"
@@ -333,7 +303,7 @@ public class LatticeTest {
       .enableMaterializations(true)
       .explainContains(
           "EnumerableCalcRel(expr#0..3=[{inputs}], expr#4=[10], expr#5=[*($t3, $t4)], proj#0..2=[{exprs}], US=[$t5])\n"
-          + "  EnumerableAggregateRel(group=[{0}], agg#0=[$SUM0($2)], Q=[MIN($1)], agg#2=[$SUM0($4)])\n"
+          + "  EnumerableAggregateRel(group=[{0}], C=[$SUM0($2)], Q=[MIN($1)], agg#2=[$SUM0($4)])\n"
           + "    EnumerableTableAccessRel(table=[[adhoc, m{27, 31}")
       .returnsUnordered("the_year=1997; C=86837; Q=Q1; US=2667730.0000")
       .sameResultWithMaterializationsDisabled();
@@ -347,9 +317,12 @@ public class LatticeTest {
    * "Use optimization algorithm to suggest which tiles of a lattice to
    * materialize"</a>. */
   @Test public void testTileAlgorithm() {
+    MaterializationService.setThreadLocal();
+    MaterializationService.instance().clear();
     foodmartModel(
         " auto: false,\n"
         + "  algorithm: true,\n"
+        + "  algorithmMaxMillis: -1,\n"
         + "  rowCountEstimate: 86000,\n"
         + "  defaultMeasures: [ {\n"
         + "      agg: 'sum',\n"
@@ -369,8 +342,8 @@ public class LatticeTest {
             + "from \"foodmart\".\"sales_fact_1997\" as s\n"
             + "join \"foodmart\".\"time_by_day\" as t using (\"time_id\")\n")
         .enableMaterializations(true)
-        .explainContains("EnumerableAggregateRel(group=[{3, 4}])\n"
-            + "  EnumerableTableAccessRel(table=[[adhoc, m{7, 16, 25, 27, 31, 37}]])")
+        .explainContains("EnumerableAggregateRel(group=[{2, 3}])\n"
+            + "  EnumerableTableAccessRel(table=[[adhoc, m{16, 17, 27, 31}]])")
         .returnsUnordered("the_year=1997; quarter=Q1",
             "the_year=1997; quarter=Q2",
             "the_year=1997; quarter=Q3",
@@ -378,6 +351,102 @@ public class LatticeTest {
         .returnsCount(4);
   }
 
+  /** Tests a query that uses no columns from the fact table. */
+  @Test public void testGroupByEmpty() {
+    foodmartModel()
+        .query("select count(*) as c from \"foodmart\".\"sales_fact_1997\"")
+        .enableMaterializations(true)
+        .returnsUnordered("C=86837");
+  }
+
+  /** Calls {@link #testDistinctCount()} followed by
+   * {@link #testGroupByEmpty()}. */
+  @Test public void testGroupByEmptyWithPrelude() {
+    testDistinctCount();
+    testGroupByEmpty();
+  }
+
+  /** Tests a query that uses no dimension columns and one measure column. */
+  @Test public void testGroupByEmpty2() {
+    foodmartModel()
+        .query("select sum(\"unit_sales\") as s\n"
+            + "from \"foodmart\".\"sales_fact_1997\"")
+        .enableMaterializations(true)
+        .returnsUnordered("S=266773.0000");
+  }
+
+  /** Tests that two queries of the same dimensionality that use different
+   * measures can use the same materialization. */
+  @Test public void testGroupByEmpty3() {
+    final List<String> mats = Lists.newArrayList();
+    final Function<String, Void> handler =
+        new Function<String, Void>() {
+          public Void apply(String materializationName) {
+            mats.add(materializationName);
+            return null;
+          }
+        };
+    final OptiqAssert.AssertThat that = foodmartModel().pooled();
+    that.query("select sum(\"unit_sales\") as s, count(*) as c\n"
+            + "from \"foodmart\".\"sales_fact_1997\"")
+        .withHook(Hook.CREATE_MATERIALIZATION, handler)
+        .enableMaterializations(true)
+        .explainContains("EnumerableTableAccessRel(table=[[adhoc, m{}]])")
+        .returnsUnordered("S=266773.0000; C=86837");
+    assertThat(mats.toString(), mats.size(), equalTo(2));
+
+    // A similar query can use the same materialization.
+    that.query("select sum(\"unit_sales\") as s\n"
+        + "from \"foodmart\".\"sales_fact_1997\"")
+        .withHook(Hook.CREATE_MATERIALIZATION, handler)
+        .enableMaterializations(true)
+        .returnsUnordered("S=266773.0000");
+    assertThat(mats.toString(), mats.size(), equalTo(2));
+  }
+
+  /** Rolling up SUM. */
+  @Test public void testSum() {
+    foodmartModelWithOneTile()
+        .query("select sum(\"unit_sales\") as c\n"
+            + "from \"foodmart\".\"sales_fact_1997\"\n"
+            + "group by \"product_id\"\n"
+            + "order by 1 desc limit 1")
+        .enableMaterializations(true)
+        .returnsUnordered("C=267.0000");
+  }
+
+  /** Tests a distinct-count query.
+   *
+   * <p>We can't just roll up count(distinct ...) as we do count(...), but we
+   * can still use the aggregate table if we're smart. */
+  @Test public void testDistinctCount() {
+    foodmartModelWithOneTile()
+        .query("select count(distinct \"quarter\") as c\n"
+            + "from \"foodmart\".\"sales_fact_1997\"\n"
+            + "join \"foodmart\".\"time_by_day\" using (\"time_id\")\n"
+            + "group by \"the_year\"")
+        .enableMaterializations(true)
+        .explainContains("EnumerableCalcRel(expr#0..1=[{inputs}], C=[$t1])\n"
+            + "  EnumerableAggregateRel(group=[{0}], C=[COUNT($1)])\n"
+            + "    EnumerableCalcRel(expr#0..4=[{inputs}], proj#0..1=[{exprs}])\n"
+            + "      EnumerableTableAccessRel(table=[[adhoc, m{27, 31}]])")
+        .returnsUnordered("C=4");
+  }
+
+  @Test public void testDistinctCount2() {
+    foodmartModelWithOneTile()
+        .query("select count(distinct \"the_year\") as c\n"
+            + "from \"foodmart\".\"sales_fact_1997\"\n"
+            + "join \"foodmart\".\"time_by_day\" using (\"time_id\")\n"
+            + "group by \"the_year\"")
+        .enableMaterializations(true)
+        .explainContains("EnumerableCalcRel(expr#0..1=[{inputs}], C=[$t1])\n"
+            + "  EnumerableAggregateRel(group=[{0}], C=[COUNT($0)])\n"
+            + "    EnumerableAggregateRel(group=[{0}])\n"
+            + "      EnumerableTableAccessRel(table=[[adhoc, m{27, 31}]])")
+        .returnsUnordered("C=1");
+  }
+
   /** Runs all queries against the Foodmart schema, using a lattice.
    *
    * <p>Disabled for normal runs, because it is slow. */
@@ -407,23 +476,7 @@ public class LatticeTest {
     if (query == null) {
       return;
     }
-    foodmartModel(
-        " auto: false,\n"
-        + "  defaultMeasures: [ {\n"
-        + "    agg: 'count'\n"
-        + "  } ],\n"
-        + "  tiles: [ {\n"
-        + "    dimensions: [ 'the_year', ['t', 'quarter'] ],\n"
-        + "    measures: [ {\n"
-        + "      agg: 'sum',\n"
-        + "      args: 'unit_sales'\n"
-        + "    }, {\n"
-        + "      agg: 'sum',\n"
-        + "      args: 'store_sales'\n"
-        + "    }, {\n"
-        + "      agg: 'count'\n"
-        + "    } ]\n"
-        + "  } ]\n")
+    foodmartModelWithOneTile()
         .withSchema("foodmart")
         .query(query.sql)
       .sameResultWithMaterializationsDisabled();
@@ -467,6 +520,36 @@ public class LatticeTest {
         + "join \"foodmart\".\"product_class\" as \"pc\" on \"p\".\"product_class_id\" = \"pc\".\"product_class_id\"",
         extras);
   }
+
+  private OptiqAssert.AssertThat foodmartModelWithOneTile() {
+    return foodmartModel(
+        " auto: false,\n"
+        + "  defaultMeasures: [ {\n"
+        + "    agg: 'count'\n"
+        + "  } ],\n"
+        + "  tiles: [ {\n"
+        + "    dimensions: [ 'the_year', ['t', 'quarter'] ],\n"
+        + "    measures: [ {\n"
+        + "      agg: 'sum',\n"
+        + "      args: 'unit_sales'\n"
+        + "    }, {\n"
+        + "      agg: 'sum',\n"
+        + "      args: 'store_sales'\n"
+        + "    }, {\n"
+        + "      agg: 'count'\n"
+        + "    } ]\n"
+        + "  } ]\n");
+  }
+
+  // Just for debugging.
+  private static void runJdbc() throws SQLException {
+    final Connection connection = DriverManager.getConnection(
+        "jdbc:calcite:model=core/src/test/resources/mysql-foodmart-lattice-model.json");
+    final ResultSet resultSet = connection.createStatement()
+        .executeQuery("select * from \"adhoc\".\"m{27, 31}\"");
+    System.out.println(OptiqAssert.toString(resultSet));
+    connection.close();
+  }
 }
 
 // End LatticeTest.java

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/41215c2f/core/src/test/java/net/hydromatic/optiq/test/MaterializationTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/net/hydromatic/optiq/test/MaterializationTest.java b/core/src/test/java/net/hydromatic/optiq/test/MaterializationTest.java
index 86bf601..3b6827b 100644
--- a/core/src/test/java/net/hydromatic/optiq/test/MaterializationTest.java
+++ b/core/src/test/java/net/hydromatic/optiq/test/MaterializationTest.java
@@ -247,7 +247,7 @@ public class MaterializationTest {
         JdbcTest.HR_MODEL,
         OptiqAssert.checkResultContains(
             "EnumerableCalcRel(expr#0..1=[{inputs}], expr#2=[1], expr#3=[+($t1, $t2)], C=[$t3], deptno=[$t0])\n"
-            + "  EnumerableAggregateRel(group=[{1}], agg#0=[SUM($2)])\n"
+            + "  EnumerableAggregateRel(group=[{1}], agg#0=[$SUM0($2)])\n"
             + "    EnumerableTableAccessRel(table=[[hr, m0]])"));
   }
 

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/41215c2f/core/src/test/java/net/hydromatic/optiq/test/ModelTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/net/hydromatic/optiq/test/ModelTest.java b/core/src/test/java/net/hydromatic/optiq/test/ModelTest.java
index ec3aea4..d1ca838 100644
--- a/core/src/test/java/net/hydromatic/optiq/test/ModelTest.java
+++ b/core/src/test/java/net/hydromatic/optiq/test/ModelTest.java
@@ -26,6 +26,7 @@ import org.junit.Test;
 import java.io.IOException;
 import java.util.*;
 
+import static org.hamcrest.CoreMatchers.equalTo;
 import static org.junit.Assert.*;
 
 /**
@@ -176,7 +177,7 @@ public class ModelTest {
         + "SemiMutableSchema");
   }
 
-  /** Tests a model containing a lattice. */
+  /** Tests a model containing a lattice and some views. */
   @Test public void testReadLattice() throws IOException {
     final ObjectMapper mapper = mapper();
     JsonRoot root = mapper.readValue(
@@ -201,12 +202,26 @@ public class ModelTest {
         + "               name: 'time_id'\n"
         + "             }\n"
         + "           ]\n"
+        + "         },\n"
+        + "         {\n"
+        + "           name: 'V',\n"
+        + "           type: 'view',\n"
+        + "           sql: 'values (1)'\n"
+        + "         },\n"
+        + "         {\n"
+        + "           name: 'V2',\n"
+        + "           type: 'view',\n"
+        + "           sql: [ 'values (1)', '(2)' ]\n"
         + "         }\n"
         + "       ],\n"
         + "       lattices: [\n"
         + "         {\n"
         + "           name: 'SalesStar',\n"
         + "           sql: 'select * from sales_fact_1997'\n"
+        + "         },\n"
+        + "         {\n"
+        + "           name: 'SalesStar2',\n"
+        + "           sql: [ 'select *', 'from sales_fact_1997' ]\n"
         + "         }\n"
         + "       ]\n"
         + "     }\n"
@@ -217,10 +232,55 @@ public class ModelTest {
     assertEquals(1, root.schemas.size());
     final JsonMapSchema schema = (JsonMapSchema) root.schemas.get(0);
     assertEquals("FoodMart", schema.name);
-    assertEquals(1, schema.lattices.size());
+    assertEquals(2, schema.lattices.size());
     final JsonLattice lattice0 = schema.lattices.get(0);
     assertEquals("SalesStar", lattice0.name);
-    assertEquals("select * from sales_fact_1997", lattice0.sql);
+    assertEquals("select * from sales_fact_1997", lattice0.getSql());
+    final JsonLattice lattice1 = schema.lattices.get(1);
+    assertEquals("SalesStar2", lattice1.name);
+    assertEquals("select *\nfrom sales_fact_1997\n", lattice1.getSql());
+    assertEquals(4, schema.tables.size());
+    final JsonTable table1 = schema.tables.get(1);
+    assertTrue(!(table1 instanceof JsonView));
+    final JsonTable table2 = schema.tables.get(2);
+    assertTrue(table2 instanceof JsonView);
+    assertThat(((JsonView) table2).getSql(), equalTo("values (1)"));
+    final JsonTable table3 = schema.tables.get(3);
+    assertTrue(table3 instanceof JsonView);
+    assertThat(((JsonView) table3).getSql(), equalTo("values (1)\n(2)\n"));
+  }
+
+  /** Tests a model with bad multi-line SQL. */
+  @Test public void testReadBadMultiLineSql() throws IOException {
+    final ObjectMapper mapper = mapper();
+    JsonRoot root = mapper.readValue(
+        "{\n"
+        + "  version: '1.0',\n"
+        + "   schemas: [\n"
+        + "     {\n"
+        + "       name: 'FoodMart',\n"
+        + "       tables: [\n"
+        + "         {\n"
+        + "           name: 'V',\n"
+        + "           type: 'view',\n"
+        + "           sql: [ 'values (1)', 2 ]\n"
+        + "         }\n"
+        + "       ]\n"
+        + "     }\n"
+        + "   ]\n"
+        + "}",
+        JsonRoot.class);
+    assertEquals(1, root.schemas.size());
+    final JsonMapSchema schema = (JsonMapSchema) root.schemas.get(0);
+    assertEquals(1, schema.tables.size());
+    final JsonView table1 = (JsonView) schema.tables.get(0);
+    try {
+      String s = table1.getSql();
+      fail("exprcted error, got " + s);
+    } catch (RuntimeException e) {
+      assertThat(e.getMessage(),
+          equalTo("each element of a string list must be a string; found: 2"));
+    }
   }
 }
 

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/41215c2f/core/src/test/resources/mysql-foodmart-lattice-model.json
----------------------------------------------------------------------
diff --git a/core/src/test/resources/mysql-foodmart-lattice-model.json b/core/src/test/resources/mysql-foodmart-lattice-model.json
new file mode 100644
index 0000000..09e574e
--- /dev/null
+++ b/core/src/test/resources/mysql-foodmart-lattice-model.json
@@ -0,0 +1,59 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to you under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+{
+  version: '1.0',
+  defaultSchema: 'foodmart',
+  schemas: [ {
+    type: 'jdbc',
+    name: 'foodmart',
+    jdbcUser: 'foodmart',
+    jdbcPassword: 'foodmart',
+    jdbcUrl: 'jdbc:mysql://localhost',
+    jdbcCatalog: 'foodmart',
+    jdbcSchema: null
+  },
+  {
+    name: 'adhoc',
+    lattices: [ {
+      name: 'star',
+      sql: [
+        'select 1 from "foodmart"."sales_fact_1997" as "s"',
+        'join "foodmart"."product" as "p" using ("product_id")',
+        'join "foodmart"."time_by_day" as "t" using ("time_id")',
+        'join "foodmart"."product_class" as "pc" on "p"."product_class_id" = "pc"."product_class_id"'
+      ],
+      auto: false,
+      algorithm: true,
+      rowCountEstimate: 86837,
+      defaultMeasures: [ {
+        agg: 'count'
+      } ],
+      tiles: [ {
+        dimensions: [ 'the_year', ['t', 'quarter'] ],
+        measures: [ {
+          agg: 'sum',
+          args: 'unit_sales'
+        }, {
+          agg: 'sum',
+          args: 'store_sales'
+        }, {
+          agg: 'count'
+        } ]
+      } ]
+    } ]
+  } ]
+}

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/41215c2f/doc/MODEL.md
----------------------------------------------------------------------
diff --git a/doc/MODEL.md b/doc/MODEL.md
index 367e66b..e9febf2 100644
--- a/doc/MODEL.md
+++ b/doc/MODEL.md
@@ -91,7 +91,8 @@ Like base class <a href="#schema">Schema</a>, occurs within `root.schemas`.
 }
 ```
 
-`name`, `type`, `path`, `cache`, `materializations` inherited from <a href="#schema">Schema</a>.
+`name`, `type`, `path`, `cache`, `materializations` inherited from
+<a href="#schema">Schema</a>.
 
 `tables` (optional list of <a href="#table">Table</a> elements)
 defines the tables in this schema.
@@ -115,7 +116,8 @@ Like base class <a href="#schema">Schema</a>, occurs within `root.schemas`.
 }
 ```
 
-`name`, `type`, `path`, `cache`, `materializations` inherited from <a href="#schema">Schema</a>.
+`name`, `type`, `path`, `cache`, `materializations` inherited from
+<a href="#schema">Schema</a>.
 
 `factory` (required string) is the name of the factory class for this
 schema. Must implement interface `net.hydromatic.optiq.SchemaFactory`
@@ -141,33 +143,43 @@ Like base class <a href="#schema">Schema</a>, occurs within `root.schemas`.
 }
 ```
 
-`name`, `type`, `path`, `cache`, `materializations` inherited from <a href="#schema">Schema</a>.
+`name`, `type`, `path`, `cache`, `materializations` inherited from
+<a href="#schema">Schema</a>.
 
-`jdbcDriver` (optional string) is TODO.
+`jdbcDriver` (optional string) is the name of the JDBC driver class. It not
+specified, uses whichever class the JDBC DriverManager chooses.
 
-`jdbcUrl` (optional string) is TODO.
+`jdbcUrl` (optional string) is the JDBC connect string, for example
+"jdbc:mysql://localhost/foodmart".
 
-`jdbcUser` (optional string) is TODO.
+`jdbcUser` (optional string) is the JDBC user name.
 
-`jdbcPassword` (optional string) is TODO.
+`jdbcPassword` (optional string) is the JDBC password.
 
-`jdbcCatalog` (optional string) is TODO.
+`jdbcCatalog` (optional string) is the name of the initial catalog in the JDBC
+data source.
 
-`jdbcSchema` (optional string) is TODO.
+`jdbcSchema` (optional string) is the name of the initial schema in the JDBC
+data source.
 
 ### Materialization
 
 Occurs within `root.schemas.materializations`.
 
 ```json
-TODO
+{
+  view: 'V',
+  table: 'T',
+  sql: 'select deptno, count(*) as c, sum(sal) as s from emp group by deptno'
+}
 ```
 
 `view` (optional string) TODO
 
 `table` (optional string) TODO
 
-`sql` (optional string) TODO
+`sql` (optional string, or list of strings that will be concatenated as a
+ multi-line string) is the SQL definition of the materialization.
 
 ### Table
 
@@ -202,7 +214,8 @@ Like base class <a href="#table">Table</a>, occurs within `root.schemas.tables`.
 
 `name`, `type`, `columns` inherited from <a href="#table">Table</a>.
 
-`sql` (required string) is the SQL definition of the view.
+`sql` (required string, or list of strings that will be concatenated as a
+ multi-line string) is the SQL definition of the view.
 
 `path` (optional list) is the SQL path to resolve the query. If not
 specified, defaults to the current schema.
@@ -236,7 +249,9 @@ factory.
 Occurs within `root.schemas.tables.columns`.
 
 ```json
-TODO
+{
+  name: 'empno'
+}
 ```
 
 `name` (required string) is the name of this column.
@@ -246,11 +261,133 @@ TODO
 Occurs within `root.schemas.functions`.
 
 ```json
-TODO
+{
+  name: 'MY_PLUS',
+  className: 'com.example.functions.MyPlusFunction',
+  methodName: 'apply',
+  path: []
+}
 ```
 
 `name` (required string) is the name of this function.
 
-`className` (required string) is the name of the class that implements this function.
+`className` (required string) is the name of the class that implements this
+function.
+
+`methodName` (optional string) is the name of the method that implements this
+function.
 
 `path` (optional list of string) is the path for resolving this function.
+
+### Lattice
+
+Occurs within `root.schemas.lattices`.
+
+```json
+{
+  name: 'star',
+  sql: [
+    'select 1 from "foodmart"."sales_fact_1997" as "s"',
+    'join "foodmart"."product" as "p" using ("product_id")',
+    'join "foodmart"."time_by_day" as "t" using ("time_id")',
+    'join "foodmart"."product_class" as "pc" on "p"."product_class_id" = "pc"."product_class_id"'
+  ],
+  auto: false,
+  algorithm: true,
+  algorithmMaxMillis: 10000,
+  rowCountEstimate: 86837,
+  defaultMeasures: [ {
+    agg: 'count'
+  } ],
+  tiles: [ {
+    dimensions: [ 'the_year', ['t', 'quarter'] ],
+    measures: [ {
+      agg: 'sum',
+      args: 'unit_sales'
+    }, {
+      agg: 'sum',
+      args: 'store_sales'
+    }, {
+      agg: 'count'
+    } ]
+  } ]
+}
+```
+
+`name` (required string) is the name of this lattice.
+
+`sql` (required string, or list of strings that will be concatenated as a
+multi-line string) is the SQL statement that defines the fact table, dimension
+tables, and join paths for this lattice.
+
+`auto` (optional boolean, default true) is whether to materialize tiles on need
+as queries are executed.
+
+`algorithm` (optional boolean, default false) is whether to use an optimization
+algorithm to suggest and populate an initial set of tiles.
+
+`algorithmMaxMillis` (optional long, default -1, meaning no limit) is the
+maximum number of milliseconds for which to run the algorithm. After this point,
+takes the best result the algorithm has come up with so far.
+
+`rowCountEstimate` (optional double, default 1000.0) estimated number of rows in
+the star
+
+`tiles` (optional list of <a href="#tile">Tile</a> elements) is a list of
+materialized aggregates to create up front.
+
+`defaultMeasures`  (optional list of <a href="#measure">Measure</a> elements)
+is a list of measures that a tile should have by default.
+Any tile defined in `tiles` can still define its own measures, including
+measures not on this list. If not specified, the default list of measures is
+just 'count(*)':
+
+```json
+[ { name: 'count' } ]
+```
+
+### Tile
+
+Occurs within `root.schemas.lattices.tiles`.
+
+```json
+{
+  dimensions: [ 'the_year', ['t', 'quarter'] ],
+  measures: [ {
+    agg: 'sum',
+    args: 'unit_sales'
+  }, {
+    agg: 'sum',
+    args: 'store_sales'
+  }, {
+    agg: 'count'
+  } ]
+}
+```
+
+`dimensions` is a list of dimensions (columns from the star), like a `GROUP BY`
+clause. Each element is either a string (the unique label of the column within
+the star) or a string list (a column name qualified by a table name).
+
+`measures` (optional list of <a href="#measure">Measure</a> elements) is a list
+of aggregate functions applied to arguments. If not specified, uses the
+lattice's default measure list.
+
+### Measure
+
+Occurs within `root.schemas.lattices.defaultMeasures`
+and `root.schemas.lattices.tiles.measures`.
+
+```json
+{
+  agg: 'sum',
+  args: [ 'unit_sales' ]
+}
+```
+
+`agg` is the name of an aggregate function (usually 'count', 'sum', 'min',
+'max').
+
+`args` (optional) is a column label (string), or list of zero or more columns.
+If a list, each element is either a string (the unique label of the column
+within the star) or a string list (a column name qualified by a table name).