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/07/04 02:01:35 UTC

[5/8] git commit: [OPTIQ-247] Add Context and FrameworkConfig

[OPTIQ-247] Add Context and FrameworkConfig


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

Branch: refs/heads/master
Commit: 31062702efc22ac2537351091eb2a8bcb4f58b75
Parents: 899a629
Author: Jacques Nadeau <ja...@apache.org>
Authored: Thu Apr 10 21:19:21 2014 -0700
Committer: Jacques Nadeau <ja...@apache.org>
Committed: Thu Jul 3 12:21:28 2014 -0700

----------------------------------------------------------------------
 .../optiq/prepare/OptiqPrepareImpl.java         |  36 ++-
 .../hydromatic/optiq/prepare/PlannerImpl.java   |  28 +-
 .../hydromatic/optiq/tools/FrameworkConfig.java | 121 +++++++++
 .../net/hydromatic/optiq/tools/Frameworks.java  | 260 ++++++++++---------
 .../net/hydromatic/optiq/tools/Programs.java    |   2 +-
 .../optiq/tools/StdFrameworkConfig.java         | 108 ++++++++
 .../eigenbase/relopt/AbstractRelOptPlanner.java |  14 +-
 .../main/java/org/eigenbase/relopt/Context.java |  34 +++
 .../eigenbase/relopt/RelOptMaterialization.java |   3 +-
 .../org/eigenbase/relopt/RelOptPlanner.java     |   6 +
 .../org/eigenbase/relopt/hep/HepPlanner.java    |  15 +-
 .../relopt/volcano/VolcanoPlanner.java          |  23 +-
 .../org/eigenbase/sql2rel/RelDecorrelator.java  |   7 +-
 .../eigenbase/sql2rel/SqlToRelConverter.java    |   3 +-
 .../net/hydromatic/optiq/tools/PlannerTest.java |  25 +-
 .../org/eigenbase/test/MockRelOptPlanner.java   |   2 +-
 16 files changed, 522 insertions(+), 165 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-optiq/blob/31062702/core/src/main/java/net/hydromatic/optiq/prepare/OptiqPrepareImpl.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/net/hydromatic/optiq/prepare/OptiqPrepareImpl.java b/core/src/main/java/net/hydromatic/optiq/prepare/OptiqPrepareImpl.java
index b11780d..4483c57 100644
--- a/core/src/main/java/net/hydromatic/optiq/prepare/OptiqPrepareImpl.java
+++ b/core/src/main/java/net/hydromatic/optiq/prepare/OptiqPrepareImpl.java
@@ -22,6 +22,7 @@ import net.hydromatic.avatica.ColumnMetaData;
 import net.hydromatic.avatica.Helper;
 
 import net.hydromatic.linq4j.*;
+
 import net.hydromatic.linq4j.expressions.*;
 import net.hydromatic.linq4j.function.Function1;
 
@@ -182,15 +183,24 @@ public class OptiqPrepareImpl implements OptiqPrepare {
     return Collections.<Function1<Context, RelOptPlanner>>singletonList(
         new Function1<Context, RelOptPlanner>() {
           public RelOptPlanner apply(Context context) {
-            return createPlanner(context);
+            return createPlanner(context, null, null);
           }
         });
   }
 
   /** Creates a query planner and initializes it with a default set of
    * rules. */
-  protected RelOptPlanner createPlanner(Context context) {
-    final VolcanoPlanner planner = new VolcanoPlanner();
+  protected RelOptPlanner createPlanner(OptiqPrepare.Context prepareContext) {
+    return createPlanner(prepareContext, null, null);
+  }
+
+  /** Creates a query planner and initializes it with a default set of
+   * rules. */
+  protected RelOptPlanner createPlanner(OptiqPrepare.Context prepareContext, //
+      org.eigenbase.relopt.Context externalContext, //
+      RelOptCostFactory costFactory) {
+    final VolcanoPlanner planner = //
+        new VolcanoPlanner(costFactory, externalContext);
     planner.addRelTraitDef(ConventionTraitDef.INSTANCE);
     if (ENABLE_COLLATION_TRAIT) {
       planner.addRelTraitDef(RelCollationTraitDef.INSTANCE);
@@ -208,7 +218,7 @@ public class OptiqPrepareImpl implements OptiqPrepare {
       }
     }
 
-    final SparkHandler spark = context.spark();
+    final SparkHandler spark = prepareContext.spark();
     if (spark.enabled()) {
       spark.registerRules(
           new SparkHandler.RuleSetBuilder() {
@@ -545,19 +555,23 @@ public class OptiqPrepareImpl implements OptiqPrepare {
   /** Executes a prepare action. */
   public <R> R perform(OptiqServerStatement statement,
       Frameworks.PrepareAction<R> action) {
-    final Context context = statement.createPrepareContext();
-    final JavaTypeFactory typeFactory = context.getTypeFactory();
+    final OptiqPrepare.Context prepareContext = statement
+        .createPrepareContext();
+    final JavaTypeFactory typeFactory = prepareContext.getTypeFactory();
     OptiqCatalogReader catalogReader = new OptiqCatalogReader(
-        context.getRootSchema(),
-        context.config().caseSensitive(),
-        context.getDefaultSchemaPath(),
+        prepareContext.getRootSchema(),
+        prepareContext.config().caseSensitive(),
+        prepareContext.getDefaultSchemaPath(),
         typeFactory);
     final RexBuilder rexBuilder = new RexBuilder(typeFactory);
-    final RelOptPlanner planner = createPlanner(context);
+    final RelOptPlanner planner = createPlanner(prepareContext, //
+        action.getConfig().getContext(), //
+        action.getConfig().getCostFactory());
     final RelOptQuery query = new RelOptQuery(planner);
     final RelOptCluster cluster =
         query.createCluster(rexBuilder.getTypeFactory(), rexBuilder);
-    return action.apply(cluster, catalogReader, context.getRootSchema().plus(),
+    return action.apply(cluster, catalogReader,
+        prepareContext.getRootSchema().plus(),
         statement);
   }
 

http://git-wip-us.apache.org/repos/asf/incubator-optiq/blob/31062702/core/src/main/java/net/hydromatic/optiq/prepare/PlannerImpl.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/net/hydromatic/optiq/prepare/PlannerImpl.java b/core/src/main/java/net/hydromatic/optiq/prepare/PlannerImpl.java
index eaf3700..c36a65b 100644
--- a/core/src/main/java/net/hydromatic/optiq/prepare/PlannerImpl.java
+++ b/core/src/main/java/net/hydromatic/optiq/prepare/PlannerImpl.java
@@ -41,6 +41,7 @@ import com.google.common.collect.ImmutableList;
 public class PlannerImpl implements Planner {
   private final SqlOperatorTable operatorTable;
   private final ImmutableList<Program> programs;
+  private final FrameworkConfig config;
 
   /** Holds the trait definitions to be registered with planner. May be null. */
   private final ImmutableList<RelTraitDef> traitDefs;
@@ -70,22 +71,20 @@ public class PlannerImpl implements Planner {
   private SqlRexConvertletTable convertletTable;
   private RelNode rel;
 
+
+
   /** Creates a planner. Not a public API; call
    * {@link net.hydromatic.optiq.tools.Frameworks#getPlanner} instead. */
-  public PlannerImpl(Lex lex, SqlParserImplFactory parserFactory,
-      SchemaPlus defaultSchema,
-      SqlOperatorTable operatorTable, ImmutableList<Program> programs,
-      ImmutableList<RelTraitDef> traitDefs,
-      SqlRexConvertletTable convertletTable) {
-    assert defaultSchema != null;
-    this.defaultSchema = defaultSchema;
-    this.operatorTable = operatorTable;
-    this.programs = programs;
-    this.lex = lex;
-    this.parserFactory = parserFactory;
+  public PlannerImpl(FrameworkConfig config) {
+    this.config = config;
+    this.defaultSchema = config.getDefaultSchema();
+    this.operatorTable = config.getOperatorTable();
+    this.programs = config.getPrograms();
+    this.lex = config.getLex();
+    this.parserFactory = config.getParserFactory();
     this.state = State.STATE_0_CLOSED;
-    this.traitDefs = traitDefs;
-    this.convertletTable = convertletTable;
+    this.traitDefs = config.getTraitDefs();
+    this.convertletTable = config.getConvertletTable();
     reset();
   }
 
@@ -132,7 +131,8 @@ public class PlannerImpl implements Planner {
             planner = cluster.getPlanner();
             return null;
           }
-        });
+        },
+        config);
 
     state = State.STATE_2_READY;
 

http://git-wip-us.apache.org/repos/asf/incubator-optiq/blob/31062702/core/src/main/java/net/hydromatic/optiq/tools/FrameworkConfig.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/net/hydromatic/optiq/tools/FrameworkConfig.java b/core/src/main/java/net/hydromatic/optiq/tools/FrameworkConfig.java
new file mode 100644
index 0000000..f32cf35
--- /dev/null
+++ b/core/src/main/java/net/hydromatic/optiq/tools/FrameworkConfig.java
@@ -0,0 +1,121 @@
+/*
+// Licensed to Julian Hyde under one or more contributor license
+// agreements. See the NOTICE file distributed with this work for
+// additional information regarding copyright ownership.
+//
+// Julian Hyde licenses this file to you under the Apache License,
+// Version 2.0 (the "License"); you may not use this file except in
+// compliance with the License. You may obtain a copy of the License at:
+//
+// http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing, software
+// distributed under the License is distributed on an "AS IS" BASIS,
+// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+// See the License for the specific language governing permissions and
+// limitations under the License.
+*/
+package net.hydromatic.optiq.tools;
+
+import net.hydromatic.optiq.SchemaPlus;
+import net.hydromatic.optiq.config.Lex;
+
+import org.eigenbase.relopt.Context;
+import org.eigenbase.relopt.RelOptCostFactory;
+import org.eigenbase.relopt.RelTraitDef;
+import org.eigenbase.sql.SqlOperatorTable;
+import org.eigenbase.sql.parser.SqlParserImplFactory;
+import org.eigenbase.sql2rel.SqlRexConvertletTable;
+
+import com.google.common.collect.ImmutableList;
+
+/**
+ * Interface that describes how to configure planning sessions generated
+ * using the Frameworks tools.
+ */
+public interface FrameworkConfig {
+
+  /**
+   * The type of lexing the SqlParser should do.  Controls case rules
+   * and quoted identifier syntax.
+   */
+  Lex getLex();
+
+
+  /**
+   * Provides the Parser factory that creates the SqlParser used in parsing
+   * queries.
+   */
+  SqlParserImplFactory getParserFactory();
+
+  /**
+   * Returns the default schema that should be checked before looking at the
+   * root schema.  Return null to only consult the root schema.
+   */
+  SchemaPlus getDefaultSchema();
+
+
+  /**
+   * List of of one or more programs used during the course of
+   *     query evaluation. The common use case is when there is a single program
+   *     created using {@link Programs#of(RuleSet)}
+   *     and {@link net.hydromatic.optiq.tools.Planner#transform}
+   *     will only be called once. However, consumers may also create programs
+   *     not based on rule sets, register multiple programs,
+   *     and do multiple repetitions
+   *     of {@link Planner#transform} planning cycles using different indices.
+   *     The order of programs provided here determines the zero-based indices
+   *     of programs elsewhere in this class.
+   */
+  ImmutableList<Program> getPrograms();
+
+
+  /**
+   * Return the instance of SqlOperatorTable that should be used to
+   * resolve Optiq operators.
+   */
+  SqlOperatorTable getOperatorTable();
+
+
+
+  /**
+   * Return the cost factory that should be used when creating the planner.
+   * If null, use the default cost factory for that planner.
+   */
+  RelOptCostFactory getCostFactory();
+
+
+/**
+   * <p>If {@code traitDefs} is non-null, the planner first de-registers any
+   * existing {@link RelTraitDef}s, then registers the {@code RelTraitDef}s in
+   * this list.</p>
+   *
+   * <p>The order of {@code RelTraitDef}s in {@code traitDefs} matters if the
+   * planner is VolcanoPlanner. The planner calls {@link RelTraitDef#convert} in
+   * the order of this list. The most important trait comes first in the list,
+   * followed by the second most important one, etc.</p>
+   *
+   * @param lex The type of lexing the SqlParser should do.  Controls case rules
+   *     and quoted identifier syntax.
+   * @param parserFactory
+   * @param operatorTable The instance of SqlOperatorTable that be should to
+   *     resolve Optiq operators.
+   * @param ruleSets
+   *  @param  traitDefs The list of RelTraitDef that would be registered with
+   *     planner, or null.
+ * @return
+ */
+  ImmutableList<RelTraitDef> getTraitDefs();
+
+  /**
+   * Return the convertlet table that should be used when converting from Sql
+   * to row expressions
+   */
+  SqlRexConvertletTable getConvertletTable();
+
+  /**
+   * Returns the PlannerContext that should be made available during planning by
+   * calling {@link org.eigenbase.relopt.RelOptPlanner#getPlannerContext}
+   */
+  Context getContext();
+}

http://git-wip-us.apache.org/repos/asf/incubator-optiq/blob/31062702/core/src/main/java/net/hydromatic/optiq/tools/Frameworks.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/net/hydromatic/optiq/tools/Frameworks.java b/core/src/main/java/net/hydromatic/optiq/tools/Frameworks.java
index 6b27169..6472c43 100644
--- a/core/src/main/java/net/hydromatic/optiq/tools/Frameworks.java
+++ b/core/src/main/java/net/hydromatic/optiq/tools/Frameworks.java
@@ -25,15 +25,19 @@ import net.hydromatic.optiq.prepare.OptiqPrepareImpl;
 import net.hydromatic.optiq.prepare.PlannerImpl;
 import net.hydromatic.optiq.server.OptiqServerStatement;
 
+import org.eigenbase.relopt.Context;
 import org.eigenbase.relopt.RelOptCluster;
+import org.eigenbase.relopt.RelOptCostFactory;
 import org.eigenbase.relopt.RelOptSchema;
 import org.eigenbase.relopt.RelTraitDef;
 import org.eigenbase.sql.SqlOperatorTable;
+import org.eigenbase.sql.fun.SqlStdOperatorTable;
 import org.eigenbase.sql.parser.SqlParserImplFactory;
 import org.eigenbase.sql.parser.impl.SqlParserImpl;
 import org.eigenbase.sql2rel.SqlRexConvertletTable;
 import org.eigenbase.sql2rel.StandardConvertletTable;
 
+import com.google.common.base.Preconditions;
 import com.google.common.collect.ImmutableList;
 
 import java.sql.Connection;
@@ -48,118 +52,8 @@ public class Frameworks {
   private Frameworks() {
   }
 
-  /**
-   * Creates an instance of {@code Planner}.
-   *
-   * @param lex The type of lexing the SqlParser should do.  Controls case rules
-   *     and quoted identifier syntax.
-   * @param defaultSchema Default schema. Must not be null.
-   * @param operatorTable The instance of SqlOperatorTable that be should to
-   *     resolve Optiq operators.
-   * @param ruleSets An array of one or more rule sets used during the course of
-   *     query evaluation. The common use case is when there is a single rule
-   *     set and {@link net.hydromatic.optiq.tools.Planner#transform}
-   *     will only be called once. However, consumers may also register multiple
-   *     {@link net.hydromatic.optiq.tools.RuleSet}s and do multiple repetitions
-   *     of {@link Planner#transform} planning cycles using different indices.
-   *     The order of rule sets provided here determines the zero-based indices
-   *     of rule sets elsewhere in this class.
-   * @return The Planner object.
-   */
-  public static Planner getPlanner(Lex lex, SchemaPlus defaultSchema,
-      SqlOperatorTable operatorTable, RuleSet... ruleSets) {
-    return getPlanner(lex, SqlParserImpl.FACTORY, defaultSchema,
-        operatorTable, null, StandardConvertletTable.INSTANCE,
-        Programs.listOf(ruleSets));
-  }
-
-  /**
-   * Creates an instance of {@code Planner}.
-   *
-   * <p>If {@code traitDefs} is specified, the planner first de-registers any
-   * existing {@link RelTraitDef}s, then registers the {@code RelTraitDef}s in
-   * this list.</p>
-   *
-   * <p>The order of {@code RelTraitDef}s in {@code traitDefs} matters if the
-   * planner is VolcanoPlanner. The planner calls {@link RelTraitDef#convert} in
-   * the order of this list. The most important trait comes first in the list,
-   * followed by the second most important one, etc.</p>
-   *
-   * @param lex The type of lexing the SqlParser should do.  Controls case rules
-   *     and quoted identifier syntax.
-   * @param parserFactory Parser factory creates and returns the SQL parser.
-   * @param operatorTable The instance of SqlOperatorTable that be should to
-   *     resolve Optiq operators.
-   * @param ruleSets An array of one or more rule sets used during the course of
-   *     query evaluation. The common use case is when there is a single rule
-   *     set and {@link net.hydromatic.optiq.tools.Planner#transform}
-   *     will only be called once. However, consumers may also register multiple
-   *     {@link net.hydromatic.optiq.tools.RuleSet}s and do multiple repetitions
-   *     of {@link Planner#transform} planning cycles using different indices.
-   *     The order of rule sets provided here determines the zero-based indices
-   *     of rule sets elsewhere in this class.
-   *  @param  traitDefs The list of RelTraitDef that would be registered with
-   *     planner, or null.
-   * @return The Planner object.
-   *
-   * @deprecated Use {@link #getPlanner(net.hydromatic.optiq.config.Lex, org.eigenbase.sql.parser.SqlParserImplFactory, net.hydromatic.optiq.SchemaPlus, org.eigenbase.sql.SqlOperatorTable, java.util.List, org.eigenbase.sql2rel.SqlRexConvertletTable, java.util.List)}.
-   * Will be removed before optiq-0.9.
-   */
-  public static Planner getPlanner(Lex lex,
-      SqlParserImplFactory parserFactory,
-      SchemaPlus defaultSchema,
-      SqlOperatorTable operatorTable,
-      List<RelTraitDef> traitDefs,
-      SqlRexConvertletTable convertletTable,
-      RuleSet... ruleSets) {
-    org.eigenbase.util.Bug.upgrade("remove before optiq-0.9");
-    return getPlanner(lex, parserFactory, defaultSchema, operatorTable,
-        traitDefs, convertletTable, Programs.listOf(ruleSets));
-  }
-
-  /**
-   * Creates an instance of {@code Planner}.
-   *
-   * <p>If {@code traitDefs} is specified, the planner first de-registers any
-   * existing {@link RelTraitDef}s, then registers the {@code RelTraitDef}s in
-   * this list.</p>
-   *
-   * <p>The order of {@code RelTraitDef}s in {@code traitDefs} matters if the
-   * planner is VolcanoPlanner. The planner calls {@link RelTraitDef#convert} in
-   * the order of this list. The most important trait comes first in the list,
-   * followed by the second most important one, etc.</p>
-   *
-   * @param lex The type of lexing the SqlParser should do.  Controls case rules
-   *     and quoted identifier syntax.
-   * @param parserFactory Parser factory creates and returns the SQL parser.
-   * @param operatorTable The instance of SqlOperatorTable that be should to
-   *     resolve Optiq operators.
-   * @param programs List of of one or more programs used during the course of
-   *     query evaluation. The common use case is when there is a single program
-   *     created using {@link Programs#of(RuleSet)}
-   *     and {@link net.hydromatic.optiq.tools.Planner#transform}
-   *     will only be called once. However, consumers may also create programs
-   *     not based on rule sets, register multiple programs,
-   *     and do multiple repetitions
-   *     of {@link Planner#transform} planning cycles using different indices.
-   *     The order of programs provided here determines the zero-based indices
-   *     of programs elsewhere in this class.
-   *  @param  traitDefs The list of RelTraitDef that would be registered with
-   *     planner, or null.
-   * @return The Planner object.
-   */
-  public static Planner getPlanner(Lex lex,
-      SqlParserImplFactory parserFactory,
-      SchemaPlus defaultSchema,
-      SqlOperatorTable operatorTable,
-      List<RelTraitDef> traitDefs,
-      SqlRexConvertletTable convertletTable,
-      List<Program> programs) {
-    org.eigenbase.util.Bug.upgrade("remove before optiq-0.9");
-    return new PlannerImpl(lex, parserFactory, defaultSchema,
-        operatorTable, ImmutableList.copyOf(programs),
-        traitDefs == null ? null : ImmutableList.copyOf(traitDefs),
-        convertletTable);
+  public static Planner getPlanner(FrameworkConfig config) {
+    return new PlannerImpl(config);
   }
 
   /** Piece of code to be run in a context where a planner is available. The
@@ -175,8 +69,23 @@ public class Frameworks {
    * are several other useful objects. The connection and
    * {@link net.hydromatic.optiq.DataContext} are accessible from the
    * statement. */
-  public interface PrepareAction<R> {
-    R apply(RelOptCluster cluster, RelOptSchema relOptSchema,
+  public abstract static class PrepareAction<R> {
+    private final FrameworkConfig config;
+
+    public PrepareAction() {
+      this.config = newConfigBuilder() //
+          .defaultSchema(Frameworks.createRootSchema(true)).build();
+    }
+
+    public PrepareAction(FrameworkConfig config) {
+      this.config = config;
+    }
+
+    public FrameworkConfig getConfig() {
+      return config;
+    }
+
+    public abstract R apply(RelOptCluster cluster, RelOptSchema relOptSchema,
         SchemaPlus rootSchema, OptiqServerStatement statement);
   }
 
@@ -184,11 +93,13 @@ public class Frameworks {
    * Initializes a container then calls user-specified code with a planner.
    *
    * @param action Callback containing user-specified code
+   * @param config FrameworkConfig to use for planner action.
    * @return Return value from action
    */
-  public static <R> R withPlanner(final PlannerAction<R> action) {
+  public static <R> R withPlanner(final PlannerAction<R> action, //
+      FrameworkConfig config) {
     return withPrepare(
-        new Frameworks.PrepareAction<R>() {
+        new Frameworks.PrepareAction<R>(config) {
           public R apply(RelOptCluster cluster, RelOptSchema relOptSchema,
               SchemaPlus rootSchema, OptiqServerStatement statement) {
             return action.apply(cluster, relOptSchema, rootSchema);
@@ -197,6 +108,18 @@ public class Frameworks {
   }
 
   /**
+   * Initializes a container then calls user-specified code with a planner.
+   *
+   * @param action Callback containing user-specified code
+   * @return Return value from action
+   */
+  public static <R> R withPlanner(final PlannerAction<R> action) {
+    FrameworkConfig config = newConfigBuilder() //
+        .defaultSchema(Frameworks.createRootSchema(true)).build();
+    return withPlanner(action, config);
+  }
+
+  /**
    * Initializes a container then calls user-specified code with a planner
    * and statement.
    *
@@ -228,6 +151,111 @@ public class Frameworks {
   public static SchemaPlus createRootSchema(boolean addMetadataSchema) {
     return OptiqSchema.createRootSchema(addMetadataSchema).plus();
   }
+
+  public static ConfigBuilder newConfigBuilder() {
+    return new ConfigBuilder();
+  }
+
+  /**
+   * A builder class to help you build a StdFrameworkConfig using defaults
+   * where values aren't required.
+   */
+  public static class ConfigBuilder {
+    private SqlRexConvertletTable convertletTable =
+        StandardConvertletTable.INSTANCE;
+    private SqlOperatorTable operatorTable = SqlStdOperatorTable.instance();
+    private ImmutableList<Program> programs = ImmutableList.of();
+    private Context context;
+    private ImmutableList<RelTraitDef> traitDefs;
+    private Lex lex = Lex.ORACLE;
+    private SchemaPlus defaultSchema;
+    private RelOptCostFactory costFactory;
+    private SqlParserImplFactory parserFactory = SqlParserImpl.FACTORY;
+
+    private ConfigBuilder() {}
+
+    public FrameworkConfig build() {
+      return new StdFrameworkConfig(context, convertletTable, operatorTable,
+          programs, traitDefs, lex, defaultSchema, costFactory, //
+          parserFactory);
+    }
+
+    public ConfigBuilder context(Context c) {
+      Preconditions.checkNotNull(c);
+      this.context = c;
+      return this;
+    };
+
+    public ConfigBuilder convertletTable(SqlRexConvertletTable table) {
+      Preconditions.checkNotNull(table);
+      this.convertletTable = table;
+      return this;
+    }
+
+    public ConfigBuilder operatorTable(SqlOperatorTable table) {
+      Preconditions.checkNotNull(table);
+      this.operatorTable = table;
+      return this;
+    }
+
+    public ConfigBuilder traitDefs(List<RelTraitDef> traitDefs) {
+      if (traitDefs == null) {
+        this.traitDefs = null;
+      } else {
+        this.traitDefs = ImmutableList.copyOf(traitDefs);
+      }
+      return this;
+    }
+
+    public ConfigBuilder traitDefs(RelTraitDef... traitDefs) {
+      this.traitDefs = ImmutableList.copyOf(traitDefs);
+      return this;
+    }
+
+    public ConfigBuilder lex(Lex lex) {
+      Preconditions.checkNotNull(lex);
+      this.lex = lex;
+      return this;
+    }
+
+    public ConfigBuilder defaultSchema(SchemaPlus defaultSchema) {
+      this.defaultSchema = defaultSchema;
+      return this;
+    }
+
+    public ConfigBuilder costFactory(RelOptCostFactory costFactory) {
+      this.costFactory = costFactory;
+      return this;
+    }
+
+    public ConfigBuilder ruleSets(List<RuleSet> ruleSets) {
+      Preconditions.checkNotNull(ruleSets);
+      this.programs = ImmutableList.copyOf(Programs.listOf(ruleSets.toArray(
+          new RuleSet[ruleSets.size()])));
+      return this;
+    }
+
+    public ConfigBuilder programs(List<Program> programs) {
+      this.programs = ImmutableList.copyOf(programs);
+      return this;
+    }
+
+    public ConfigBuilder programs(Program... programs) {
+      this.programs = ImmutableList.copyOf(programs);
+      return this;
+    }
+
+    public ConfigBuilder ruleSets(RuleSet... ruleSets) {
+      this.programs = ImmutableList.copyOf(Programs.listOf(ruleSets));
+      return this;
+    }
+
+    public ConfigBuilder parserFactory(SqlParserImplFactory parserFactory) {
+      Preconditions.checkNotNull(parserFactory);
+      this.parserFactory = parserFactory;
+      return this;
+    }
+  }
 }
 
 // End Frameworks.java

http://git-wip-us.apache.org/repos/asf/incubator-optiq/blob/31062702/core/src/main/java/net/hydromatic/optiq/tools/Programs.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/net/hydromatic/optiq/tools/Programs.java b/core/src/main/java/net/hydromatic/optiq/tools/Programs.java
index 4996fda..6c14857 100644
--- a/core/src/main/java/net/hydromatic/optiq/tools/Programs.java
+++ b/core/src/main/java/net/hydromatic/optiq/tools/Programs.java
@@ -86,7 +86,7 @@ public class Programs {
       public RelNode run(RelOptPlanner planner, RelNode rel,
           RelTraitSet requiredOutputTraits) {
         final HepPlanner hepPlanner = new HepPlanner(hepProgram,
-            noDag, null, RelOptCostImpl.FACTORY);
+            null, noDag, null, RelOptCostImpl.FACTORY);
 
         if (metadataProvider != null) {
           List<RelMetadataProvider> list = Lists.newArrayList();

http://git-wip-us.apache.org/repos/asf/incubator-optiq/blob/31062702/core/src/main/java/net/hydromatic/optiq/tools/StdFrameworkConfig.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/net/hydromatic/optiq/tools/StdFrameworkConfig.java b/core/src/main/java/net/hydromatic/optiq/tools/StdFrameworkConfig.java
new file mode 100644
index 0000000..c850228
--- /dev/null
+++ b/core/src/main/java/net/hydromatic/optiq/tools/StdFrameworkConfig.java
@@ -0,0 +1,108 @@
+/*
+// Licensed to Julian Hyde under one or more contributor license
+// agreements. See the NOTICE file distributed with this work for
+// additional information regarding copyright ownership.
+//
+// Julian Hyde licenses this file to you under the Apache License,
+// Version 2.0 (the "License"); you may not use this file except in
+// compliance with the License. You may obtain a copy of the License at:
+//
+// http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing, software
+// distributed under the License is distributed on an "AS IS" BASIS,
+// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+// See the License for the specific language governing permissions and
+// limitations under the License.
+*/
+package net.hydromatic.optiq.tools;
+
+
+import net.hydromatic.optiq.SchemaPlus;
+import net.hydromatic.optiq.config.Lex;
+
+import org.eigenbase.relopt.Context;
+import org.eigenbase.relopt.RelOptCostFactory;
+import org.eigenbase.relopt.RelTraitDef;
+import org.eigenbase.sql.SqlOperatorTable;
+import org.eigenbase.sql.parser.SqlParserImplFactory;
+import org.eigenbase.sql2rel.SqlRexConvertletTable;
+
+import com.google.common.collect.ImmutableList;
+
+/**
+ * An implementation of {@link FrameworkConfig} that uses standard Optiq
+ * classes to provide basic planner functionality.
+ */
+class StdFrameworkConfig implements FrameworkConfig {
+
+  private final Context context;
+  private final SqlRexConvertletTable convertletTable;
+  private final SqlOperatorTable operatorTable;
+  private final ImmutableList<Program> programs;
+  private final ImmutableList<RelTraitDef> traitDefs;
+  private final Lex lex;
+  private final SchemaPlus defaultSchema;
+  private final RelOptCostFactory costFactory;
+  private final SqlParserImplFactory parserFactory;
+
+
+  public StdFrameworkConfig(Context context, //
+      SqlRexConvertletTable convertletTable, //
+      SqlOperatorTable operatorTable, //
+      ImmutableList<Program> programs, //
+      ImmutableList<RelTraitDef> traitDefs,
+      Lex lex, //
+      SchemaPlus defaultSchema, //
+      RelOptCostFactory costFactory, //
+      SqlParserImplFactory parserFactory) {
+    super();
+    this.context = context;
+    this.convertletTable = convertletTable;
+    this.operatorTable = operatorTable;
+    this.programs = programs;
+    this.traitDefs = traitDefs;
+    this.lex = lex;
+    this.defaultSchema = defaultSchema;
+    this.costFactory = costFactory;
+    this.parserFactory = parserFactory;
+  }
+
+  public Lex getLex() {
+    return lex;
+  }
+
+  public SqlParserImplFactory getParserFactory() {
+    return parserFactory;
+  }
+
+  public SchemaPlus getDefaultSchema() {
+    return defaultSchema;
+  }
+
+  public ImmutableList<Program> getPrograms() {
+    return programs;
+  }
+
+  public RelOptCostFactory getCostFactory() {
+    return costFactory;
+  }
+
+  public ImmutableList<RelTraitDef> getTraitDefs() {
+    return traitDefs;
+  }
+
+  public SqlRexConvertletTable getConvertletTable() {
+    return convertletTable;
+  }
+
+  public Context getContext() {
+    return context;
+  }
+
+  public SqlOperatorTable getOperatorTable() {
+    return operatorTable;
+  }
+
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-optiq/blob/31062702/core/src/main/java/org/eigenbase/relopt/AbstractRelOptPlanner.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/eigenbase/relopt/AbstractRelOptPlanner.java b/core/src/main/java/org/eigenbase/relopt/AbstractRelOptPlanner.java
index d238474..c15de72 100644
--- a/core/src/main/java/org/eigenbase/relopt/AbstractRelOptPlanner.java
+++ b/core/src/main/java/org/eigenbase/relopt/AbstractRelOptPlanner.java
@@ -26,10 +26,11 @@ import org.eigenbase.rel.metadata.*;
 import org.eigenbase.relopt.volcano.RelSubset;
 import org.eigenbase.util.*;
 
+import static org.eigenbase.util.Static.RESOURCE;
+
 import com.google.common.base.Predicate;
 import com.google.common.collect.Iterables;
 
-import static org.eigenbase.util.Static.RESOURCE;
 
 /**
  * Abstract base for implementations of the {@link RelOptPlanner} interface.
@@ -63,6 +64,8 @@ public abstract class AbstractRelOptPlanner implements RelOptPlanner {
 
   private final Set<RelTrait> traits = new HashSet<RelTrait>();
 
+  private final Context context;
+
   private Executor executor;
 
   //~ Constructors -----------------------------------------------------------
@@ -70,8 +73,11 @@ public abstract class AbstractRelOptPlanner implements RelOptPlanner {
   /**
    * Creates an AbstractRelOptPlanner.
    */
-  protected AbstractRelOptPlanner(RelOptCostFactory costFactory) {
+  protected AbstractRelOptPlanner(RelOptCostFactory costFactory, //
+      Context context) {
+    assert costFactory != null;
     this.costFactory = costFactory;
+    this.context = context;
 
     // In case no one calls setCancelFlag, set up a
     // dummy here.
@@ -87,6 +93,10 @@ public abstract class AbstractRelOptPlanner implements RelOptPlanner {
 
   public void clear() {}
 
+  public Context getContext() {
+    return context;
+  }
+
   public RelOptCostFactory getCostFactory() {
     return costFactory;
   }

http://git-wip-us.apache.org/repos/asf/incubator-optiq/blob/31062702/core/src/main/java/org/eigenbase/relopt/Context.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/eigenbase/relopt/Context.java b/core/src/main/java/org/eigenbase/relopt/Context.java
new file mode 100644
index 0000000..ac4ec75
--- /dev/null
+++ b/core/src/main/java/org/eigenbase/relopt/Context.java
@@ -0,0 +1,34 @@
+/*
+// Licensed to Julian Hyde under one or more contributor license
+// agreements. See the NOTICE file distributed with this work for
+// additional information regarding copyright ownership.
+//
+// Julian Hyde licenses this file to you under the Apache License,
+// Version 2.0 (the "License"); you may not use this file except in
+// compliance with the License. You may obtain a copy of the License at:
+//
+// http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing, software
+// distributed under the License is distributed on an "AS IS" BASIS,
+// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+// See the License for the specific language governing permissions and
+// limitations under the License.
+*/
+package org.eigenbase.relopt;
+
+/**
+ * Provides library users a way to store data within the planner session and
+ * access it within rules. Frameworks can implement their own implementation
+ * of Context and pass that as part of the FrameworkConfig.
+ */
+public interface Context {
+
+  /**
+   * If assignable to clazz, provide the underlying clazz.
+   * @param clazz The Class object of the desired class.
+   * @return Underlying object if matches, otherwise null.
+   */
+  <T> T unwrap(Class<T> clazz);
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-optiq/blob/31062702/core/src/main/java/org/eigenbase/relopt/RelOptMaterialization.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/eigenbase/relopt/RelOptMaterialization.java b/core/src/main/java/org/eigenbase/relopt/RelOptMaterialization.java
index e8758cf..c836f03 100644
--- a/core/src/main/java/org/eigenbase/relopt/RelOptMaterialization.java
+++ b/core/src/main/java/org/eigenbase/relopt/RelOptMaterialization.java
@@ -188,7 +188,8 @@ public class RelOptMaterialization {
         .addRuleInstance(PullUpProjectsAboveJoinRule.LEFT_PROJECT)
         .build();
     final HepPlanner planner =
-        new HepPlanner(program);
+        new HepPlanner(program, //
+            rel.getCluster().getPlanner().getContext());
     planner.setRoot(rel);
     System.out.println(
         RelOptUtil.dumpPlan(

http://git-wip-us.apache.org/repos/asf/incubator-optiq/blob/31062702/core/src/main/java/org/eigenbase/relopt/RelOptPlanner.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/eigenbase/relopt/RelOptPlanner.java b/core/src/main/java/org/eigenbase/relopt/RelOptPlanner.java
index 5d6f349..192d2aa 100644
--- a/core/src/main/java/org/eigenbase/relopt/RelOptPlanner.java
+++ b/core/src/main/java/org/eigenbase/relopt/RelOptPlanner.java
@@ -99,6 +99,12 @@ public interface RelOptPlanner {
   boolean removeRule(RelOptRule rule);
 
   /**
+   * Provides the Context created when this planner was constructed.
+   * @return Either null or an externally defined context.
+   */
+  Context getContext();
+
+  /**
    * Sets the exclusion filter to use for this planner. Rules which match the
    * given pattern will not be fired regardless of whether or when they are
    * added to the planner.

http://git-wip-us.apache.org/repos/asf/incubator-optiq/blob/31062702/core/src/main/java/org/eigenbase/relopt/hep/HepPlanner.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/eigenbase/relopt/hep/HepPlanner.java b/core/src/main/java/org/eigenbase/relopt/hep/HepPlanner.java
index 6b884c1..1371c35 100644
--- a/core/src/main/java/org/eigenbase/relopt/hep/HepPlanner.java
+++ b/core/src/main/java/org/eigenbase/relopt/hep/HepPlanner.java
@@ -77,7 +77,17 @@ public class HepPlanner extends AbstractRelOptPlanner {
    * @param program program controlling rule application
    */
   public HepPlanner(HepProgram program) {
-    this(program, false, null, RelOptCostImpl.FACTORY);
+    this(program, null, false, null, RelOptCostImpl.FACTORY);
+  }
+
+  /**
+   * Creates a new HepPlanner that allows DAG.
+   *
+   * @param program program controlling rule application
+   * @param context to carry while planning
+   */
+  public HepPlanner(HepProgram program, Context context) {
+    this(program, context, false, null, RelOptCostImpl.FACTORY);
   }
 
   /**
@@ -89,10 +99,11 @@ public class HepPlanner extends AbstractRelOptPlanner {
    */
   public HepPlanner(
       HepProgram program,
+      Context context,
       boolean noDAG,
       Function2<RelNode, RelNode, Void> onCopyHook,
       RelOptCostFactory costFactory) {
-    super(costFactory);
+    super(costFactory, context);
     this.mainProgram = program;
     this.onCopyHook =
         Util.first(onCopyHook, Functions.<RelNode, RelNode, Void>ignore2());

http://git-wip-us.apache.org/repos/asf/incubator-optiq/blob/31062702/core/src/main/java/org/eigenbase/relopt/volcano/VolcanoPlanner.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/eigenbase/relopt/volcano/VolcanoPlanner.java b/core/src/main/java/org/eigenbase/relopt/volcano/VolcanoPlanner.java
index dcd6f9c..7bb3d8d 100644
--- a/core/src/main/java/org/eigenbase/relopt/volcano/VolcanoPlanner.java
+++ b/core/src/main/java/org/eigenbase/relopt/volcano/VolcanoPlanner.java
@@ -35,6 +35,7 @@ import org.eigenbase.util.*;
 import net.hydromatic.linq4j.expressions.Expressions;
 
 import net.hydromatic.optiq.runtime.Spaces;
+
 import net.hydromatic.optiq.util.graph.*;
 
 import com.google.common.collect.*;
@@ -199,15 +200,26 @@ public class VolcanoPlanner extends AbstractRelOptPlanner {
    * calling conventions.
    */
   public VolcanoPlanner() {
-    this(VolcanoCost.FACTORY);
+    this(null, null);
+  }
+
+  /**
+   * Creates a uninitialized <code>VolcanoPlanner</code>. To fully initialize
+   * it, the caller must register the desired set of relations, rules, and
+   * calling conventions.
+   */
+  public VolcanoPlanner(Context externalContext) {
+    this(null, externalContext);
   }
 
   /**
    * Creates a {@code VolcanoPlanner} with a given cost factory.
    */
-  protected VolcanoPlanner(RelOptCostFactory costFactory) {
-    super(costFactory);
-    this.zeroCost = costFactory.makeZeroCost();
+  public VolcanoPlanner(RelOptCostFactory costFactory, //
+      Context externalContext) {
+    super(costFactory == null ? VolcanoCost.FACTORY : costFactory, //
+        externalContext);
+    this.zeroCost = this.costFactory.makeZeroCost();
   }
 
   //~ Methods ----------------------------------------------------------------
@@ -289,7 +301,8 @@ public class VolcanoPlanner extends AbstractRelOptPlanner {
             .addRuleInstance(MergeProjectRule.INSTANCE)
             .build();
 
-    final HepPlanner hepPlanner = new HepPlanner(program);
+    final HepPlanner hepPlanner = new HepPlanner(program, //
+        getContext());
     hepPlanner.setRoot(target);
     target = hepPlanner.findBestExp();
 

http://git-wip-us.apache.org/repos/asf/incubator-optiq/blob/31062702/core/src/main/java/org/eigenbase/sql2rel/RelDecorrelator.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/eigenbase/sql2rel/RelDecorrelator.java b/core/src/main/java/org/eigenbase/sql2rel/RelDecorrelator.java
index e0bd1ab..e20a9d1 100644
--- a/core/src/main/java/org/eigenbase/sql2rel/RelDecorrelator.java
+++ b/core/src/main/java/org/eigenbase/sql2rel/RelDecorrelator.java
@@ -73,6 +73,8 @@ public class RelDecorrelator implements ReflectiveVisitor {
   // The rel which is being visited
   private RelNode currentRel;
 
+  private final Context context;
+
   // maps built during decorrelation
   private final Map<RelNode, RelNode> mapOldToNewRel;
 
@@ -94,11 +96,13 @@ public class RelDecorrelator implements ReflectiveVisitor {
       RexBuilder rexBuilder,
       Map<RelNode, SortedSet<CorrelatorRel.Correlation>> mapRefRelToCorVar,
       SortedMap<CorrelatorRel.Correlation, CorrelatorRel> mapCorVarToCorRel,
-      Map<RexFieldAccess, CorrelatorRel.Correlation> mapFieldAccessToCorVar) {
+      Map<RexFieldAccess, CorrelatorRel.Correlation> mapFieldAccessToCorVar,
+      Context context) {
     this.rexBuilder = rexBuilder;
     this.mapRefRelToCorVar = mapRefRelToCorVar;
     this.mapCorVarToCorRel = mapCorVarToCorRel;
     this.mapFieldAccessToCorVar = mapFieldAccessToCorVar;
+    this.context = context;
 
     decorrelateVisitor = new DecorrelateRelVisitor();
     mapOldToNewRel = new HashMap<RelNode, RelNode>();
@@ -169,6 +173,7 @@ public class RelDecorrelator implements ReflectiveVisitor {
     // node is copied when it is registered.
     return new HepPlanner(
         program,
+        context,
         true,
         createCopyHook(),
         RelOptCostImpl.FACTORY);

http://git-wip-us.apache.org/repos/asf/incubator-optiq/blob/31062702/core/src/main/java/org/eigenbase/sql2rel/SqlToRelConverter.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/eigenbase/sql2rel/SqlToRelConverter.java b/core/src/main/java/org/eigenbase/sql2rel/SqlToRelConverter.java
index 682914d..4cf2cc3 100644
--- a/core/src/main/java/org/eigenbase/sql2rel/SqlToRelConverter.java
+++ b/core/src/main/java/org/eigenbase/sql2rel/SqlToRelConverter.java
@@ -2773,7 +2773,8 @@ public class SqlToRelConverter {
             rexBuilder,
             mapRefRelToCorVar,
             mapCorVarToCorRel,
-            mapFieldAccessToCorVar);
+            mapFieldAccessToCorVar,
+            cluster.getPlanner().getContext());
     boolean dumpPlan = SQL2REL_LOGGER.isLoggable(Level.FINE);
 
     RelNode newRootRel = decorrelator.removeCorrelationViaRule(rootRel);

http://git-wip-us.apache.org/repos/asf/incubator-optiq/blob/31062702/core/src/test/java/net/hydromatic/optiq/tools/PlannerTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/net/hydromatic/optiq/tools/PlannerTest.java b/core/src/test/java/net/hydromatic/optiq/tools/PlannerTest.java
index 354f9e6..9981696 100644
--- a/core/src/test/java/net/hydromatic/optiq/tools/PlannerTest.java
+++ b/core/src/test/java/net/hydromatic/optiq/tools/PlannerTest.java
@@ -37,13 +37,11 @@ import org.eigenbase.reltype.RelDataTypeFactory;
 import org.eigenbase.sql.*;
 import org.eigenbase.sql.fun.SqlStdOperatorTable;
 import org.eigenbase.sql.parser.SqlParseException;
-import org.eigenbase.sql.parser.impl.SqlParserImpl;
 import org.eigenbase.sql.type.*;
 import org.eigenbase.sql.util.ChainedSqlOperatorTable;
 import org.eigenbase.sql.util.ListSqlOperatorTable;
 import org.eigenbase.sql.validate.SqlValidator;
 import org.eigenbase.sql.validate.SqlValidatorScope;
-import org.eigenbase.sql2rel.StandardConvertletTable;
 import org.eigenbase.util.Util;
 
 import com.google.common.collect.ImmutableList;
@@ -144,9 +142,10 @@ public class PlannerTest {
         ImmutableList.of(stdOpTab,
             new ListSqlOperatorTable(
                 ImmutableList.<SqlOperator>of(new MyCountAggFunction()))));
-    Planner planner = Frameworks.getPlanner(Lex.ORACLE, SqlParserImpl.FACTORY,
-        createHrSchema(), opTab, null, StandardConvertletTable.INSTANCE,
-        ImmutableList.<Program>of());
+    Planner planner = Frameworks.getPlanner(Frameworks.newConfigBuilder() //
+        .defaultSchema(createHrSchema()) //
+        .operatorTable(opTab) //
+        .build());
     SqlNode parse =
         planner.parse("select \"deptno\", my_count(\"empid\") from \"emps\"\n"
             + "group by \"deptno\"");
@@ -181,9 +180,12 @@ public class PlannerTest {
   }
 
   private Planner getPlanner(List<RelTraitDef> traitDefs, Program... programs) {
-    return Frameworks.getPlanner(Lex.ORACLE, SqlParserImpl.FACTORY,
-        createHrSchema(), SqlStdOperatorTable.instance(), traitDefs,
-        StandardConvertletTable.INSTANCE, ImmutableList.copyOf(programs));
+    return Frameworks.getPlanner(Frameworks.newConfigBuilder() //
+        .lex(Lex.ORACLE) //
+        .defaultSchema(createHrSchema()) //
+        .traitDefs(traitDefs) //
+        .programs(programs) //
+        .build());
   }
 
   /** Tests that planner throws an error if you pass to
@@ -578,8 +580,11 @@ public class PlannerTest {
         Frameworks.createRootSchema(true).add("tpch",
             new ReflectiveSchema(new TpchSchema()));
 
-    Planner p = Frameworks.getPlanner(Lex.MYSQL, schema,
-        SqlStdOperatorTable.instance(), RuleSets.ofList(RULE_SET));
+    Planner p = Frameworks.getPlanner(Frameworks.newConfigBuilder() //
+        .lex(Lex.MYSQL) //
+        .defaultSchema(schema) //
+        .programs(Programs.ofRules(RULE_SET)) //
+        .build());
     SqlNode n = p.parse(tpchTestQuery);
     n = p.validate(n);
     RelNode r = p.convert(n);

http://git-wip-us.apache.org/repos/asf/incubator-optiq/blob/31062702/core/src/test/java/org/eigenbase/test/MockRelOptPlanner.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/eigenbase/test/MockRelOptPlanner.java b/core/src/test/java/org/eigenbase/test/MockRelOptPlanner.java
index fcd9d52..72919dc 100644
--- a/core/src/test/java/org/eigenbase/test/MockRelOptPlanner.java
+++ b/core/src/test/java/org/eigenbase/test/MockRelOptPlanner.java
@@ -45,7 +45,7 @@ public class MockRelOptPlanner extends AbstractRelOptPlanner {
 
   /** Creates MockRelOptPlanner. */
   public MockRelOptPlanner() {
-    super(RelOptCostImpl.FACTORY);
+    super(RelOptCostImpl.FACTORY, null);
     setExecutor(new RexExecutorImpl(Schemas.createDataContext(null)));
   }