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 2015/06/17 20:57:03 UTC

[1/3] incubator-calcite git commit: Related to [CALCITE-758], speed up matching by not considering tiles separately from other materialized views

Repository: incubator-calcite
Updated Branches:
  refs/heads/master ccdda9b20 -> 31cab4da1


Related to [CALCITE-758], speed up matching by not considering tiles separately from other materialized views


Project: http://git-wip-us.apache.org/repos/asf/incubator-calcite/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-calcite/commit/31cab4da
Tree: http://git-wip-us.apache.org/repos/asf/incubator-calcite/tree/31cab4da
Diff: http://git-wip-us.apache.org/repos/asf/incubator-calcite/diff/31cab4da

Branch: refs/heads/master
Commit: 31cab4da126f9cf21d1e56a450f492b58a010c80
Parents: e7d15ad
Author: Julian Hyde <jh...@apache.org>
Authored: Tue Jun 16 14:15:40 2015 -0700
Committer: Julian Hyde <jh...@apache.org>
Committed: Wed Jun 17 11:39:02 2015 -0700

----------------------------------------------------------------------
 .../java/org/apache/calcite/plan/volcano/VolcanoPlanner.java     | 4 ++++
 1 file changed, 4 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/31cab4da/core/src/main/java/org/apache/calcite/plan/volcano/VolcanoPlanner.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/plan/volcano/VolcanoPlanner.java b/core/src/main/java/org/apache/calcite/plan/volcano/VolcanoPlanner.java
index 9206eb2..3bb3a16 100644
--- a/core/src/main/java/org/apache/calcite/plan/volcano/VolcanoPlanner.java
+++ b/core/src/main/java/org/apache/calcite/plan/volcano/VolcanoPlanner.java
@@ -449,6 +449,10 @@ public class VolcanoPlanner extends AbstractRelOptPlanner {
         Graphs.makeImmutable(usesGraph);
     final Set<RelOptTable> queryTables = findTables(originalRoot);
     for (RelOptMaterialization materialization : materializations) {
+      if (materialization.starTable != null) {
+        // Materialization is a tile in a lattice. We will deal with it shortly.
+        continue;
+      }
       if (materialization.table != null) {
         if (usesTable(materialization.table, queryTables, frozenGraph)) {
           useMaterialization(materialization);


[3/3] incubator-calcite git commit: [CALCITE-758] Use more than one lattice in the same query (Rajat Venkatesh)

Posted by jh...@apache.org.
[CALCITE-758] Use more than one lattice in the same query (Rajat Venkatesh)

Close apache/incubator-calcite#95


Project: http://git-wip-us.apache.org/repos/asf/incubator-calcite/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-calcite/commit/bc8c4a98
Tree: http://git-wip-us.apache.org/repos/asf/incubator-calcite/tree/bc8c4a98
Diff: http://git-wip-us.apache.org/repos/asf/incubator-calcite/diff/bc8c4a98

Branch: refs/heads/master
Commit: bc8c4a98b7e80f8a4c1e0b7ac8b8e61b2ca434f0
Parents: ccdda9b
Author: Rajat Venkatesh <rv...@qubole.com>
Authored: Mon Jun 1 12:29:20 2015 +0530
Committer: Julian Hyde <jh...@apache.org>
Committed: Wed Jun 17 11:39:02 2015 -0700

----------------------------------------------------------------------
 .../calcite/plan/SubstitutionVisitor.java       | 140 +++++++++++++++++++
 .../org/apache/calcite/test/LatticeTest.java    |  74 ++++++++++
 2 files changed, 214 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/bc8c4a98/core/src/main/java/org/apache/calcite/plan/SubstitutionVisitor.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/plan/SubstitutionVisitor.java b/core/src/main/java/org/apache/calcite/plan/SubstitutionVisitor.java
index 410339f..db69196 100644
--- a/core/src/main/java/org/apache/calcite/plan/SubstitutionVisitor.java
+++ b/core/src/main/java/org/apache/calcite/plan/SubstitutionVisitor.java
@@ -25,11 +25,14 @@ import org.apache.calcite.rel.SingleRel;
 import org.apache.calcite.rel.core.Aggregate;
 import org.apache.calcite.rel.core.AggregateCall;
 import org.apache.calcite.rel.core.Filter;
+import org.apache.calcite.rel.core.Join;
+import org.apache.calcite.rel.core.JoinRelType;
 import org.apache.calcite.rel.core.Project;
 import org.apache.calcite.rel.core.TableScan;
 import org.apache.calcite.rel.core.Values;
 import org.apache.calcite.rel.logical.LogicalAggregate;
 import org.apache.calcite.rel.logical.LogicalFilter;
+import org.apache.calcite.rel.logical.LogicalJoin;
 import org.apache.calcite.rel.logical.LogicalProject;
 import org.apache.calcite.rel.logical.LogicalSort;
 import org.apache.calcite.rel.logical.LogicalUnion;
@@ -64,6 +67,7 @@ import com.google.common.base.Objects;
 import com.google.common.base.Preconditions;
 import com.google.common.base.Predicate;
 import com.google.common.collect.ImmutableList;
+import com.google.common.collect.ImmutableSet;
 import com.google.common.collect.Iterables;
 import com.google.common.collect.LinkedHashMultimap;
 import com.google.common.collect.Lists;
@@ -73,6 +77,7 @@ import com.google.common.collect.Sets;
 
 import java.util.AbstractList;
 import java.util.ArrayList;
+import java.util.Collections;
 import java.util.HashMap;
 import java.util.Iterator;
 import java.util.List;
@@ -228,6 +233,13 @@ public class SubstitutionVisitor {
           aggregate.getGroupSet(), aggregate.getGroupSets(),
           aggregate.getAggCallList());
     }
+    if (rel instanceof Join) {
+      final Join join = (Join) rel;
+      final MutableRel left = toMutable(join.getLeft());
+      final MutableRel right = toMutable(join.getRight());
+      return MutableJoin.of(join.getCluster(), left, right,
+          join.getCondition(), join.getJoinType(), join.getVariablesStopped());
+    }
     throw new RuntimeException("cannot translate " + rel + " to MutableRel");
   }
 
@@ -607,6 +619,10 @@ public class SubstitutionVisitor {
     case UNION:
       final MutableUnion union = (MutableUnion) node;
       return LogicalUnion.create(fromMutables(union.inputs), union.all);
+    case JOIN:
+      final MutableJoin join = (MutableJoin) node;
+      return LogicalJoin.create(fromMutable(join.getLeft()), fromMutable(join.getRight()),
+          join.getCondition(), join.getJoinType(), join.getVariablesStopped());
     default:
       throw new AssertionError(node.deep());
     }
@@ -1798,6 +1814,130 @@ public class SubstitutionVisitor {
     }
   }
 
+  /** Base Class for relations with two inputs */
+  private abstract static class MutableBiRel extends MutableRel {
+    protected MutableRel left;
+    protected MutableRel right;
+
+    MutableBiRel(MutableRelType type, RelOptCluster cluster, RelDataType rowType,
+                        MutableRel left, MutableRel right) {
+      super(cluster, rowType, type);
+      this.left = left;
+      left.parent = this;
+      left.ordinalInParent = 0;
+
+      this.right = right;
+      right.parent = this;
+      right.ordinalInParent = 1;
+    }
+
+    public void setInput(int ordinalInParent, MutableRel input) {
+      if (ordinalInParent > 1) {
+        throw new IllegalArgumentException();
+      }
+      if (ordinalInParent == 0) {
+        this.left = input;
+      } else {
+        this.right = input;
+      }
+      if (input != null) {
+        input.parent = this;
+        input.ordinalInParent = 0;
+      }
+    }
+
+    public List<MutableRel> getInputs() {
+      return ImmutableList.of(left, right);
+    }
+
+    public MutableRel getLeft() {
+      return left;
+    }
+
+    public MutableRel getRight() {
+      return right;
+    }
+
+    public void childrenAccept(MutableRelVisitor visitor) {
+
+      visitor.visit(left);
+      visitor.visit(right);
+    }
+  }
+
+  /** Mutable equivalent of
+   * {@link org.apache.calcite.rel.logical.LogicalJoin}. */
+  private static class MutableJoin extends MutableBiRel {
+    //~ Instance fields --------------------------------------------------------
+
+    protected final RexNode condition;
+    protected final ImmutableSet<String> variablesStopped;
+
+    /**
+     * Values must be of enumeration {@link JoinRelType}, except that
+     * {@link JoinRelType#RIGHT} is disallowed.
+     */
+    protected JoinRelType joinType;
+
+    //~ Constructors -----------------------------------------------------------
+
+    /**
+     * Creates a Join.
+     *
+     * @param cluster          Cluster
+     * @param traits           Traits
+     * @param left             Left input
+     * @param right            Right input
+     * @param condition        Join condition
+     * @param joinType         Join type
+     * @param variablesStopped Set of names of variables which are set by the
+     *                         LHS and used by the RHS and are not available to
+     *                         nodes above this LogicalJoin in the tree
+     */
+    private MutableJoin(
+        RelOptCluster cluster,
+        RelDataType rowType,
+        MutableRel left,
+        MutableRel right,
+        RexNode condition,
+        JoinRelType joinType,
+        Set<String> variablesStopped) {
+      super(MutableRelType.JOIN, cluster, rowType, left, right);
+      this.condition = condition;
+      this.variablesStopped = ImmutableSet.copyOf(variablesStopped);
+      assert joinType != null;
+      assert condition != null;
+      this.joinType = joinType;
+    }
+
+    public RexNode getCondition() {
+      return condition;
+    }
+
+    public JoinRelType getJoinType() {
+      return joinType;
+    }
+
+    public ImmutableSet getVariablesStopped() {
+      return variablesStopped;
+    }
+
+    static MutableJoin of(RelOptCluster cluster, MutableRel left, MutableRel right,
+        RexNode condition, JoinRelType joinType, Set<String> variablesStopped) {
+      List<RelDataTypeField> fieldList = Collections.emptyList();
+      RelDataType rowType =
+          Join.deriveJoinRowType(left.getRowType(), right.getRowType(),
+              joinType, cluster.getTypeFactory(), null, fieldList);
+      return new MutableJoin(cluster, rowType, left, right,
+          condition, joinType, variablesStopped);
+    }
+
+    @Override public StringBuilder digest(StringBuilder buf) {
+      return buf.append("Join(left: ").append(left).append(", right:")
+          .append(right).append(")");
+    }
+  }
+
   /** Utilities for dealing with {@link MutableRel}s. */
   private static class MutableRels {
     public static boolean contains(MutableRel ancestor,

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/bc8c4a98/core/src/test/java/org/apache/calcite/test/LatticeTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/calcite/test/LatticeTest.java b/core/src/test/java/org/apache/calcite/test/LatticeTest.java
index b4a79af..0b649cc 100644
--- a/core/src/test/java/org/apache/calcite/test/LatticeTest.java
+++ b/core/src/test/java/org/apache/calcite/test/LatticeTest.java
@@ -547,6 +547,80 @@ public class LatticeTest {
     // TODO
   }
 
+  @Test public void testTwoLattices() {
+    final String sales_lattice = "{\n"
+        + "  name: 'star',\n"
+        + "  sql: [\n"
+        + "    'select 1 from \"foodmart\".\"sales_fact_1997\" as \"s\"',\n"
+        + "    'join \"foodmart\".\"product\" as \"p\" using (\"product_id\")',\n"
+        + "    'join \"foodmart\".\"time_by_day\" as \"t\" using (\"time_id\")',\n"
+        + "    'join \"foodmart\".\"product_class\" as \"pc\" on \"p\".\"product_class_id\" = \"pc\".\"product_class_id\"'\n"
+        + "  ],\n"
+        + "  auto: false,\n"
+        + "  algorithm: true,\n"
+        + "  algorithmMaxMillis: 10000,\n"
+        + "  rowCountEstimate: 86837,\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"
+        + "}\n";
+
+    final String inventory_lattice = "{\n"
+        + "  name: 'warehouse',\n"
+        + "  sql: [\n"
+        + "  'select 1 from \"foodmart\".\"inventory_fact_1997\" as \"s\"',\n"
+        + "  'join \"foodmart\".\"product\" as \"p\" using (\"product_id\")',\n"
+        + "  'join \"foodmart\".\"time_by_day\" as \"t\" using (\"time_id\")',\n"
+        + "  'join \"foodmart\".\"warehouse\" as \"w\" using (\"warehouse_id\")'\n"
+        + "  ],\n"
+        + "  auto: false,\n"
+        + "  algorithm: true,\n"
+        + "  algorithmMaxMillis: 10000,\n"
+        + "  rowCountEstimate: 86837,\n"
+        + "  defaultMeasures: [ {\n"
+        + "    agg: 'count'\n"
+        + "  } ],\n"
+        + "  tiles: [ {\n"
+        + "    dimensions: [ 'the_year', 'warehouse_name'],\n"
+        + "    measures: [ {\n"
+        + "      agg: 'sum',\n"
+        + "      args: 'store_invoice'\n"
+        + "    }, {\n"
+        + "      agg: 'sum',\n"
+        + "      args: 'supply_time'\n"
+        + "    }, {\n"
+        + "      agg: 'sum',\n"
+        + "      args: 'warehouse_cost'\n"
+        + "    } ]\n"
+        + "  } ]\n"
+        + "}\n";
+    final AtomicInteger counter = new AtomicInteger();
+    modelWithLattices(sales_lattice, inventory_lattice)
+        .query("select s.\"unit_sales\", p.\"brand_name\"\n"
+            + "from \"foodmart\".\"sales_fact_1997\" as s\n"
+            + "join \"foodmart\".\"product\" as p using (\"product_id\")\n")
+        .enableMaterializations(true)
+        .substitutionMatches(
+            CalciteAssert.checkRel(
+                "LogicalProject(unit_sales=[$7], brand_name=[$10])\n"
+                    + "  LogicalProject(product_id=[$0], time_id=[$1], customer_id=[$2], promotion_id=[$3], store_id=[$4], store_sales=[$5], store_cost=[$6], unit_sales=[$7], product_class_id=[$8], product_id0=[$9], brand_name=[$10], product_name=[$11], SKU=[$12], SRP=[$13], gross_weight=[$14], net_weight=[$15], recyclable_package=[$16], low_fat=[$17], units_per_case=[$18], cases_per_pallet=[$19], shelf_width=[$20], shelf_height=[$21], shelf_depth=[$22])\n"
+                    + "    LogicalTableScan(table=[[adhoc, star]])\n",
+                counter));
+    assertThat(counter.intValue(), equalTo(1));
+  }
+
   private CalciteAssert.AssertThat foodmartModel(String... extras) {
     return modelWithLattice("star",
         "select 1 from \"foodmart\".\"sales_fact_1997\" as \"s\"\n"


[2/3] incubator-calcite git commit: Test case and workaround for [CALCITE-760] Aggregate recommender blows up if row count estimate is too high

Posted by jh...@apache.org.
Test case and workaround for [CALCITE-760] Aggregate recommender blows up if row count estimate is too high


Project: http://git-wip-us.apache.org/repos/asf/incubator-calcite/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-calcite/commit/e7d15add
Tree: http://git-wip-us.apache.org/repos/asf/incubator-calcite/tree/e7d15add
Diff: http://git-wip-us.apache.org/repos/asf/incubator-calcite/diff/e7d15add

Branch: refs/heads/master
Commit: e7d15adde55029953166ec0b35e81133df4c1e68
Parents: bc8c4a9
Author: Julian Hyde <jh...@apache.org>
Authored: Tue Jun 16 14:14:55 2015 -0700
Committer: Julian Hyde <jh...@apache.org>
Committed: Wed Jun 17 11:39:02 2015 -0700

----------------------------------------------------------------------
 .../org/apache/calcite/test/LatticeTest.java    | 134 +++++++++++--------
 1 file changed, 75 insertions(+), 59 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/e7d15add/core/src/test/java/org/apache/calcite/test/LatticeTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/calcite/test/LatticeTest.java b/core/src/test/java/org/apache/calcite/test/LatticeTest.java
index 0b649cc..92d121b 100644
--- a/core/src/test/java/org/apache/calcite/test/LatticeTest.java
+++ b/core/src/test/java/org/apache/calcite/test/LatticeTest.java
@@ -43,12 +43,72 @@ import java.util.concurrent.atomic.AtomicInteger;
 import static org.hamcrest.CoreMatchers.anyOf;
 import static org.hamcrest.CoreMatchers.containsString;
 import static org.hamcrest.CoreMatchers.equalTo;
+import static org.junit.Assert.assertFalse;
 import static org.junit.Assert.assertThat;
 
 /**
  * Unit test for lattices.
  */
 public class LatticeTest {
+  private static final String SALES_LATTICE = "{\n"
+      + "  name: 'star',\n"
+      + "  sql: [\n"
+      + "    'select 1 from \"foodmart\".\"sales_fact_1997\" as \"s\"',\n"
+      + "    'join \"foodmart\".\"product\" as \"p\" using (\"product_id\")',\n"
+      + "    'join \"foodmart\".\"time_by_day\" as \"t\" using (\"time_id\")',\n"
+      + "    'join \"foodmart\".\"product_class\" as \"pc\" on \"p\".\"product_class_id\" = \"pc\".\"product_class_id\"'\n"
+      + "  ],\n"
+      + "  auto: false,\n"
+      + "  algorithm: true,\n"
+      + "  algorithmMaxMillis: 10000,\n"
+      + "  rowCountEstimate: 86837,\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"
+      + "}\n";
+
+  private static final String INVENTORY_LATTICE = "{\n"
+      + "  name: 'warehouse',\n"
+      + "  sql: [\n"
+      + "  'select 1 from \"foodmart\".\"inventory_fact_1997\" as \"s\"',\n"
+      + "  'join \"foodmart\".\"product\" as \"p\" using (\"product_id\")',\n"
+      + "  'join \"foodmart\".\"time_by_day\" as \"t\" using (\"time_id\")',\n"
+      + "  'join \"foodmart\".\"warehouse\" as \"w\" using (\"warehouse_id\")'\n"
+      + "  ],\n"
+      + "  auto: false,\n"
+      + "  algorithm: true,\n"
+      + "  algorithmMaxMillis: 10000,\n"
+      + "  rowCountEstimate: 4070,\n"
+      + "  defaultMeasures: [ {\n"
+      + "    agg: 'count'\n"
+      + "  } ],\n"
+      + "  tiles: [ {\n"
+      + "    dimensions: [ 'the_year', 'warehouse_name'],\n"
+      + "    measures: [ {\n"
+      + "      agg: 'sum',\n"
+      + "      args: 'store_invoice'\n"
+      + "    }, {\n"
+      + "      agg: 'sum',\n"
+      + "      args: 'supply_time'\n"
+      + "    }, {\n"
+      + "      agg: 'sum',\n"
+      + "      args: 'warehouse_cost'\n"
+      + "    } ]\n"
+      + "  } ]\n"
+      + "}\n";
+
   private CalciteAssert.AssertThat modelWithLattice(String name, String sql,
       String... extras) {
     final StringBuilder buf = new StringBuilder("{ name: '")
@@ -548,66 +608,8 @@ public class LatticeTest {
   }
 
   @Test public void testTwoLattices() {
-    final String sales_lattice = "{\n"
-        + "  name: 'star',\n"
-        + "  sql: [\n"
-        + "    'select 1 from \"foodmart\".\"sales_fact_1997\" as \"s\"',\n"
-        + "    'join \"foodmart\".\"product\" as \"p\" using (\"product_id\")',\n"
-        + "    'join \"foodmart\".\"time_by_day\" as \"t\" using (\"time_id\")',\n"
-        + "    'join \"foodmart\".\"product_class\" as \"pc\" on \"p\".\"product_class_id\" = \"pc\".\"product_class_id\"'\n"
-        + "  ],\n"
-        + "  auto: false,\n"
-        + "  algorithm: true,\n"
-        + "  algorithmMaxMillis: 10000,\n"
-        + "  rowCountEstimate: 86837,\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"
-        + "}\n";
-
-    final String inventory_lattice = "{\n"
-        + "  name: 'warehouse',\n"
-        + "  sql: [\n"
-        + "  'select 1 from \"foodmart\".\"inventory_fact_1997\" as \"s\"',\n"
-        + "  'join \"foodmart\".\"product\" as \"p\" using (\"product_id\")',\n"
-        + "  'join \"foodmart\".\"time_by_day\" as \"t\" using (\"time_id\")',\n"
-        + "  'join \"foodmart\".\"warehouse\" as \"w\" using (\"warehouse_id\")'\n"
-        + "  ],\n"
-        + "  auto: false,\n"
-        + "  algorithm: true,\n"
-        + "  algorithmMaxMillis: 10000,\n"
-        + "  rowCountEstimate: 86837,\n"
-        + "  defaultMeasures: [ {\n"
-        + "    agg: 'count'\n"
-        + "  } ],\n"
-        + "  tiles: [ {\n"
-        + "    dimensions: [ 'the_year', 'warehouse_name'],\n"
-        + "    measures: [ {\n"
-        + "      agg: 'sum',\n"
-        + "      args: 'store_invoice'\n"
-        + "    }, {\n"
-        + "      agg: 'sum',\n"
-        + "      args: 'supply_time'\n"
-        + "    }, {\n"
-        + "      agg: 'sum',\n"
-        + "      args: 'warehouse_cost'\n"
-        + "    } ]\n"
-        + "  } ]\n"
-        + "}\n";
     final AtomicInteger counter = new AtomicInteger();
-    modelWithLattices(sales_lattice, inventory_lattice)
+    modelWithLattices(SALES_LATTICE, INVENTORY_LATTICE)
         .query("select s.\"unit_sales\", p.\"brand_name\"\n"
             + "from \"foodmart\".\"sales_fact_1997\" as s\n"
             + "join \"foodmart\".\"product\" as p using (\"product_id\")\n")
@@ -621,6 +623,20 @@ public class LatticeTest {
     assertThat(counter.intValue(), equalTo(1));
   }
 
+  /** Test case for
+   * <a href="https://issues.apache.org/jira/browse/CALCITE-760">[CALCITE-760]
+   * Aggregate recommender blows up if row count estimate is too high</a>. */
+  @Ignore
+  @Test public void testLatticeWithBadRowCountEstimate() {
+    final String lattice =
+        INVENTORY_LATTICE.replace("rowCountEstimate: 4070,",
+            "rowCountEstimate: 4074070,");
+    assertFalse(lattice.equals(INVENTORY_LATTICE));
+    modelWithLattices(lattice)
+        .query("values 1\n")
+        .returns("EXPR$0=1\n");
+  }
+
   private CalciteAssert.AssertThat foodmartModel(String... extras) {
     return modelWithLattice("star",
         "select 1 from \"foodmart\".\"sales_fact_1997\" as \"s\"\n"