You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@calcite.apache.org by ho...@apache.org on 2019/05/30 08:28:50 UTC

[calcite] branch master updated (7302314 -> 037250b)

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

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


    omit 7302314  [CALCITE-3095] Add several system properties to control rules and traits
     new 5650beb  [CALCITE-3095] Add several system properties to control enabling/disabling of rules and traits
     new 037250b  [CALCITE-3093] Remove JDBC connection calls from PlannerImpl

This update added new revisions after undoing existing revisions.
That is to say, some revisions that were in the old version of the
branch are not in the new version.  This situation occurs
when a user --force pushes a change and generates a repository
containing something like this:

 * -- * -- B -- O -- O -- O   (7302314)
            \
             N -- N -- N   refs/heads/master (037250b)

You should already have received notification emails for all of the O
revisions, and so the following emails describe only the N revisions
from the common base, B.

Any revisions marked "omit" are not gone; other references still
refer to them.  Any revisions marked "discard" are gone forever.

The 2 revisions listed above as "new" are entirely new to this
repository and will be described in separate emails.  The revisions
listed as "add" were already present in the repository and have only
been added to this reference.


Summary of changes:
 .../org/apache/calcite/prepare/PlannerImpl.java    | 84 ++++++++++++----------
 .../java/org/apache/calcite/tools/Frameworks.java  |  2 +
 2 files changed, 48 insertions(+), 38 deletions(-)


[calcite] 02/02: [CALCITE-3093] Remove JDBC connection calls from PlannerImpl

Posted by ho...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

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

commit 037250b99d6db8a605b64780a3643f784d07d308
Author: Hongze Zhang <ho...@apache.org>
AuthorDate: Mon May 27 17:33:28 2019 +0800

    [CALCITE-3093] Remove JDBC connection calls from PlannerImpl
    
    Close apache/calcite#1241
---
 .../org/apache/calcite/prepare/PlannerImpl.java    | 84 ++++++++++++----------
 .../java/org/apache/calcite/tools/Frameworks.java  |  2 +
 2 files changed, 48 insertions(+), 38 deletions(-)

diff --git a/core/src/main/java/org/apache/calcite/prepare/PlannerImpl.java b/core/src/main/java/org/apache/calcite/prepare/PlannerImpl.java
index 5305e91..ec0f8d9 100644
--- a/core/src/main/java/org/apache/calcite/prepare/PlannerImpl.java
+++ b/core/src/main/java/org/apache/calcite/prepare/PlannerImpl.java
@@ -20,19 +20,27 @@ import org.apache.calcite.adapter.java.JavaTypeFactory;
 import org.apache.calcite.config.CalciteConnectionConfig;
 import org.apache.calcite.config.CalciteConnectionConfigImpl;
 import org.apache.calcite.config.CalciteConnectionProperty;
+import org.apache.calcite.config.CalciteSystemProperty;
 import org.apache.calcite.jdbc.CalciteSchema;
+import org.apache.calcite.jdbc.JavaTypeFactoryImpl;
 import org.apache.calcite.plan.Context;
+import org.apache.calcite.plan.ConventionTraitDef;
 import org.apache.calcite.plan.RelOptCluster;
 import org.apache.calcite.plan.RelOptPlanner;
 import org.apache.calcite.plan.RelOptTable.ViewExpander;
+import org.apache.calcite.plan.RelOptUtil;
 import org.apache.calcite.plan.RelTraitDef;
 import org.apache.calcite.plan.RelTraitSet;
+import org.apache.calcite.plan.volcano.VolcanoPlanner;
+import org.apache.calcite.rel.RelCollationTraitDef;
 import org.apache.calcite.rel.RelNode;
 import org.apache.calcite.rel.RelRoot;
 import org.apache.calcite.rel.metadata.CachingRelMetadataProvider;
 import org.apache.calcite.rel.type.RelDataType;
+import org.apache.calcite.rel.type.RelDataTypeSystem;
 import org.apache.calcite.rex.RexBuilder;
 import org.apache.calcite.rex.RexExecutor;
+import org.apache.calcite.runtime.Hook;
 import org.apache.calcite.schema.SchemaPlus;
 import org.apache.calcite.sql.SqlNode;
 import org.apache.calcite.sql.SqlOperatorTable;
@@ -44,14 +52,12 @@ import org.apache.calcite.sql2rel.RelDecorrelator;
 import org.apache.calcite.sql2rel.SqlRexConvertletTable;
 import org.apache.calcite.sql2rel.SqlToRelConverter;
 import org.apache.calcite.tools.FrameworkConfig;
-import org.apache.calcite.tools.Frameworks;
 import org.apache.calcite.tools.Planner;
 import org.apache.calcite.tools.Program;
 import org.apache.calcite.tools.RelBuilder;
 import org.apache.calcite.tools.RelConversionException;
 import org.apache.calcite.tools.ValidationException;
 import org.apache.calcite.util.Pair;
-import org.apache.calcite.util.Util;
 
 import com.google.common.collect.ImmutableList;
 
@@ -63,7 +69,9 @@ import java.util.Properties;
 public class PlannerImpl implements Planner, ViewExpander {
   private final SqlOperatorTable operatorTable;
   private final ImmutableList<Program> programs;
-  private final FrameworkConfig config;
+  private final FrameworkConfig frameworkConfig;
+  private final Context context;
+  private final CalciteConnectionConfig connectionConfig;
 
   /** Holds the trait definitions to be registered with planner. May be null. */
   private final ImmutableList<RelTraitDef> traitDefs;
@@ -84,7 +92,7 @@ public class PlannerImpl implements Planner, ViewExpander {
   private RexExecutor executor;
 
   // set in STATE_4_VALIDATE
-  private CalciteSqlValidator validator;
+  private SqlValidator validator;
   private SqlNode validatedSqlNode;
 
   // set in STATE_5_CONVERT
@@ -93,7 +101,7 @@ public class PlannerImpl implements Planner, ViewExpander {
   /** Creates a planner. Not a public API; call
    * {@link org.apache.calcite.tools.Frameworks#getPlanner} instead. */
   public PlannerImpl(FrameworkConfig config) {
-    this.config = config;
+    this.frameworkConfig = config;
     this.defaultSchema = config.getDefaultSchema();
     this.operatorTable = config.getOperatorTable();
     this.programs = config.getPrograms();
@@ -103,9 +111,24 @@ public class PlannerImpl implements Planner, ViewExpander {
     this.traitDefs = config.getTraitDefs();
     this.convertletTable = config.getConvertletTable();
     this.executor = config.getExecutor();
+    this.context = config.getContext();
+    this.connectionConfig = connConfig();
     reset();
   }
 
+  private CalciteConnectionConfig connConfig() {
+    CalciteConnectionConfig unwrapped = context.unwrap(CalciteConnectionConfig.class);
+    if (unwrapped != null) {
+      return unwrapped;
+    }
+    Properties properties = new Properties();
+    properties.setProperty(CalciteConnectionProperty.CASE_SENSITIVE.camelName(),
+        String.valueOf(parserConfig.caseSensitive()));
+    properties.setProperty(CalciteConnectionProperty.CONFORMANCE.camelName(),
+        String.valueOf(frameworkConfig.getParserConfig().conformance()));
+    return new CalciteConnectionConfigImpl(properties);
+  }
+
   /** Makes sure that the state is at least the given state. */
   private void ensure(State state) {
     if (state == this.state) {
@@ -140,23 +163,27 @@ public class PlannerImpl implements Planner, ViewExpander {
       reset();
     }
     ensure(State.STATE_1_RESET);
-    Frameworks.withPlanner(
-        (cluster, relOptSchema, rootSchema) -> {
-          Util.discard(rootSchema); // use our own defaultSchema
-          typeFactory = (JavaTypeFactory) cluster.getTypeFactory();
-          planner = cluster.getPlanner();
-          planner.setExecutor(executor);
-          return null;
-        },
-        config);
+
+    RelDataTypeSystem typeSystem =
+        connectionConfig.typeSystem(RelDataTypeSystem.class,
+            RelDataTypeSystem.DEFAULT);
+    typeFactory = new JavaTypeFactoryImpl(typeSystem);
+    planner = new VolcanoPlanner(frameworkConfig.getCostFactory(), context);
+    RelOptUtil.registerDefaultRules(planner,
+        connectionConfig.materializationsEnabled(),
+        Hook.ENABLE_BINDABLE.get(false));
+    planner.setExecutor(executor);
 
     state = State.STATE_2_READY;
 
     // If user specify own traitDef, instead of default default trait,
-    // first, clear the default trait def registered with planner
-    // then, register the trait def specified in traitDefs.
-    if (this.traitDefs != null) {
-      planner.clearRelTraitDefs();
+    // register the trait def specified in traitDefs.
+    if (this.traitDefs == null) {
+      planner.addRelTraitDef(ConventionTraitDef.INSTANCE);
+      if (CalciteSystemProperty.ENABLE_COLLATION_TRAIT.value()) {
+        planner.addRelTraitDef(RelCollationTraitDef.INSTANCE);
+      }
+    } else {
       for (RelTraitDef def : this.traitDefs) {
         planner.addRelTraitDef(def);
       }
@@ -194,15 +221,7 @@ public class PlannerImpl implements Planner, ViewExpander {
   }
 
   private SqlConformance conformance() {
-    final Context context = config.getContext();
-    if (context != null) {
-      final CalciteConnectionConfig connectionConfig =
-          context.unwrap(CalciteConnectionConfig.class);
-      if (connectionConfig != null) {
-        return connectionConfig.conformance();
-      }
-    }
-    return config.getParserConfig().conformance();
+    return connectionConfig.conformance();
   }
 
   public Pair<SqlNode, RelDataType> validateAndGetType(SqlNode sqlNode)
@@ -302,17 +321,6 @@ public class PlannerImpl implements Planner, ViewExpander {
   // CalciteCatalogReader is stateless; no need to store one
   private CalciteCatalogReader createCatalogReader() {
     final SchemaPlus rootSchema = rootSchema(defaultSchema);
-    final Context context = config.getContext();
-    final CalciteConnectionConfig connectionConfig;
-
-    if (context != null) {
-      connectionConfig = context.unwrap(CalciteConnectionConfig.class);
-    } else {
-      Properties properties = new Properties();
-      properties.setProperty(CalciteConnectionProperty.CASE_SENSITIVE.camelName(),
-              String.valueOf(parserConfig.caseSensitive()));
-      connectionConfig = new CalciteConnectionConfigImpl(properties);
-    }
 
     return new CalciteCatalogReader(
         CalciteSchema.from(rootSchema),
diff --git a/core/src/main/java/org/apache/calcite/tools/Frameworks.java b/core/src/main/java/org/apache/calcite/tools/Frameworks.java
index a273a26..c1f0a31 100644
--- a/core/src/main/java/org/apache/calcite/tools/Frameworks.java
+++ b/core/src/main/java/org/apache/calcite/tools/Frameworks.java
@@ -20,6 +20,7 @@ import org.apache.calcite.config.CalciteConnectionProperty;
 import org.apache.calcite.jdbc.CalciteSchema;
 import org.apache.calcite.materialize.SqlStatisticProvider;
 import org.apache.calcite.plan.Context;
+import org.apache.calcite.plan.Contexts;
 import org.apache.calcite.plan.RelOptCluster;
 import org.apache.calcite.plan.RelOptCostFactory;
 import org.apache.calcite.plan.RelOptSchema;
@@ -210,6 +211,7 @@ public class Frameworks {
       convertletTable = StandardConvertletTable.INSTANCE;
       operatorTable = SqlStdOperatorTable.instance();
       programs = ImmutableList.of();
+      context = Contexts.empty();
       parserConfig = SqlParser.Config.DEFAULT;
       sqlToRelConverterConfig = SqlToRelConverter.Config.DEFAULT;
       typeSystem = RelDataTypeSystem.DEFAULT;


[calcite] 01/02: [CALCITE-3095] Add several system properties to control enabling/disabling of rules and traits

Posted by ho...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

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

commit 5650beb9e81a5706d614140b43060084b955ce29
Author: Hongze Zhang <ho...@apache.org>
AuthorDate: Tue May 28 15:17:25 2019 +0800

    [CALCITE-3095] Add several system properties to control enabling/disabling of rules and traits
    
    Also, add utility class RelOptRules to reorganize built-in rules from different places.
    
    Close apache/calcite#1237
---
 .../adapter/enumerable/EnumerableRules.java        |  37 +++-
 .../calcite/config/CalciteSystemProperty.java      |  16 ++
 .../java/org/apache/calcite/plan/RelOptRules.java  | 195 +++++++++++++++++++++
 .../java/org/apache/calcite/plan/RelOptUtil.java   | 116 +++++++++---
 .../calcite/plan/volcano/VolcanoPlanner.java       |  32 +---
 .../apache/calcite/prepare/CalcitePrepareImpl.java | 186 +++-----------------
 .../apache/calcite/prepare/RelOptTableImpl.java    |   5 +-
 .../java/org/apache/calcite/tools/Programs.java    |  29 +--
 8 files changed, 367 insertions(+), 249 deletions(-)

diff --git a/core/src/main/java/org/apache/calcite/adapter/enumerable/EnumerableRules.java b/core/src/main/java/org/apache/calcite/adapter/enumerable/EnumerableRules.java
index a99337f..e7f4bc6 100644
--- a/core/src/main/java/org/apache/calcite/adapter/enumerable/EnumerableRules.java
+++ b/core/src/main/java/org/apache/calcite/adapter/enumerable/EnumerableRules.java
@@ -20,8 +20,12 @@ import org.apache.calcite.plan.RelOptRule;
 import org.apache.calcite.rel.core.RelFactories;
 import org.apache.calcite.util.trace.CalciteTrace;
 
+import com.google.common.collect.ImmutableList;
+
 import org.slf4j.Logger;
 
+import java.util.List;
+
 /**
  * Rules and relational operators for the
  * {@link EnumerableConvention enumerable calling convention}.
@@ -31,6 +35,9 @@ public class EnumerableRules {
 
   public static final boolean BRIDGE_METHODS = true;
 
+  private EnumerableRules() {
+  }
+
   public static final RelOptRule ENUMERABLE_JOIN_RULE =
       new EnumerableJoinRule();
 
@@ -44,9 +51,6 @@ public class EnumerableRules {
   public static final RelOptRule ENUMERABLE_CORRELATE_RULE =
       new EnumerableCorrelateRule(RelFactories.LOGICAL_BUILDER);
 
-  private EnumerableRules() {
-  }
-
   public static final EnumerableProjectRule ENUMERABLE_PROJECT_RULE =
       new EnumerableProjectRule();
 
@@ -106,6 +110,33 @@ public class EnumerableRules {
 
   public static final EnumerableTableFunctionScanRule ENUMERABLE_TABLE_FUNCTION_SCAN_RULE =
       new EnumerableTableFunctionScanRule(RelFactories.LOGICAL_BUILDER);
+
+  public static final List<RelOptRule> ENUMERABLE_RULES = ImmutableList.of(
+      EnumerableRules.ENUMERABLE_JOIN_RULE,
+      EnumerableRules.ENUMERABLE_MERGE_JOIN_RULE,
+      EnumerableRules.ENUMERABLE_SEMI_JOIN_RULE,
+      EnumerableRules.ENUMERABLE_CORRELATE_RULE,
+      EnumerableRules.ENUMERABLE_PROJECT_RULE,
+      EnumerableRules.ENUMERABLE_FILTER_RULE,
+      EnumerableRules.ENUMERABLE_AGGREGATE_RULE,
+      EnumerableRules.ENUMERABLE_SORT_RULE,
+      EnumerableRules.ENUMERABLE_LIMIT_RULE,
+      EnumerableRules.ENUMERABLE_COLLECT_RULE,
+      EnumerableRules.ENUMERABLE_UNCOLLECT_RULE,
+      EnumerableRules.ENUMERABLE_UNION_RULE,
+      EnumerableRules.ENUMERABLE_REPEAT_UNION_RULE,
+      EnumerableRules.ENUMERABLE_TABLE_SPOOL_RULE,
+      EnumerableRules.ENUMERABLE_INTERSECT_RULE,
+      EnumerableRules.ENUMERABLE_MINUS_RULE,
+      EnumerableRules.ENUMERABLE_TABLE_MODIFICATION_RULE,
+      EnumerableRules.ENUMERABLE_VALUES_RULE,
+      EnumerableRules.ENUMERABLE_WINDOW_RULE,
+      EnumerableRules.ENUMERABLE_TABLE_SCAN_RULE,
+      EnumerableRules.ENUMERABLE_TABLE_FUNCTION_SCAN_RULE);
+
+  public static List<RelOptRule> rules() {
+    return ENUMERABLE_RULES;
+  }
 }
 
 // End EnumerableRules.java
diff --git a/core/src/main/java/org/apache/calcite/config/CalciteSystemProperty.java b/core/src/main/java/org/apache/calcite/config/CalciteSystemProperty.java
index d1f3a5a..92b1142 100644
--- a/core/src/main/java/org/apache/calcite/config/CalciteSystemProperty.java
+++ b/core/src/main/java/org/apache/calcite/config/CalciteSystemProperty.java
@@ -66,6 +66,22 @@ public final class CalciteSystemProperty<T> {
   public static final CalciteSystemProperty<Boolean> COMMUTE =
       booleanProperty("calcite.enable.join.commute", false);
 
+  /** Whether to enable the collation trait in the default planner configuration.
+   *
+   * Some extra optimizations are possible if enabled, but queries should work either way.
+   * At some point this will become a preference, or we will run multiple phases: first
+   * disabled, then enabled. */
+  public static final CalciteSystemProperty<Boolean> ENABLE_COLLATION_TRAIT =
+      booleanProperty("calcite.enable.collation.trait", true);
+
+  /** Whether the enumerable convention is enabled in the default planner configuration. */
+  public static final CalciteSystemProperty<Boolean> ENABLE_ENUMERABLE =
+      booleanProperty("calcite.enable.enumerable", true);
+
+  /** Whether streaming is enabled in the default planner configuration. */
+  public static final CalciteSystemProperty<Boolean> ENABLE_STREAM =
+      booleanProperty("calcite.enable.stream", true);
+
   /**
    *  Whether to follow the SQL standard strictly.
    */
diff --git a/core/src/main/java/org/apache/calcite/plan/RelOptRules.java b/core/src/main/java/org/apache/calcite/plan/RelOptRules.java
new file mode 100644
index 0000000..dcb2f7d
--- /dev/null
+++ b/core/src/main/java/org/apache/calcite/plan/RelOptRules.java
@@ -0,0 +1,195 @@
+/*
+ * 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.
+ */
+package org.apache.calcite.plan;
+
+import org.apache.calcite.adapter.enumerable.EnumerableRules;
+import org.apache.calcite.config.CalciteSystemProperty;
+import org.apache.calcite.interpreter.NoneToBindableConverterRule;
+import org.apache.calcite.linq4j.function.Experimental;
+import org.apache.calcite.plan.volcano.AbstractConverter;
+import org.apache.calcite.rel.rules.AbstractMaterializedViewRule;
+import org.apache.calcite.rel.rules.AggregateExpandDistinctAggregatesRule;
+import org.apache.calcite.rel.rules.AggregateJoinTransposeRule;
+import org.apache.calcite.rel.rules.AggregateMergeRule;
+import org.apache.calcite.rel.rules.AggregateProjectMergeRule;
+import org.apache.calcite.rel.rules.AggregateProjectPullUpConstantsRule;
+import org.apache.calcite.rel.rules.AggregateReduceFunctionsRule;
+import org.apache.calcite.rel.rules.AggregateRemoveRule;
+import org.apache.calcite.rel.rules.AggregateStarTableRule;
+import org.apache.calcite.rel.rules.AggregateValuesRule;
+import org.apache.calcite.rel.rules.CalcMergeRule;
+import org.apache.calcite.rel.rules.CalcRemoveRule;
+import org.apache.calcite.rel.rules.DateRangeRules;
+import org.apache.calcite.rel.rules.ExchangeRemoveConstantKeysRule;
+import org.apache.calcite.rel.rules.FilterAggregateTransposeRule;
+import org.apache.calcite.rel.rules.FilterCalcMergeRule;
+import org.apache.calcite.rel.rules.FilterJoinRule;
+import org.apache.calcite.rel.rules.FilterMergeRule;
+import org.apache.calcite.rel.rules.FilterProjectTransposeRule;
+import org.apache.calcite.rel.rules.FilterTableScanRule;
+import org.apache.calcite.rel.rules.FilterToCalcRule;
+import org.apache.calcite.rel.rules.IntersectToDistinctRule;
+import org.apache.calcite.rel.rules.JoinAssociateRule;
+import org.apache.calcite.rel.rules.JoinCommuteRule;
+import org.apache.calcite.rel.rules.JoinPushExpressionsRule;
+import org.apache.calcite.rel.rules.JoinPushThroughJoinRule;
+import org.apache.calcite.rel.rules.MaterializedViewFilterScanRule;
+import org.apache.calcite.rel.rules.ProjectCalcMergeRule;
+import org.apache.calcite.rel.rules.ProjectFilterTransposeRule;
+import org.apache.calcite.rel.rules.ProjectMergeRule;
+import org.apache.calcite.rel.rules.ProjectRemoveRule;
+import org.apache.calcite.rel.rules.ProjectToCalcRule;
+import org.apache.calcite.rel.rules.ProjectToWindowRule;
+import org.apache.calcite.rel.rules.ProjectWindowTransposeRule;
+import org.apache.calcite.rel.rules.PruneEmptyRules;
+import org.apache.calcite.rel.rules.ReduceExpressionsRule;
+import org.apache.calcite.rel.rules.SemiJoinRule;
+import org.apache.calcite.rel.rules.SortJoinTransposeRule;
+import org.apache.calcite.rel.rules.SortProjectTransposeRule;
+import org.apache.calcite.rel.rules.SortRemoveConstantKeysRule;
+import org.apache.calcite.rel.rules.SortRemoveRule;
+import org.apache.calcite.rel.rules.SortUnionTransposeRule;
+import org.apache.calcite.rel.rules.TableScanRule;
+import org.apache.calcite.rel.rules.UnionMergeRule;
+import org.apache.calcite.rel.rules.UnionPullUpConstantsRule;
+import org.apache.calcite.rel.rules.UnionToDistinctRule;
+import org.apache.calcite.rel.rules.ValuesReduceRule;
+
+import com.google.common.collect.ImmutableList;
+
+import java.util.List;
+
+/**
+ * A utility class for organizing built-in rules and rule related
+ * methods. Currently some rule sets are package private for serving core Calcite.
+ *
+ * @see RelOptRule
+ * @see RelOptUtil
+ */
+@Experimental
+public class RelOptRules {
+
+  private RelOptRules() {
+  }
+
+  /**
+   * The calc rule set is public for use from {@link org.apache.calcite.tools.Programs}
+   */
+  public static final ImmutableList<RelOptRule> CALC_RULES =
+      ImmutableList.of(
+          NoneToBindableConverterRule.INSTANCE,
+          EnumerableRules.ENUMERABLE_CALC_RULE,
+          EnumerableRules.ENUMERABLE_FILTER_TO_CALC_RULE,
+          EnumerableRules.ENUMERABLE_PROJECT_TO_CALC_RULE,
+          CalcMergeRule.INSTANCE,
+          FilterCalcMergeRule.INSTANCE,
+          ProjectCalcMergeRule.INSTANCE,
+          FilterToCalcRule.INSTANCE,
+          ProjectToCalcRule.INSTANCE,
+          CalcMergeRule.INSTANCE,
+
+          // REVIEW jvs 9-Apr-2006: Do we still need these two?  Doesn't the
+          // combination of CalcMergeRule, FilterToCalcRule, and
+          // ProjectToCalcRule have the same effect?
+          FilterCalcMergeRule.INSTANCE,
+          ProjectCalcMergeRule.INSTANCE);
+
+  static final List<RelOptRule> BASE_RULES = ImmutableList.of(
+      AggregateStarTableRule.INSTANCE,
+      AggregateStarTableRule.INSTANCE2,
+      TableScanRule.INSTANCE,
+      CalciteSystemProperty.COMMUTE.value()
+          ? JoinAssociateRule.INSTANCE
+          : ProjectMergeRule.INSTANCE,
+      FilterTableScanRule.INSTANCE,
+      ProjectFilterTransposeRule.INSTANCE,
+      FilterProjectTransposeRule.INSTANCE,
+      FilterJoinRule.FILTER_ON_JOIN,
+      JoinPushExpressionsRule.INSTANCE,
+      AggregateExpandDistinctAggregatesRule.INSTANCE,
+      AggregateReduceFunctionsRule.INSTANCE,
+      FilterAggregateTransposeRule.INSTANCE,
+      ProjectWindowTransposeRule.INSTANCE,
+      JoinCommuteRule.INSTANCE,
+      JoinPushThroughJoinRule.RIGHT,
+      JoinPushThroughJoinRule.LEFT,
+      SortProjectTransposeRule.INSTANCE,
+      SortJoinTransposeRule.INSTANCE,
+      SortRemoveConstantKeysRule.INSTANCE,
+      SortUnionTransposeRule.INSTANCE,
+      ExchangeRemoveConstantKeysRule.EXCHANGE_INSTANCE,
+      ExchangeRemoveConstantKeysRule.SORT_EXCHANGE_INSTANCE);
+
+  static final List<RelOptRule> ABSTRACT_RULES = ImmutableList.of(
+      AggregateProjectPullUpConstantsRule.INSTANCE2,
+      UnionPullUpConstantsRule.INSTANCE,
+      PruneEmptyRules.UNION_INSTANCE,
+      PruneEmptyRules.INTERSECT_INSTANCE,
+      PruneEmptyRules.MINUS_INSTANCE,
+      PruneEmptyRules.PROJECT_INSTANCE,
+      PruneEmptyRules.FILTER_INSTANCE,
+      PruneEmptyRules.SORT_INSTANCE,
+      PruneEmptyRules.AGGREGATE_INSTANCE,
+      PruneEmptyRules.JOIN_LEFT_INSTANCE,
+      PruneEmptyRules.JOIN_RIGHT_INSTANCE,
+      PruneEmptyRules.SORT_FETCH_ZERO_INSTANCE,
+      UnionMergeRule.INSTANCE,
+      UnionMergeRule.INTERSECT_INSTANCE,
+      UnionMergeRule.MINUS_INSTANCE,
+      ProjectToWindowRule.PROJECT,
+      FilterMergeRule.INSTANCE,
+      DateRangeRules.FILTER_INSTANCE,
+      IntersectToDistinctRule.INSTANCE);
+
+  static final List<RelOptRule> ABSTRACT_RELATIONAL_RULES = ImmutableList.of(
+      FilterJoinRule.FILTER_ON_JOIN,
+      FilterJoinRule.JOIN,
+      AbstractConverter.ExpandConversionRule.INSTANCE,
+      JoinCommuteRule.INSTANCE,
+      SemiJoinRule.PROJECT,
+      SemiJoinRule.JOIN,
+      AggregateRemoveRule.INSTANCE,
+      UnionToDistinctRule.INSTANCE,
+      ProjectRemoveRule.INSTANCE,
+      AggregateJoinTransposeRule.INSTANCE,
+      AggregateMergeRule.INSTANCE,
+      AggregateProjectMergeRule.INSTANCE,
+      CalcRemoveRule.INSTANCE,
+      SortRemoveRule.INSTANCE);
+
+  static final List<RelOptRule> CONSTANT_REDUCTION_RULES = ImmutableList.of(
+      ReduceExpressionsRule.PROJECT_INSTANCE,
+      ReduceExpressionsRule.FILTER_INSTANCE,
+      ReduceExpressionsRule.CALC_INSTANCE,
+      ReduceExpressionsRule.WINDOW_INSTANCE,
+      ReduceExpressionsRule.JOIN_INSTANCE,
+      ValuesReduceRule.FILTER_INSTANCE,
+      ValuesReduceRule.PROJECT_FILTER_INSTANCE,
+      ValuesReduceRule.PROJECT_INSTANCE,
+      AggregateValuesRule.INSTANCE);
+
+  static final List<RelOptRule> MATERIALIZATION_RULES = ImmutableList.of(
+      MaterializedViewFilterScanRule.INSTANCE,
+      AbstractMaterializedViewRule.INSTANCE_PROJECT_FILTER,
+      AbstractMaterializedViewRule.INSTANCE_FILTER,
+      AbstractMaterializedViewRule.INSTANCE_PROJECT_JOIN,
+      AbstractMaterializedViewRule.INSTANCE_JOIN,
+      AbstractMaterializedViewRule.INSTANCE_PROJECT_AGGREGATE,
+      AbstractMaterializedViewRule.INSTANCE_AGGREGATE);
+}
+
+// End RelOptRules.java
diff --git a/core/src/main/java/org/apache/calcite/plan/RelOptUtil.java b/core/src/main/java/org/apache/calcite/plan/RelOptUtil.java
index 9ba2809..fc76890 100644
--- a/core/src/main/java/org/apache/calcite/plan/RelOptUtil.java
+++ b/core/src/main/java/org/apache/calcite/plan/RelOptUtil.java
@@ -16,8 +16,14 @@
  */
 package org.apache.calcite.plan;
 
+import org.apache.calcite.adapter.enumerable.EnumerableBindable;
+import org.apache.calcite.adapter.enumerable.EnumerableInterpreterRule;
+import org.apache.calcite.adapter.enumerable.EnumerableRules;
 import org.apache.calcite.avatica.AvaticaConnection;
+import org.apache.calcite.config.CalciteSystemProperty;
+import org.apache.calcite.interpreter.Bindables;
 import org.apache.calcite.linq4j.Ord;
+import org.apache.calcite.linq4j.function.Experimental;
 import org.apache.calcite.rel.RelCollations;
 import org.apache.calcite.rel.RelHomogeneousShuttle;
 import org.apache.calcite.rel.RelNode;
@@ -46,15 +52,10 @@ 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.metadata.RelMetadataQuery;
-import org.apache.calcite.rel.rules.AggregateProjectPullUpConstantsRule;
-import org.apache.calcite.rel.rules.DateRangeRules;
-import org.apache.calcite.rel.rules.FilterMergeRule;
-import org.apache.calcite.rel.rules.IntersectToDistinctRule;
+import org.apache.calcite.rel.rules.JoinAssociateRule;
 import org.apache.calcite.rel.rules.MultiJoin;
-import org.apache.calcite.rel.rules.ProjectToWindowRule;
-import org.apache.calcite.rel.rules.PruneEmptyRules;
-import org.apache.calcite.rel.rules.UnionMergeRule;
-import org.apache.calcite.rel.rules.UnionPullUpConstantsRule;
+import org.apache.calcite.rel.rules.ProjectTableScanRule;
+import org.apache.calcite.rel.stream.StreamRules;
 import org.apache.calcite.rel.type.RelDataType;
 import org.apache.calcite.rel.type.RelDataTypeFactory;
 import org.apache.calcite.rel.type.RelDataTypeField;
@@ -1769,26 +1770,87 @@ public abstract class RelOptUtil {
     return joinRel;
   }
 
+  @Deprecated // to be removed before 2.0
   public static void registerAbstractRels(RelOptPlanner planner) {
-    planner.addRule(AggregateProjectPullUpConstantsRule.INSTANCE2);
-    planner.addRule(UnionPullUpConstantsRule.INSTANCE);
-    planner.addRule(PruneEmptyRules.UNION_INSTANCE);
-    planner.addRule(PruneEmptyRules.INTERSECT_INSTANCE);
-    planner.addRule(PruneEmptyRules.MINUS_INSTANCE);
-    planner.addRule(PruneEmptyRules.PROJECT_INSTANCE);
-    planner.addRule(PruneEmptyRules.FILTER_INSTANCE);
-    planner.addRule(PruneEmptyRules.SORT_INSTANCE);
-    planner.addRule(PruneEmptyRules.AGGREGATE_INSTANCE);
-    planner.addRule(PruneEmptyRules.JOIN_LEFT_INSTANCE);
-    planner.addRule(PruneEmptyRules.JOIN_RIGHT_INSTANCE);
-    planner.addRule(PruneEmptyRules.SORT_FETCH_ZERO_INSTANCE);
-    planner.addRule(UnionMergeRule.INSTANCE);
-    planner.addRule(UnionMergeRule.INTERSECT_INSTANCE);
-    planner.addRule(UnionMergeRule.MINUS_INSTANCE);
-    planner.addRule(ProjectToWindowRule.PROJECT);
-    planner.addRule(FilterMergeRule.INSTANCE);
-    planner.addRule(DateRangeRules.FILTER_INSTANCE);
-    planner.addRule(IntersectToDistinctRule.INSTANCE);
+    registerAbstractRules(planner);
+  }
+
+  @Experimental
+  public static void registerAbstractRules(RelOptPlanner planner) {
+    RelOptRules.ABSTRACT_RULES.forEach(planner::addRule);
+  }
+
+  @Experimental
+  public static void registerAbstractRelationalRules(RelOptPlanner planner) {
+    RelOptRules.ABSTRACT_RELATIONAL_RULES.forEach(planner::addRule);
+    if (CalciteSystemProperty.COMMUTE.value()) {
+      planner.addRule(JoinAssociateRule.INSTANCE);
+    }
+    // todo: rule which makes Project({OrdinalRef}) disappear
+  }
+
+  private static void registerEnumerableRules(RelOptPlanner planner) {
+    EnumerableRules.ENUMERABLE_RULES.forEach(planner::addRule);
+  }
+
+  private static void registerBaseRules(RelOptPlanner planner) {
+    RelOptRules.BASE_RULES.forEach(planner::addRule);
+  }
+
+  private static void registerReductionRules(RelOptPlanner planner) {
+    RelOptRules.CONSTANT_REDUCTION_RULES.forEach(planner::addRule);
+  }
+
+  private static void registerMaterializationRules(RelOptPlanner planner) {
+    RelOptRules.MATERIALIZATION_RULES.forEach(planner::addRule);
+  }
+
+  private static void registerCalcRules(RelOptPlanner planner) {
+    RelOptRules.CALC_RULES.forEach(planner::addRule);
+  }
+
+  @Experimental
+  public static void registerDefaultRules(RelOptPlanner planner,
+      boolean enableMaterialziations, boolean enableBindable) {
+    if (CalciteSystemProperty.ENABLE_COLLATION_TRAIT.value()) {
+      registerAbstractRelationalRules(planner);
+    }
+    registerAbstractRules(planner);
+    registerBaseRules(planner);
+
+    if (enableMaterialziations) {
+      registerMaterializationRules(planner);
+    }
+    if (enableBindable) {
+      for (RelOptRule rule : Bindables.RULES) {
+        planner.addRule(rule);
+      }
+    }
+    planner.addRule(Bindables.BINDABLE_TABLE_SCAN_RULE);
+    planner.addRule(ProjectTableScanRule.INSTANCE);
+    planner.addRule(ProjectTableScanRule.INTERPRETER);
+
+    if (CalciteSystemProperty.ENABLE_ENUMERABLE.value()) {
+      registerEnumerableRules(planner);
+      planner.addRule(EnumerableInterpreterRule.INSTANCE);
+    }
+
+    if (enableBindable && CalciteSystemProperty.ENABLE_ENUMERABLE.value()) {
+      planner.addRule(
+          EnumerableBindable.EnumerableToBindableConverterRule.INSTANCE);
+    }
+
+    if (CalciteSystemProperty.ENABLE_STREAM.value()) {
+      for (RelOptRule rule : StreamRules.RULES) {
+        planner.addRule(rule);
+      }
+    }
+
+    // Change the below to enable constant-reduction.
+    if (false) {
+      registerReductionRules(planner);
+    }
+
   }
 
   /**
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 c70f2fe..a49627a 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
@@ -48,18 +48,6 @@ import org.apache.calcite.rel.externalize.RelWriterImpl;
 import org.apache.calcite.rel.metadata.JaninoRelMetadataProvider;
 import org.apache.calcite.rel.metadata.RelMetadataProvider;
 import org.apache.calcite.rel.metadata.RelMetadataQuery;
-import org.apache.calcite.rel.rules.AggregateJoinTransposeRule;
-import org.apache.calcite.rel.rules.AggregateMergeRule;
-import org.apache.calcite.rel.rules.AggregateProjectMergeRule;
-import org.apache.calcite.rel.rules.AggregateRemoveRule;
-import org.apache.calcite.rel.rules.CalcRemoveRule;
-import org.apache.calcite.rel.rules.FilterJoinRule;
-import org.apache.calcite.rel.rules.JoinAssociateRule;
-import org.apache.calcite.rel.rules.JoinCommuteRule;
-import org.apache.calcite.rel.rules.ProjectRemoveRule;
-import org.apache.calcite.rel.rules.SemiJoinRule;
-import org.apache.calcite.rel.rules.SortRemoveRule;
-import org.apache.calcite.rel.rules.UnionToDistinctRule;
 import org.apache.calcite.rel.type.RelDataType;
 import org.apache.calcite.runtime.Hook;
 import org.apache.calcite.sql.SqlExplainLevel;
@@ -907,25 +895,7 @@ public class VolcanoPlanner extends AbstractRelOptPlanner {
   }
 
   public void registerAbstractRelationalRules() {
-    addRule(FilterJoinRule.FILTER_ON_JOIN);
-    addRule(FilterJoinRule.JOIN);
-    addRule(AbstractConverter.ExpandConversionRule.INSTANCE);
-    addRule(JoinCommuteRule.INSTANCE);
-    addRule(SemiJoinRule.PROJECT);
-    addRule(SemiJoinRule.JOIN);
-    if (CalciteSystemProperty.COMMUTE.value()) {
-      addRule(JoinAssociateRule.INSTANCE);
-    }
-    addRule(AggregateRemoveRule.INSTANCE);
-    addRule(AggregateMergeRule.INSTANCE);
-    addRule(UnionToDistinctRule.INSTANCE);
-    addRule(ProjectRemoveRule.INSTANCE);
-    addRule(AggregateJoinTransposeRule.INSTANCE);
-    addRule(AggregateProjectMergeRule.INSTANCE);
-    addRule(CalcRemoveRule.INSTANCE);
-    addRule(SortRemoveRule.INSTANCE);
-
-    // todo: rule which makes Project({OrdinalRef}) disappear
+    RelOptUtil.registerAbstractRelationalRules(this);
   }
 
   public void registerSchema(RelOptSchema schema) {
diff --git a/core/src/main/java/org/apache/calcite/prepare/CalcitePrepareImpl.java b/core/src/main/java/org/apache/calcite/prepare/CalcitePrepareImpl.java
index fed31ac..7dc1109 100644
--- a/core/src/main/java/org/apache/calcite/prepare/CalcitePrepareImpl.java
+++ b/core/src/main/java/org/apache/calcite/prepare/CalcitePrepareImpl.java
@@ -16,11 +16,9 @@
  */
 package org.apache.calcite.prepare;
 
-import org.apache.calcite.adapter.enumerable.EnumerableBindable;
 import org.apache.calcite.adapter.enumerable.EnumerableCalc;
 import org.apache.calcite.adapter.enumerable.EnumerableConvention;
 import org.apache.calcite.adapter.enumerable.EnumerableInterpretable;
-import org.apache.calcite.adapter.enumerable.EnumerableInterpreterRule;
 import org.apache.calcite.adapter.enumerable.EnumerableRel;
 import org.apache.calcite.adapter.enumerable.EnumerableRules;
 import org.apache.calcite.adapter.enumerable.RexToLixTranslator;
@@ -31,7 +29,6 @@ import org.apache.calcite.avatica.Meta;
 import org.apache.calcite.config.CalciteConnectionConfig;
 import org.apache.calcite.config.CalciteSystemProperty;
 import org.apache.calcite.interpreter.BindableConvention;
-import org.apache.calcite.interpreter.Bindables;
 import org.apache.calcite.interpreter.Interpreters;
 import org.apache.calcite.jdbc.CalcitePrepare;
 import org.apache.calcite.jdbc.CalciteSchema;
@@ -72,33 +69,6 @@ import org.apache.calcite.rel.core.Filter;
 import org.apache.calcite.rel.core.Project;
 import org.apache.calcite.rel.core.Sort;
 import org.apache.calcite.rel.core.TableScan;
-import org.apache.calcite.rel.rules.AbstractMaterializedViewRule;
-import org.apache.calcite.rel.rules.AggregateExpandDistinctAggregatesRule;
-import org.apache.calcite.rel.rules.AggregateReduceFunctionsRule;
-import org.apache.calcite.rel.rules.AggregateStarTableRule;
-import org.apache.calcite.rel.rules.AggregateValuesRule;
-import org.apache.calcite.rel.rules.ExchangeRemoveConstantKeysRule;
-import org.apache.calcite.rel.rules.FilterAggregateTransposeRule;
-import org.apache.calcite.rel.rules.FilterJoinRule;
-import org.apache.calcite.rel.rules.FilterProjectTransposeRule;
-import org.apache.calcite.rel.rules.FilterTableScanRule;
-import org.apache.calcite.rel.rules.JoinAssociateRule;
-import org.apache.calcite.rel.rules.JoinCommuteRule;
-import org.apache.calcite.rel.rules.JoinPushExpressionsRule;
-import org.apache.calcite.rel.rules.JoinPushThroughJoinRule;
-import org.apache.calcite.rel.rules.MaterializedViewFilterScanRule;
-import org.apache.calcite.rel.rules.ProjectFilterTransposeRule;
-import org.apache.calcite.rel.rules.ProjectMergeRule;
-import org.apache.calcite.rel.rules.ProjectTableScanRule;
-import org.apache.calcite.rel.rules.ProjectWindowTransposeRule;
-import org.apache.calcite.rel.rules.ReduceExpressionsRule;
-import org.apache.calcite.rel.rules.SortJoinTransposeRule;
-import org.apache.calcite.rel.rules.SortProjectTransposeRule;
-import org.apache.calcite.rel.rules.SortRemoveConstantKeysRule;
-import org.apache.calcite.rel.rules.SortUnionTransposeRule;
-import org.apache.calcite.rel.rules.TableScanRule;
-import org.apache.calcite.rel.rules.ValuesReduceRule;
-import org.apache.calcite.rel.stream.StreamRules;
 import org.apache.calcite.rel.type.RelDataType;
 import org.apache.calcite.rel.type.RelDataTypeFactory;
 import org.apache.calcite.rel.type.RelDataTypeField;
@@ -165,22 +135,24 @@ import static org.apache.calcite.util.Static.RESOURCE;
  * subject to change without notice.</p>
  */
 public class CalcitePrepareImpl implements CalcitePrepare {
-  /** Whether to enable the collation trait. Some extra optimizations are
-   * possible if enabled, but queries should work either way. At some point
-   * this will become a preference, or we will run multiple phases: first
-   * disabled, then enabled. */
-  private static final boolean ENABLE_COLLATION_TRAIT = true;
+
+  @Deprecated // to be removed before 2.0
+  public static final boolean ENABLE_ENUMERABLE =
+      CalciteSystemProperty.ENABLE_ENUMERABLE.value();
+
+  @Deprecated // to be removed before 2.0
+  public static final boolean ENABLE_STREAM =
+      CalciteSystemProperty.ENABLE_STREAM.value();
+
+  @Deprecated // to be removed before 2.0
+  public static final List<RelOptRule> ENUMERABLE_RULES =
+      EnumerableRules.ENUMERABLE_RULES;
+
 
   /** Whether the bindable convention should be the root convention of any
    * plan. If not, enumerable convention is the default. */
   public final boolean enableBindable = Hook.ENABLE_BINDABLE.get(false);
 
-  /** Whether the enumerable convention is enabled. */
-  public static final boolean ENABLE_ENUMERABLE = true;
-
-  /** Whether the streaming is enabled. */
-  public static final boolean ENABLE_STREAM = true;
-
   private static final Set<String> SIMPLE_SQLS =
       ImmutableSet.of(
           "SELECT 1",
@@ -190,69 +162,6 @@ public class CalcitePrepareImpl implements CalcitePrepare {
           "values 1",
           "VALUES 1");
 
-  public static final List<RelOptRule> ENUMERABLE_RULES =
-      ImmutableList.of(
-          EnumerableRules.ENUMERABLE_JOIN_RULE,
-          EnumerableRules.ENUMERABLE_MERGE_JOIN_RULE,
-          EnumerableRules.ENUMERABLE_SEMI_JOIN_RULE,
-          EnumerableRules.ENUMERABLE_CORRELATE_RULE,
-          EnumerableRules.ENUMERABLE_PROJECT_RULE,
-          EnumerableRules.ENUMERABLE_FILTER_RULE,
-          EnumerableRules.ENUMERABLE_AGGREGATE_RULE,
-          EnumerableRules.ENUMERABLE_SORT_RULE,
-          EnumerableRules.ENUMERABLE_LIMIT_RULE,
-          EnumerableRules.ENUMERABLE_COLLECT_RULE,
-          EnumerableRules.ENUMERABLE_UNCOLLECT_RULE,
-          EnumerableRules.ENUMERABLE_UNION_RULE,
-          EnumerableRules.ENUMERABLE_REPEAT_UNION_RULE,
-          EnumerableRules.ENUMERABLE_TABLE_SPOOL_RULE,
-          EnumerableRules.ENUMERABLE_INTERSECT_RULE,
-          EnumerableRules.ENUMERABLE_MINUS_RULE,
-          EnumerableRules.ENUMERABLE_TABLE_MODIFICATION_RULE,
-          EnumerableRules.ENUMERABLE_VALUES_RULE,
-          EnumerableRules.ENUMERABLE_WINDOW_RULE,
-          EnumerableRules.ENUMERABLE_TABLE_SCAN_RULE,
-          EnumerableRules.ENUMERABLE_TABLE_FUNCTION_SCAN_RULE);
-
-  private static final List<RelOptRule> DEFAULT_RULES =
-      ImmutableList.of(
-          AggregateStarTableRule.INSTANCE,
-          AggregateStarTableRule.INSTANCE2,
-          TableScanRule.INSTANCE,
-          CalciteSystemProperty.COMMUTE.value()
-              ? JoinAssociateRule.INSTANCE
-              : ProjectMergeRule.INSTANCE,
-          FilterTableScanRule.INSTANCE,
-          ProjectFilterTransposeRule.INSTANCE,
-          FilterProjectTransposeRule.INSTANCE,
-          FilterJoinRule.FILTER_ON_JOIN,
-          JoinPushExpressionsRule.INSTANCE,
-          AggregateExpandDistinctAggregatesRule.INSTANCE,
-          AggregateReduceFunctionsRule.INSTANCE,
-          FilterAggregateTransposeRule.INSTANCE,
-          ProjectWindowTransposeRule.INSTANCE,
-          JoinCommuteRule.INSTANCE,
-          JoinPushThroughJoinRule.RIGHT,
-          JoinPushThroughJoinRule.LEFT,
-          SortProjectTransposeRule.INSTANCE,
-          SortJoinTransposeRule.INSTANCE,
-          SortRemoveConstantKeysRule.INSTANCE,
-          SortUnionTransposeRule.INSTANCE,
-          ExchangeRemoveConstantKeysRule.EXCHANGE_INSTANCE,
-          ExchangeRemoveConstantKeysRule.SORT_EXCHANGE_INSTANCE);
-
-  private static final List<RelOptRule> CONSTANT_REDUCTION_RULES =
-      ImmutableList.of(
-          ReduceExpressionsRule.PROJECT_INSTANCE,
-          ReduceExpressionsRule.FILTER_INSTANCE,
-          ReduceExpressionsRule.CALC_INSTANCE,
-          ReduceExpressionsRule.WINDOW_INSTANCE,
-          ReduceExpressionsRule.JOIN_INSTANCE,
-          ValuesReduceRule.FILTER_INSTANCE,
-          ValuesReduceRule.PROJECT_FILTER_INSTANCE,
-          ValuesReduceRule.PROJECT_INSTANCE,
-          AggregateValuesRule.INSTANCE);
-
   public CalcitePrepareImpl() {
   }
 
@@ -522,71 +431,26 @@ public class CalcitePrepareImpl implements CalcitePrepare {
     final VolcanoPlanner planner =
         new VolcanoPlanner(costFactory, externalContext);
     planner.addRelTraitDef(ConventionTraitDef.INSTANCE);
-    if (ENABLE_COLLATION_TRAIT) {
+    if (CalciteSystemProperty.ENABLE_COLLATION_TRAIT.value()) {
       planner.addRelTraitDef(RelCollationTraitDef.INSTANCE);
-      planner.registerAbstractRelationalRules();
-    }
-    RelOptUtil.registerAbstractRels(planner);
-    for (RelOptRule rule : DEFAULT_RULES) {
-      planner.addRule(rule);
     }
-    if (prepareContext.config().materializationsEnabled()) {
-      planner.addRule(MaterializedViewFilterScanRule.INSTANCE);
-      planner.addRule(AbstractMaterializedViewRule.INSTANCE_PROJECT_FILTER);
-      planner.addRule(AbstractMaterializedViewRule.INSTANCE_FILTER);
-      planner.addRule(AbstractMaterializedViewRule.INSTANCE_PROJECT_JOIN);
-      planner.addRule(AbstractMaterializedViewRule.INSTANCE_JOIN);
-      planner.addRule(AbstractMaterializedViewRule.INSTANCE_PROJECT_AGGREGATE);
-      planner.addRule(AbstractMaterializedViewRule.INSTANCE_AGGREGATE);
-    }
-    if (enableBindable) {
-      for (RelOptRule rule : Bindables.RULES) {
-        planner.addRule(rule);
-      }
-    }
-    planner.addRule(Bindables.BINDABLE_TABLE_SCAN_RULE);
-    planner.addRule(ProjectTableScanRule.INSTANCE);
-    planner.addRule(ProjectTableScanRule.INTERPRETER);
+    RelOptUtil.registerDefaultRules(planner,
+        prepareContext.config().materializationsEnabled(),
+        enableBindable);
 
-    if (ENABLE_ENUMERABLE) {
-      for (RelOptRule rule : ENUMERABLE_RULES) {
-        planner.addRule(rule);
-      }
-      planner.addRule(EnumerableInterpreterRule.INSTANCE);
-    }
-
-    if (enableBindable && ENABLE_ENUMERABLE) {
-      planner.addRule(
-          EnumerableBindable.EnumerableToBindableConverterRule.INSTANCE);
-    }
-
-    if (ENABLE_STREAM) {
-      for (RelOptRule rule : StreamRules.RULES) {
-        planner.addRule(rule);
-      }
-    }
-
-    // Change the below to enable constant-reduction.
-    if (false) {
-      for (RelOptRule rule : CONSTANT_REDUCTION_RULES) {
-        planner.addRule(rule);
-      }
-    }
-
-    final SparkHandler spark = prepareContext.spark();
+    final CalcitePrepare.SparkHandler spark = prepareContext.spark();
     if (spark.enabled()) {
       spark.registerRules(
           new SparkHandler.RuleSetBuilder() {
-          public void addRule(RelOptRule rule) {
-            // TODO:
-          }
+            public void addRule(RelOptRule rule) {
+              // TODO:
+            }
 
-          public void removeRule(RelOptRule rule) {
-            // TODO:
-          }
-        });
+            public void removeRule(RelOptRule rule) {
+              // TODO:
+            }
+          });
     }
-
     Hook.PLANNER.run(planner); // allow test to add or remove rules
 
     return planner;
diff --git a/core/src/main/java/org/apache/calcite/prepare/RelOptTableImpl.java b/core/src/main/java/org/apache/calcite/prepare/RelOptTableImpl.java
index b83b153..46eb2c4 100644
--- a/core/src/main/java/org/apache/calcite/prepare/RelOptTableImpl.java
+++ b/core/src/main/java/org/apache/calcite/prepare/RelOptTableImpl.java
@@ -17,6 +17,7 @@
 package org.apache.calcite.prepare;
 
 import org.apache.calcite.adapter.enumerable.EnumerableTableScan;
+import org.apache.calcite.config.CalciteSystemProperty;
 import org.apache.calcite.jdbc.CalciteSchema;
 import org.apache.calcite.linq4j.tree.Expression;
 import org.apache.calcite.materialize.Lattice;
@@ -272,7 +273,7 @@ public class RelOptTableImpl extends Prepare.AbstractPreparingTable {
     if (Hook.ENABLE_BINDABLE.get(false)) {
       return LogicalTableScan.create(cluster, this);
     }
-    if (CalcitePrepareImpl.ENABLE_ENUMERABLE
+    if (CalciteSystemProperty.ENABLE_ENUMERABLE.value()
         && table instanceof QueryableTable) {
       return EnumerableTableScan.create(cluster, this);
     }
@@ -281,7 +282,7 @@ public class RelOptTableImpl extends Prepare.AbstractPreparingTable {
         || table instanceof ProjectableFilterableTable) {
       return LogicalTableScan.create(cluster, this);
     }
-    if (CalcitePrepareImpl.ENABLE_ENUMERABLE) {
+    if (CalciteSystemProperty.ENABLE_ENUMERABLE.value()) {
       return EnumerableTableScan.create(cluster, this);
     }
     throw new AssertionError();
diff --git a/core/src/main/java/org/apache/calcite/tools/Programs.java b/core/src/main/java/org/apache/calcite/tools/Programs.java
index bc1da90..c46fd36 100644
--- a/core/src/main/java/org/apache/calcite/tools/Programs.java
+++ b/core/src/main/java/org/apache/calcite/tools/Programs.java
@@ -19,12 +19,12 @@ package org.apache.calcite.tools;
 import org.apache.calcite.adapter.enumerable.EnumerableRules;
 import org.apache.calcite.config.CalciteConnectionConfig;
 import org.apache.calcite.config.CalciteSystemProperty;
-import org.apache.calcite.interpreter.NoneToBindableConverterRule;
 import org.apache.calcite.plan.RelOptCostImpl;
 import org.apache.calcite.plan.RelOptLattice;
 import org.apache.calcite.plan.RelOptMaterialization;
 import org.apache.calcite.plan.RelOptPlanner;
 import org.apache.calcite.plan.RelOptRule;
+import org.apache.calcite.plan.RelOptRules;
 import org.apache.calcite.plan.RelOptUtil;
 import org.apache.calcite.plan.RelTraitSet;
 import org.apache.calcite.plan.hep.HepMatchOrder;
@@ -40,22 +40,17 @@ import org.apache.calcite.rel.metadata.RelMetadataProvider;
 import org.apache.calcite.rel.rules.AggregateExpandDistinctAggregatesRule;
 import org.apache.calcite.rel.rules.AggregateReduceFunctionsRule;
 import org.apache.calcite.rel.rules.AggregateStarTableRule;
-import org.apache.calcite.rel.rules.CalcMergeRule;
 import org.apache.calcite.rel.rules.FilterAggregateTransposeRule;
-import org.apache.calcite.rel.rules.FilterCalcMergeRule;
 import org.apache.calcite.rel.rules.FilterJoinRule;
 import org.apache.calcite.rel.rules.FilterProjectTransposeRule;
 import org.apache.calcite.rel.rules.FilterTableScanRule;
-import org.apache.calcite.rel.rules.FilterToCalcRule;
 import org.apache.calcite.rel.rules.JoinAssociateRule;
 import org.apache.calcite.rel.rules.JoinCommuteRule;
 import org.apache.calcite.rel.rules.JoinPushThroughJoinRule;
 import org.apache.calcite.rel.rules.JoinToMultiJoinRule;
 import org.apache.calcite.rel.rules.LoptOptimizeJoinRule;
 import org.apache.calcite.rel.rules.MultiJoinOptimizeBushyRule;
-import org.apache.calcite.rel.rules.ProjectCalcMergeRule;
 import org.apache.calcite.rel.rules.ProjectMergeRule;
-import org.apache.calcite.rel.rules.ProjectToCalcRule;
 import org.apache.calcite.rel.rules.SemiJoinRule;
 import org.apache.calcite.rel.rules.SortProjectTransposeRule;
 import org.apache.calcite.rel.rules.SubQueryRemoveRule;
@@ -76,24 +71,8 @@ import java.util.List;
  * Utilities for creating {@link Program}s.
  */
 public class Programs {
-  public static final ImmutableList<RelOptRule> CALC_RULES =
-      ImmutableList.of(
-          NoneToBindableConverterRule.INSTANCE,
-          EnumerableRules.ENUMERABLE_CALC_RULE,
-          EnumerableRules.ENUMERABLE_FILTER_TO_CALC_RULE,
-          EnumerableRules.ENUMERABLE_PROJECT_TO_CALC_RULE,
-          CalcMergeRule.INSTANCE,
-          FilterCalcMergeRule.INSTANCE,
-          ProjectCalcMergeRule.INSTANCE,
-          FilterToCalcRule.INSTANCE,
-          ProjectToCalcRule.INSTANCE,
-          CalcMergeRule.INSTANCE,
-
-          // REVIEW jvs 9-Apr-2006: Do we still need these two?  Doesn't the
-          // combination of CalcMergeRule, FilterToCalcRule, and
-          // ProjectToCalcRule have the same effect?
-          FilterCalcMergeRule.INSTANCE,
-          ProjectCalcMergeRule.INSTANCE);
+  @Deprecated // to be removed before 2.0
+  public static final ImmutableList<RelOptRule> CALC_RULES = RelOptRules.CALC_RULES;
 
   /** Program that converts filters and projects to {@link Calc}s. */
   public static final Program CALC_PROGRAM =
@@ -255,7 +234,7 @@ public class Programs {
   }
 
   public static Program calc(RelMetadataProvider metadataProvider) {
-    return hep(CALC_RULES, true, metadataProvider);
+    return hep(RelOptRules.CALC_RULES, true, metadataProvider);
   }
 
   @Deprecated // to be removed before 2.0