You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@calcite.apache.org by jh...@apache.org on 2020/07/29 00:59:13 UTC

[calcite] branch master updated (b7aad0b -> 885a3da)

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

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


    from b7aad0b  [CALCITE-4139] Prevent NPE in ListTransientTable
     new 998cd83  [CALCITE-3923] Refactor how planner rules are parameterized
     new 8151c6f  [CALCITE-4079] Dialect constants in SqlDialect can cause class initialization deadlock
     new a1bdba6  [CALCITE-4137] Checkstyle should ensure that every class has a Javadoc comment
     new 0431498  [CALCITE-4080] Allow character literals as column aliases, if SqlConformance.allowCharLiteralAlias()
     new e17a5c9  In SqlReturnType, add methods orElse and andThen
     new 03c76a7  [CALCITE-4134] Interval expressions
     new 9b678f1  [CALCITE-4128] Remove dependency of File adapter on Example CSV adapter
     new 19edf52  [CALCITE-2569] UDFs that are table functions must implement SqlTableFunction and have CURSOR as their return type
     new 885a3da  [CALCITE-2160] Spatial: Add functions ST_MakeGrid and ST_MakeGridPoints

The 9 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:
 .../adapter/cassandra/CassandraEnumerator.java     |   2 +-
 .../calcite/adapter/cassandra/CassandraRules.java  | 181 +++--
 .../calcite/adapter/cassandra/CassandraSchema.java |   7 +-
 .../adapter/cassandra/CassandraSchemaFactory.java  |   2 +-
 .../calcite/adapter/cassandra/CassandraTable.java  |   2 +-
 .../CassandraToEnumerableConverterRule.java        |  27 +-
 .../apache/calcite/test/CassandraExtension.java    |   9 +-
 core/src/main/codegen/templates/Parser.jj          | 198 ++++-
 .../calcite/adapter/enumerable/EnumUtils.java      |  27 +-
 .../enumerable/EnumerableAggregateRule.java        |  16 +-
 .../EnumerableBatchNestedLoopJoinRule.java         |  68 +-
 .../adapter/enumerable/EnumerableBindable.java     |  26 +-
 .../adapter/enumerable/EnumerableCalcRule.java     |  26 +-
 .../adapter/enumerable/EnumerableCollectRule.java  |  16 +-
 .../enumerable/EnumerableCorrelateRule.java        |  26 +-
 .../adapter/enumerable/EnumerableFilterRule.java   |  23 +-
 .../enumerable/EnumerableFilterToCalcRule.java     |  37 +-
 .../enumerable/EnumerableInterpreterRule.java      |  25 +-
 .../enumerable/EnumerableIntersectRule.java        |  16 +-
 .../adapter/enumerable/EnumerableJoinRule.java     |  19 +-
 .../adapter/enumerable/EnumerableLimitRule.java    |  28 +-
 .../adapter/enumerable/EnumerableMatchRule.java    |  14 +-
 .../enumerable/EnumerableMergeJoinRule.java        |  15 +-
 .../adapter/enumerable/EnumerableMinusRule.java    |  16 +-
 .../adapter/enumerable/EnumerableProjectRule.java  |  23 +-
 .../enumerable/EnumerableProjectToCalcRule.java    |  36 +-
 .../enumerable/EnumerableRepeatUnionRule.java      |  20 +-
 .../adapter/enumerable/EnumerableRules.java        | 126 ++-
 .../adapter/enumerable/EnumerableSortRule.java     |  14 +-
 .../enumerable/EnumerableSortedAggregateRule.java  |  14 +-
 .../EnumerableTableFunctionScanRule.java           |  40 +-
 .../enumerable/EnumerableTableModifyRule.java      |  28 +-
 .../enumerable/EnumerableTableScanRule.java        |  36 +-
 .../enumerable/EnumerableTableSpoolRule.java       |  15 +-
 .../enumerable/EnumerableUncollectRule.java        |  16 +-
 .../adapter/enumerable/EnumerableUnionRule.java    |  16 +-
 .../adapter/enumerable/EnumerableValuesRule.java   |  25 +-
 .../adapter/enumerable/EnumerableWindowRule.java   |  16 +-
 .../calcite/adapter/enumerable/JavaRowFormat.java  |   2 +-
 .../adapter/enumerable/NestedBlockBuilder.java     |   3 +-
 .../adapter/enumerable/NestedBlockBuilderImpl.java |   3 +-
 .../ReflectiveCallNotNullImplementor.java          |   5 +-
 .../adapter/enumerable/RexToLixTranslator.java     |   2 +-
 .../enumerable/impl/AggResetContextImpl.java       |   2 +-
 .../enumerable/impl/AggResultContextImpl.java      |   2 +-
 .../calcite/adapter/java/ReflectiveSchema.java     |   2 +-
 .../calcite/adapter/jdbc/JdbcImplementor.java      |   1 +
 .../org/apache/calcite/adapter/jdbc/JdbcRules.java | 282 ++++---
 .../apache/calcite/adapter/jdbc/JdbcSchema.java    |   2 +-
 .../jdbc/JdbcToEnumerableConverterRule.java        |  19 +-
 .../calcite/config/CalciteConnectionConfig.java    |  67 +-
 .../calcite/config/CalciteSystemProperty.java      |   5 +-
 .../org/apache/calcite/interpreter/Bindables.java  | 270 ++++---
 .../interpreter/NoneToBindableConverterRule.java   |  25 +-
 .../java/org/apache/calcite/interpreter/Row.java   |   5 +-
 .../apache/calcite/jdbc/ContextSqlValidator.java   |   5 +-
 .../org/apache/calcite/model/ModelHandler.java     |   6 +-
 .../apache/calcite/plan/AbstractRelOptPlanner.java |   9 +-
 .../apache/calcite/plan/CommonRelSubExprRule.java  |  23 +-
 .../java/org/apache/calcite/plan/Convention.java   |   5 +-
 .../org/apache/calcite/plan/RelOptCluster.java     |   1 +
 .../java/org/apache/calcite/plan/RelOptCost.java   |  22 +-
 .../apache/calcite/plan/RelOptMaterialization.java |   4 +-
 .../org/apache/calcite/plan/RelOptPlanner.java     |   8 +-
 .../java/org/apache/calcite/plan/RelOptRule.java   |  38 +-
 .../org/apache/calcite/plan/RelOptRuleCall.java    |   4 +-
 .../org/apache/calcite/plan/RelOptRuleOperand.java |   4 +-
 .../calcite/plan/RelOptRuleOperandChildren.java    |   5 +-
 .../java/org/apache/calcite/plan/RelOptRules.java  |   5 +-
 .../java/org/apache/calcite/plan/RelOptTable.java  |   2 +-
 .../java/org/apache/calcite/plan/RelOptUtil.java   |  47 +-
 .../main/java/org/apache/calcite/plan/RelRule.java | 324 ++++++++
 .../java/org/apache/calcite/plan/RelTraitDef.java  |  10 +-
 .../java/org/apache/calcite/plan/RelTraitSet.java  |   2 +-
 .../org/apache/calcite/plan/TableAccessMap.java    |   4 +-
 .../apache/calcite/plan/VisitorDataContext.java    |   2 +-
 .../org/apache/calcite/plan/hep/HepPlanner.java    |   3 +-
 .../apache/calcite/plan/hep/HepProgramBuilder.java |   3 +-
 .../org/apache/calcite/plan/hep/HepRelVertex.java  |   2 +-
 .../org/apache/calcite/plan/hep/HepRuleCall.java   |   3 +-
 .../calcite/plan/volcano/AbstractConverter.java    |  44 +-
 .../calcite/plan/volcano/IterativeRuleQueue.java   |   2 +-
 .../org/apache/calcite/plan/volcano/RelSet.java    |   6 +-
 .../org/apache/calcite/plan/volcano/RelSubset.java |  34 +-
 .../calcite/plan/volcano/TopDownRuleDriver.java    |   6 +-
 .../calcite/plan/volcano/VolcanoPlanner.java       |   2 +-
 .../calcite/plan/volcano/VolcanoRuleCall.java      |   3 +-
 .../calcite/prepare/CalciteCatalogReader.java      |  11 +-
 .../org/apache/calcite/prepare/PlannerImpl.java    |   5 +-
 .../org/apache/calcite/rel/AbstractRelNode.java    |  68 +-
 .../main/java/org/apache/calcite/rel/RelNode.java  | 126 +--
 .../java/org/apache/calcite/rel/RelWriter.java     |   4 +-
 .../org/apache/calcite/rel/convert/Converter.java  |   2 +-
 .../apache/calcite/rel/convert/ConverterRule.java  | 100 ++-
 .../calcite/rel/convert/TraitMatchingRule.java     |  77 +-
 .../org/apache/calcite/rel/core/Aggregate.java     |   4 +-
 .../java/org/apache/calcite/rel/core/Calc.java     |   6 +
 .../org/apache/calcite/rel/core/CorrelationId.java |   2 +-
 .../java/org/apache/calcite/rel/core/Filter.java   |  12 +-
 .../java/org/apache/calcite/rel/core/Join.java     |  20 +-
 .../java/org/apache/calcite/rel/core/Project.java  |  10 +-
 .../org/apache/calcite/rel/core/RelFactories.java  |  17 +-
 .../org/apache/calcite/rel/core/RepeatUnion.java   |   2 +-
 .../java/org/apache/calcite/rel/core/Snapshot.java |   6 -
 .../java/org/apache/calcite/rel/core/Sort.java     |  34 +-
 .../apache/calcite/rel/core/TableFunctionScan.java |   4 -
 .../org/apache/calcite/rel/core/TableScan.java     |   6 -
 .../calcite/rel/metadata/RelColumnOrigin.java      |  12 +-
 .../calcite/rel/metadata/RelMdPredicates.java      |   1 +
 .../org/apache/calcite/rel/metadata/RelMdUtil.java |  23 +-
 .../apache/calcite/rel/mutable/MutableCalc.java    |   2 +-
 .../calcite/rel/mutable/MutableMultiRel.java       |   2 +-
 .../calcite/rel/rel2sql/RelToSqlConverter.java     |  32 +-
 .../apache/calcite/rel/rel2sql/SqlImplementor.java |  10 +-
 .../rel/rules/AbstractJoinExtractFilterRule.java   |  22 +-
 .../rel/rules/AbstractMaterializedViewRule.java    |  48 --
 .../rel/rules/AggregateCaseToFilterRule.java       |  34 +-
 .../AggregateExpandDistinctAggregatesRule.java     |  62 +-
 .../rel/rules/AggregateExtractProjectRule.java     |  61 +-
 .../rel/rules/AggregateFilterTransposeRule.java    |  58 +-
 .../rel/rules/AggregateJoinJoinRemoveRule.java     |  66 +-
 .../calcite/rel/rules/AggregateJoinRemoveRule.java |  51 +-
 .../rel/rules/AggregateJoinTransposeRule.java      |  79 +-
 .../calcite/rel/rules/AggregateMergeRule.java      |  48 +-
 .../rel/rules/AggregateProjectMergeRule.java       |  46 +-
 .../rules/AggregateProjectPullUpConstantsRule.java |  71 +-
 .../rel/rules/AggregateProjectStarTableRule.java   |  78 ++
 .../rel/rules/AggregateReduceFunctionsRule.java    | 132 +--
 .../calcite/rel/rules/AggregateRemoveRule.java     |  52 +-
 .../calcite/rel/rules/AggregateStarTableRule.java  |  62 +-
 .../rel/rules/AggregateUnionAggregateRule.java     |  89 +-
 .../rel/rules/AggregateUnionTransposeRule.java     |  48 +-
 .../calcite/rel/rules/AggregateValuesRule.java     |  53 +-
 .../apache/calcite/rel/rules/CalcMergeRule.java    |  46 +-
 .../apache/calcite/rel/rules/CalcRemoveRule.java   |  50 +-
 .../apache/calcite/rel/rules/CalcSplitRule.java    |  36 +-
 .../apache/calcite/rel/rules/CoerceInputsRule.java |  87 +-
 .../org/apache/calcite/rel/rules/CoreRules.java    | 470 ++++-------
 .../apache/calcite/rel/rules/DateRangeRules.java   |  68 +-
 .../rel/rules/ExchangeRemoveConstantKeysRule.java  | 199 ++---
 .../rel/rules/FilterAggregateTransposeRule.java    |  72 +-
 .../calcite/rel/rules/FilterCalcMergeRule.java     |  57 +-
 .../calcite/rel/rules/FilterCorrelateRule.java     |  59 +-
 .../apache/calcite/rel/rules/FilterJoinRule.java   | 195 +++--
 .../apache/calcite/rel/rules/FilterMergeRule.java  |  44 +-
 .../rel/rules/FilterMultiJoinMergeRule.java        |  61 +-
 .../rel/rules/FilterProjectTransposeRule.java      | 153 +++-
 .../rules/FilterRemoveIsNotDistinctFromRule.java   |  46 +-
 .../rel/rules/FilterSetOpTransposeRule.java        |  47 +-
 .../rules/FilterTableFunctionTransposeRule.java    |  42 +-
 .../calcite/rel/rules/FilterTableScanRule.java     |  74 +-
 .../apache/calcite/rel/rules/FilterToCalcRule.java |  42 +-
 .../calcite/rel/rules/IntersectToDistinctRule.java |  43 +-
 .../rel/rules/JoinAddRedundantSemiJoinRule.java    |  46 +-
 .../calcite/rel/rules/JoinAssociateRule.java       |  51 +-
 .../apache/calcite/rel/rules/JoinCommuteRule.java  |  86 +-
 .../calcite/rel/rules/JoinExtractFilterRule.java   |  38 +-
 .../rel/rules/JoinProjectTransposeRule.java        | 266 +++---
 .../calcite/rel/rules/JoinPushExpressionsRule.java |  47 +-
 .../calcite/rel/rules/JoinPushThroughJoinRule.java |  87 +-
 .../rules/JoinPushTransitivePredicatesRule.java    |  66 +-
 .../calcite/rel/rules/JoinToCorrelateRule.java     |  71 +-
 .../calcite/rel/rules/JoinToMultiJoinRule.java     |  72 +-
 .../calcite/rel/rules/JoinUnionTransposeRule.java  |  65 +-
 .../org/apache/calcite/rel/rules/LoptJoinTree.java |   4 +-
 .../apache/calcite/rel/rules/LoptMultiJoin.java    | 224 +++---
 .../calcite/rel/rules/LoptOptimizeJoinRule.java    |  54 +-
 .../calcite/rel/rules/LoptSemiJoinOptimizer.java   |  10 +-
 .../org/apache/calcite/rel/rules/MatchRule.java    |  32 +-
 .../rel/rules/MaterializedViewFilterScanRule.java  |  61 +-
 .../org/apache/calcite/rel/rules/MultiJoin.java    |  26 +-
 .../rel/rules/MultiJoinOptimizeBushyRule.java      |  36 +-
 .../rel/rules/MultiJoinProjectTransposeRule.java   |  87 +-
 .../calcite/rel/rules/ProjectCalcMergeRule.java    |  60 +-
 .../rel/rules/ProjectCorrelateTransposeRule.java   | 129 +--
 .../rel/rules/ProjectFilterTransposeRule.java      | 163 ++--
 .../rel/rules/ProjectJoinJoinRemoveRule.java       |  67 +-
 .../calcite/rel/rules/ProjectJoinRemoveRule.java   |  56 +-
 .../rel/rules/ProjectJoinTransposeRule.java        | 124 +--
 .../apache/calcite/rel/rules/ProjectMergeRule.java |  99 ++-
 .../rel/rules/ProjectMultiJoinMergeRule.java       |  61 +-
 .../calcite/rel/rules/ProjectRemoveRule.java       |  46 +-
 .../rel/rules/ProjectSetOpTransposeRule.java       | 101 +--
 .../rel/rules/ProjectSortTransposeRule.java        |  85 --
 .../calcite/rel/rules/ProjectTableScanRule.java    |  77 +-
 .../calcite/rel/rules/ProjectToCalcRule.java       |  44 +-
 .../calcite/rel/rules/ProjectToWindowRule.java     | 119 +--
 .../rel/rules/ProjectWindowTransposeRule.java      |  61 +-
 .../apache/calcite/rel/rules/PruneEmptyRules.java  | 425 ++++++----
 .../apache/calcite/rel/rules/PushProjector.java    |  10 +-
 .../calcite/rel/rules/ReduceDecimalsRule.java      | 136 ++--
 .../calcite/rel/rules/ReduceExpressionsRule.java   | 286 +++++--
 .../rel/rules/SemiJoinFilterTransposeRule.java     |  55 +-
 .../rel/rules/SemiJoinJoinTransposeRule.java       |  91 ++-
 .../rel/rules/SemiJoinProjectTransposeRule.java    |  62 +-
 .../calcite/rel/rules/SemiJoinRemoveRule.java      |  45 +-
 .../org/apache/calcite/rel/rules/SemiJoinRule.java | 131 +--
 .../apache/calcite/rel/rules/SortJoinCopyRule.java |  48 +-
 .../calcite/rel/rules/SortJoinTransposeRule.java   |  51 +-
 .../rel/rules/SortProjectTransposeRule.java        |  85 +-
 .../rel/rules/SortRemoveConstantKeysRule.java      |  30 +-
 .../apache/calcite/rel/rules/SortRemoveRule.java   |  38 +-
 .../calcite/rel/rules/SortUnionTransposeRule.java  |  82 +-
 .../calcite/rel/rules/SubQueryRemoveRule.java      | 241 +++---
 .../apache/calcite/rel/rules/TableScanRule.java    |  35 +-
 .../calcite/rel/rules/UnionEliminatorRule.java     |  45 +-
 .../apache/calcite/rel/rules/UnionMergeRule.java   |  79 +-
 .../rel/rules/UnionPullUpConstantsRule.java        |  50 +-
 .../calcite/rel/rules/UnionToDistinctRule.java     |  54 +-
 .../apache/calcite/rel/rules/ValuesReduceRule.java | 113 ++-
 .../materialize/MaterializedViewAggregateRule.java | 138 ++--
 .../materialize/MaterializedViewJoinRule.java      |  28 +-
 .../MaterializedViewOnlyAggregateRule.java         |  63 +-
 .../MaterializedViewOnlyFilterRule.java            |  40 +-
 .../materialize/MaterializedViewOnlyJoinRule.java  |  40 +-
 .../MaterializedViewProjectAggregateRule.java      |  67 +-
 .../MaterializedViewProjectFilterRule.java         |  44 +-
 .../MaterializedViewProjectJoinRule.java           |  44 +-
 .../rules/materialize/MaterializedViewRule.java    |  79 +-
 .../rules/materialize/MaterializedViewRules.java   |  34 +-
 .../org/apache/calcite/rel/stream/StreamRules.java | 293 ++++---
 .../org/apache/calcite/rel/type/RelDataType.java   |  23 +-
 .../calcite/rel/type/RelDataTypeFactory.java       |   4 +-
 .../java/org/apache/calcite/rex/RexBuilder.java    |  15 +-
 .../java/org/apache/calcite/rex/RexLiteral.java    |   6 +-
 .../org/apache/calcite/rex/RexMultisetUtil.java    |   4 +-
 .../org/apache/calcite/rex/RexPatternFieldRef.java |   2 +-
 .../java/org/apache/calcite/rex/RexProgram.java    |   2 +-
 .../org/apache/calcite/rex/RexProgramBuilder.java  |   6 +-
 .../java/org/apache/calcite/rex/RexShuttle.java    |   8 -
 .../java/org/apache/calcite/rex/RexSimplify.java   |   4 +-
 .../rex/RexSqlReflectiveConvertletTable.java       |   2 +-
 .../org/apache/calcite/rex/RexTableInputRef.java   |   3 +-
 .../main/java/org/apache/calcite/rex/RexUtil.java  |   6 +-
 .../calcite/runtime/CalciteContextException.java   |  15 +-
 .../apache/calcite/runtime/CalciteResource.java    |   6 +
 .../org/apache/calcite/runtime/Enumerables.java    |   2 +-
 .../apache/calcite/runtime/EnumeratorCursor.java   |   3 +-
 .../org/apache/calcite/runtime/GeoFunctions.java   |  96 +++
 .../main/java/org/apache/calcite/runtime/Hook.java |   3 +
 .../java/org/apache/calcite/runtime/Matcher.java   |   2 +-
 .../java/org/apache/calcite/runtime/Pattern.java   |   6 +-
 .../calcite/runtime/ResultSetEnumerable.java       |   4 +-
 .../apache/calcite/runtime/SocketFactoryImpl.java  |  24 +-
 .../org/apache/calcite/runtime/SqlFunctions.java   |   6 +-
 .../calcite/runtime/TrustAllSslSocketFactory.java  |   2 +
 .../java/org/apache/calcite/runtime/Utilities.java |   1 +
 .../schema/impl/ReflectiveFunctionBase.java        |  10 +-
 .../calcite/schema/impl/ScalarFunctionImpl.java    |  31 +
 .../org/apache/calcite/sql/JoinConditionType.java  |  10 +-
 .../java/org/apache/calcite/sql/SqlAccessEnum.java |   2 +-
 .../java/org/apache/calcite/sql/SqlAccessType.java |   4 +-
 .../java/org/apache/calcite/sql/SqlAsOperator.java |   2 +-
 .../apache/calcite/sql/SqlBasicTypeNameSpec.java   |   2 +-
 .../apache/calcite/sql/SqlBinaryStringLiteral.java |  12 +-
 .../main/java/org/apache/calcite/sql/SqlCall.java  |   4 +-
 .../org/apache/calcite/sql/SqlCallBinding.java     |  86 +-
 .../apache/calcite/sql/SqlCharStringLiteral.java   |  11 +-
 .../java/org/apache/calcite/sql/SqlCollation.java  |   6 +-
 .../java/org/apache/calcite/sql/SqlDelete.java     |   4 +-
 .../java/org/apache/calcite/sql/SqlDialect.java    |  14 +-
 .../apache/calcite/sql/SqlExecutableStatement.java |  32 -
 .../java/org/apache/calcite/sql/SqlExplain.java    |  10 +-
 .../java/org/apache/calcite/sql/SqlFunction.java   |  15 +-
 .../java/org/apache/calcite/sql/SqlInsert.java     |   8 +-
 .../apache/calcite/sql/SqlIntervalQualifier.java   |   7 +-
 .../apache/calcite/sql/SqlJdbcFunctionCall.java    |  10 +-
 .../calcite/sql/SqlJsonConstructorNullClause.java  |   2 +-
 .../main/java/org/apache/calcite/sql/SqlKind.java  | 461 ++++-------
 .../java/org/apache/calcite/sql/SqlLiteral.java    |  62 +-
 .../main/java/org/apache/calcite/sql/SqlMerge.java |  25 +-
 .../main/java/org/apache/calcite/sql/SqlNode.java  |  12 +
 .../java/org/apache/calcite/sql/SqlOperator.java   |  27 +-
 .../org/apache/calcite/sql/SqlOperatorBinding.java |  24 +-
 .../org/apache/calcite/sql/SqlOverOperator.java    |   2 +-
 ...perandCountRange.java => SqlTableFunction.java} |  24 +-
 .../java/org/apache/calcite/sql/SqlUpdate.java     |  16 +-
 .../main/java/org/apache/calcite/sql/SqlUtil.java  |  34 +-
 .../java/org/apache/calcite/sql/SqlWindow.java     |   1 +
 .../apache/calcite/sql/SqlWindowTableFunction.java |  91 ++-
 .../apache/calcite/sql/SqlWithinGroupOperator.java |   4 +-
 .../org/apache/calcite/sql/advise/SqlAdvisor.java  |  45 +-
 .../apache/calcite/sql/advise/SqlAdvisorHint2.java |   2 +-
 .../apache/calcite/sql/advise/SqlSimpleParser.java |  27 +-
 .../org/apache/calcite/sql/ddl/SqlDdlNodes.java    |   2 +-
 .../calcite/sql/dialect/BigQuerySqlDialect.java    |  12 +-
 .../apache/calcite/sql/dialect/Db2SqlDialect.java  |   2 +-
 .../calcite/sql/dialect/MssqlSqlDialect.java       |   2 +-
 .../sql/fun/SqlArgumentAssignmentOperator.java     |   2 +-
 .../java/org/apache/calcite/sql/fun/SqlCase.java   |   2 +-
 .../calcite/sql/fun/SqlCoalesceFunction.java       |   3 +-
 .../sql/fun/SqlCollectionTableOperator.java        |   2 +-
 .../apache/calcite/sql/fun/SqlGeoFunctions.java    | 134 ++++
 .../calcite/sql/fun/SqlIntervalOperator.java       |  83 ++
 .../calcite/sql/fun/SqlJsonDepthFunction.java      |   3 +-
 .../calcite/sql/fun/SqlJsonExistsFunction.java     |   5 +-
 .../calcite/sql/fun/SqlJsonKeysFunction.java       |  10 +-
 .../calcite/sql/fun/SqlJsonLengthFunction.java     |   3 +-
 .../calcite/sql/fun/SqlJsonPrettyFunction.java     |   9 +-
 .../calcite/sql/fun/SqlJsonQueryFunction.java      |   7 +-
 .../calcite/sql/fun/SqlJsonRemoveFunction.java     |  13 +-
 .../sql/fun/SqlJsonStorageSizeFunction.java        |  10 +-
 .../calcite/sql/fun/SqlJsonTypeFunction.java       |  12 +-
 .../sql/fun/SqlJsonValueExpressionOperator.java    |   5 +-
 .../calcite/sql/fun/SqlLeadLagAggFunction.java     |  25 +-
 .../sql/fun/SqlLibraryOperatorTableFactory.java    |   4 +-
 .../calcite/sql/fun/SqlLibraryOperators.java       | 105 ++-
 .../calcite/sql/fun/SqlLiteralChainOperator.java   |   2 +-
 .../calcite/sql/fun/SqlRegexpReplaceFunction.java  |  14 +-
 .../calcite/sql/fun/SqlStdOperatorTable.java       |  17 +-
 .../apache/calcite/sql/fun/SqlTrimFunction.java    |   8 +-
 .../apache/calcite/sql/parser/SqlParserPos.java    |  22 +-
 .../apache/calcite/sql/parser/SqlParserUtil.java   |  23 +-
 .../calcite/sql/type/FamilyOperandTypeChecker.java |  18 +-
 .../sql/type/MultisetOperandTypeChecker.java       |   4 +-
 .../org/apache/calcite/sql/type/OperandTypes.java  |  30 +-
 .../org/apache/calcite/sql/type/ReturnTypes.java   | 120 +--
 .../calcite/sql/type/SqlOperandTypeChecker.java    |   6 +-
 .../calcite/sql/type/SqlOperandTypeInference.java  |   2 +
 .../calcite/sql/type/SqlReturnTypeInference.java   |  17 +-
 .../sql/type/SqlReturnTypeInferenceChain.java      |   7 +-
 .../org/apache/calcite/sql/type/SqlTypeFamily.java |  13 +-
 .../org/apache/calcite/sql/type/SqlTypeName.java   |  16 +-
 .../apache/calcite/sql/type/SqlTypeTransform.java  |   2 +
 .../calcite/sql/type/SqlTypeTransformCascade.java  |  10 +-
 .../apache/calcite/sql/type/SqlTypeTransforms.java |   2 +-
 .../org/apache/calcite/sql/type/SqlTypeUtil.java   | 164 +---
 .../org/apache/calcite/sql/util/SqlVisitor.java    |   2 +-
 .../apache/calcite/sql/validate/AggVisitor.java    |  22 +-
 .../sql/validate/AggregatingSelectScope.java       |   2 +-
 .../calcite/sql/validate/ProcedureNamespace.java   |  27 +-
 .../apache/calcite/sql/validate/SelectScope.java   |   4 +-
 .../sql/validate/SqlAbstractConformance.java       |   4 +
 .../calcite/sql/validate/SqlConformance.java       |  19 +
 .../calcite/sql/validate/SqlConformanceEnum.java   |  14 +
 .../apache/calcite/sql/validate/SqlMoniker.java    |   2 +-
 .../sql/validate/SqlUserDefinedTableFunction.java  |  40 +-
 .../sql/validate/SqlUserDefinedTableMacro.java     | 135 +---
 .../apache/calcite/sql/validate/SqlValidator.java  |  16 +-
 .../sql/validate/SqlValidatorCatalogReader.java    |   5 +-
 .../calcite/sql/validate/SqlValidatorImpl.java     |  39 +-
 .../calcite/sql/validate/SqlValidatorScope.java    |   2 +
 .../calcite/sql/validate/SqlValidatorTable.java    |   2 +-
 .../sql/validate/SqlValidatorWithHints.java        |   4 +-
 .../calcite/sql/validate/TableNamespace.java       |   4 +-
 .../sql/validate/implicit/TypeCoercionImpl.java    |   7 +-
 .../calcite/sql2rel/ReflectiveConvertletTable.java |   4 +-
 .../apache/calcite/sql2rel/RelDecorrelator.java    | 348 +++++---
 .../calcite/sql2rel/SqlNodeToRexConverterImpl.java |   9 +-
 .../apache/calcite/sql2rel/SqlToRelConverter.java  |  17 +-
 .../calcite/sql2rel/StandardConvertletTable.java   |  20 +-
 .../apache/calcite/sql2rel/SubQueryConverter.java  |   4 +-
 .../org/apache/calcite/tools/FrameworkConfig.java  |   2 +-
 .../org/apache/calcite/tools/PigRelBuilder.java    |   4 +-
 .../java/org/apache/calcite/tools/Planner.java     |   1 +
 .../java/org/apache/calcite/tools/Programs.java    |   3 +-
 .../java/org/apache/calcite/tools/RelBuilder.java  |  15 +-
 .../java/org/apache/calcite/util/BlackholeMap.java |   2 +-
 .../src/main/java/org/apache/calcite/util/Bug.java |   3 +-
 .../java/org/apache/calcite/util/CancelFlag.java   |   4 +-
 .../org/apache/calcite/util/ConversionUtil.java    |   4 +-
 .../apache/calcite/util/DateTimeStringUtils.java   |   3 +-
 .../org/apache/calcite/util/ImmutableBeans.java    |   2 +-
 .../java/org/apache/calcite/util/NumberUtil.java   |   6 +-
 .../main/java/org/apache/calcite/util/Sources.java |   7 +-
 .../java/org/apache/calcite/util/StackWriter.java  |  16 +-
 .../main/java/org/apache/calcite/util/Util.java    |  11 +-
 .../java/org/apache/calcite/util/XmlOutput.java    |  29 +-
 .../util/graph/AttributedDirectedGraph.java        |   1 +
 .../util/mapping/AbstractSourceMapping.java        |   4 +-
 .../util/mapping/AbstractTargetMapping.java        |   4 +-
 .../apache/calcite/util/trace/CalciteLogger.java   |  24 +-
 .../calcite/runtime/CalciteResource.properties     |   2 +
 .../calcite/adapter/enumerable/TypeFinderTest.java |   3 +-
 .../calcite/jdbc/CalciteRemoteDriverTest.java      |   4 +-
 .../apache/calcite/jdbc/JavaTypeFactoryTest.java   |   4 +-
 .../materialize/NormalizationTrimFieldTest.java    |  23 +-
 .../org/apache/calcite/plan/RelOptUtilTest.java    | 133 ++-
 .../org/apache/calcite/plan/RelWriterTest.java     |  13 +-
 .../plan/volcano/CollationConversionTest.java      |  58 +-
 .../apache/calcite/plan/volcano/ComboRuleTest.java |  62 +-
 .../apache/calcite/plan/volcano/PlannerTests.java  |  73 +-
 .../calcite/plan/volcano/TraitConversionTest.java  |  55 +-
 .../calcite/plan/volcano/TraitPropagationTest.java | 130 +--
 .../calcite/plan/volcano/VolcanoPlannerTest.java   | 319 +++++---
 .../plan/volcano/VolcanoPlannerTraitTest.java      | 182 +++--
 .../rel/rel2sql/RelToSqlConverterStructsTest.java  |  22 +-
 .../calcite/rel/rel2sql/RelToSqlConverterTest.java |  19 +-
 .../calcite/rel/rules/SortRemoveRuleTest.java      |   3 +-
 .../org/apache/calcite/rex/RexBuilderTest.java     |  10 +-
 .../org/apache/calcite/rex/RexProgramTest.java     |   7 +-
 .../org/apache/calcite/rex/RexProgramTestBase.java |   5 +-
 .../runtime/DeterministicAutomatonTest.java        |   2 +-
 .../apache/calcite/sql/parser/SqlParserTest.java   | 171 +++-
 .../apache/calcite/sql/test/AbstractSqlTester.java |   2 +-
 .../java/org/apache/calcite/sql/test/SqlTests.java |   4 +-
 .../calcite/sql/type/RelDataTypeSystemTest.java    |   2 +-
 .../calcite/sql2rel/RelFieldTrimmerTest.java       |   1 +
 .../org/apache/calcite/test/BookstoreSchema.java   |  15 +-
 .../org/apache/calcite/test/CalciteAssert.java     |  62 +-
 .../java/org/apache/calcite/test/DiffTestCase.java |   9 +-
 .../org/apache/calcite/test/HepPlannerTest.java    |  36 +-
 .../org/apache/calcite/test/HierarchySchema.java   |   4 +-
 .../org/apache/calcite/test/JdbcAdapterTest.java   |   5 +-
 .../apache/calcite/test/JdbcFrontLinqBackTest.java |  12 +-
 .../java/org/apache/calcite/test/JdbcTest.java     |  43 +-
 .../java/org/apache/calcite/test/LatticeTest.java  |  10 +-
 .../calcite/test/LogicalProjectDigestTest.java     |   4 +-
 .../java/org/apache/calcite/test/Matchers.java     |   4 +-
 .../org/apache/calcite/test/MockRelOptPlanner.java |   3 +-
 .../apache/calcite/test/MockSqlOperatorTable.java  | 111 ++-
 .../org/apache/calcite/test/MutableRelTest.java    |   6 +-
 .../apache/calcite/test/ReflectiveSchemaTest.java  |  18 +-
 .../org/apache/calcite/test/RelBuilderTest.java    |   8 +-
 .../test/RelMdPercentageOriginalRowsTest.java      |  43 -
 .../org/apache/calcite/test/RelOptRulesTest.java   | 892 +++++++++++----------
 .../org/apache/calcite/test/RelOptTestBase.java    |   1 +
 .../org/apache/calcite/test/RexShuttleTest.java    |   2 +-
 .../apache/calcite/test/SqlHintsConverterTest.java |  60 +-
 .../calcite/test/SqlOperatorBindingTest.java       |  89 +-
 .../apache/calcite/test/SqlToRelConverterTest.java | 144 ++--
 .../org/apache/calcite/test/SqlValidatorTest.java  | 189 +++--
 .../apache/calcite/test/StatesTableFunction.java   | 121 +++
 .../org/apache/calcite/test/TableFunctionTest.java |  32 +-
 .../org/apache/calcite/test/TypeCoercionTest.java  |   2 +-
 .../org/apache/calcite/test/catalog/Fixture.java   |   2 +-
 .../calcite/test/catalog/MockCatalogReader.java    |   2 +-
 .../test/concurrent/ConcurrentTestCommand.java     |   2 +-
 .../concurrent/ConcurrentTestCommandExecutor.java  |   2 +-
 .../concurrent/ConcurrentTestCommandGenerator.java |  10 +-
 .../concurrent/ConcurrentTestCommandScript.java    |   6 +-
 .../test/concurrent/ConcurrentTestPlugin.java      |   2 +-
 .../EnumerableBatchNestedLoopJoinTest.java         |   2 +-
 .../test/enumerable/EnumerableCalcTest.java        |   2 +-
 .../test/enumerable/EnumerableCorrelateTest.java   |  11 +-
 .../enumerable/EnumerableSortedAggregateTest.java  |   5 +-
 .../test/enumerable/EnumerableUncollectTest.java   |   1 +
 .../calcite/test/fuzzer/RexProgramFuzzyTest.java   |   2 +-
 .../org/apache/calcite/tools/FrameworksTest.java   |  31 +-
 .../java/org/apache/calcite/tools/PlannerTest.java | 173 ++--
 .../java/org/apache/calcite/util/BitSetsTest.java  |   4 +-
 .../test/java/org/apache/calcite/util/Smalls.java  |  12 +-
 .../java/org/apache/calcite/util/SourceTest.java   |   2 +-
 .../java/org/apache/calcite/util/TestUtilTest.java |   2 +-
 .../apache/calcite/test/SqlToRelConverterTest.xml  |  12 +-
 core/src/test/resources/sql/misc.iq                |  27 +
 core/src/test/resources/sql/spatial.iq             |  96 ++-
 .../adapter/druid/BinaryOperatorConversion.java    |   3 +-
 .../adapter/druid/CeilOperatorConversion.java      |   3 +-
 .../calcite/adapter/druid/ComplexMetric.java       |  12 +-
 .../calcite/adapter/druid/DruidDateTimeUtils.java  |   8 +-
 .../calcite/adapter/druid/DruidExpressions.java    |  16 +-
 .../calcite/adapter/druid/DruidJsonFilter.java     |  70 +-
 .../apache/calcite/adapter/druid/DruidQuery.java   |  75 +-
 .../apache/calcite/adapter/druid/DruidRules.java   | 462 +++++------
 .../adapter/druid/DruidSqlCastConverter.java       |   3 +-
 .../adapter/druid/DruidSqlOperatorConverter.java   |   3 +-
 .../apache/calcite/adapter/druid/DruidTable.java   |  13 +-
 .../apache/calcite/adapter/druid/DruidType.java    |  13 +-
 .../adapter/druid/ExtractOperatorConversion.java   |  12 +-
 .../adapter/druid/ExtractionDimensionSpec.java     |   6 +-
 .../adapter/druid/FloorOperatorConversion.java     |   3 +-
 .../adapter/druid/NaryOperatorConverter.java       |   3 +-
 .../adapter/druid/SubstringOperatorConversion.java |   2 +-
 .../adapter/druid/TimeExtractionFunction.java      |   8 +-
 .../druid/UnaryPrefixOperatorConversion.java       |   3 +-
 .../druid/UnarySuffixOperatorConversion.java       |   3 +-
 .../calcite/adapter/druid/VirtualColumn.java       |   2 +-
 .../org/apache/calcite/test/DruidAdapter2IT.java   | 127 ++-
 .../org/apache/calcite/test/DruidAdapterIT.java    | 195 ++---
 .../elasticsearch/ElasticsearchConstants.java      |   3 +-
 .../adapter/elasticsearch/ElasticsearchJson.java   |  31 +-
 .../adapter/elasticsearch/ElasticsearchRules.java  |  68 +-
 .../adapter/elasticsearch/ElasticsearchSchema.java |   2 +-
 .../elasticsearch/ElasticsearchSchemaFactory.java  |   3 +-
 .../elasticsearch/ElasticsearchSearchResult.java   |   6 +-
 .../ElasticsearchToEnumerableConverterRule.java    |  26 +-
 .../elasticsearch/ElasticsearchTransport.java      |   3 +-
 .../adapter/elasticsearch/PredicateAnalyzer.java   |  26 +-
 .../adapter/elasticsearch/QueryBuilders.java       |   7 +-
 .../calcite/adapter/elasticsearch/Scrolling.java   |   5 +-
 .../adapter/elasticsearch/AggregationTest.java     |  10 +-
 .../adapter/elasticsearch/BooleanLogicTest.java    |   3 +-
 .../elasticsearch/ElasticSearchAdapterTest.java    |  14 +-
 .../elasticsearch/ElasticsearchVersionTest.java    |   2 +-
 .../elasticsearch/EmbeddedElasticsearchNode.java   |  13 +-
 .../elasticsearch/EmbeddedElasticsearchPolicy.java |   1 +
 .../calcite/adapter/elasticsearch/MatchTest.java   |   5 +-
 .../adapter/elasticsearch/Projection2Test.java     |   6 +-
 .../adapter/elasticsearch/ProjectionTest.java      |   2 +-
 .../adapter/elasticsearch/QueryBuildersTest.java   |   3 +-
 .../apache/calcite/test/ElasticsearchChecker.java  |   2 +-
 example/csv/build.gradle.kts                       |   1 +
 .../calcite/adapter/csv/CsvFilterableTable.java    |   7 +-
 .../adapter/csv/CsvProjectTableScanRule.java       |  39 +-
 .../org/apache/calcite/adapter/csv/CsvRules.java   |   4 +-
 .../calcite/adapter/csv/CsvScannableTable.java     |   7 +-
 .../org/apache/calcite/adapter/csv/CsvSchema.java  |   1 +
 .../calcite/adapter/csv/CsvSchemaFactory.java      |   4 -
 .../adapter/csv/CsvStreamScannableTable.java       |   7 +-
 .../org/apache/calcite/adapter/csv/CsvTable.java   |   2 +
 .../apache/calcite/adapter/csv/CsvTableScan.java   |   3 +-
 .../calcite/adapter/csv/CsvTranslatableTable.java  |   9 +-
 .../apache/calcite/test/ExampleFunctionTest.java   |   2 +-
 file/build.gradle.kts                              |   4 +-
 .../calcite/adapter/file}/CsvEnumerator.java       |  85 +-
 .../apache/calcite/adapter/file}/CsvFieldType.java |   4 +-
 .../adapter/file}/CsvProjectTableScanRule.java     |  41 +-
 .../calcite/adapter/file}/CsvStreamReader.java     |   2 +-
 .../org/apache/calcite/adapter/file}/CsvTable.java |   5 +-
 .../calcite/adapter/file}/CsvTableFactory.java     |   8 +-
 .../apache/calcite/adapter/file}/CsvTableScan.java |   8 +-
 .../adapter/file}/CsvTranslatableTable.java        |  18 +-
 .../org/apache/calcite/adapter/file/FileRules.java |  12 +-
 .../apache/calcite/adapter/file/FileSchema.java    |  55 +-
 .../calcite/adapter/file/FileSchemaFactory.java    |  20 +-
 .../calcite/adapter/file}/JsonEnumerator.java      |   2 +-
 .../calcite/adapter/file}/JsonScannableTable.java  |   2 +-
 .../apache/calcite/adapter/file}/JsonTable.java    |   4 +-
 .../calcite/adapter/file/FileAdapterTest.java      | 727 +++++++----------
 .../calcite/adapter/file/FileAdapterTests.java     | 222 +++++
 .../calcite/adapter/file/FileReaderTest.java       |   2 +-
 .../org/apache/calcite/adapter/file/SqlTest.java   | 419 ----------
 {example/csv => file}/src/test/resources/bug.json  |   2 +-
 {example/csv => file}/src/test/resources/bug.yaml  |   0
 .../src/test/resources/bug/ARCHERS.json            |   0
 .../test/resources/{sales-json => bug}/DATE.csv    |   0
 .../src/test/resources/bug/LONG_EMPS.csv           |   0
 .../src/test/resources/bug/WACKY_COLUMN_NAMES.csv  |   0
 .../src/test/resources/filterable-model.json       |   2 +-
 file/src/test/resources/geo/states.json            | 168 ++++
 .../test/resources/model-with-custom-table.json    |   2 +-
 .../csv => file}/src/test/resources/model.json     |   2 +-
 .../test/resources/{sales-csv => sales}/DEPTS.csv  |   0
 .../resources/{sales-csv => sales}/EMPS.csv.gz     | Bin
 .../csv => file}/src/test/resources/smart.json     |   2 +-
 .../calcite/adapter/geode/rel/GeodeFilter.java     |  14 +-
 .../calcite/adapter/geode/rel/GeodeRules.java      | 104 ++-
 .../calcite/adapter/geode/rel/GeodeTable.java      |   2 +-
 .../geode/rel/GeodeToEnumerableConverterRule.java  |  12 +-
 .../calcite/adapter/geode/rel/package-info.java    |   2 +-
 .../geode/simple/GeodeSimpleScannableTable.java    |   2 +-
 .../calcite/adapter/geode/simple/package-info.java |   2 +-
 .../calcite/adapter/geode/util/GeodeUtils.java     |   2 +-
 .../calcite/adapter/geode/util/package-info.java   |   2 +-
 .../adapter/geode/rel/GeodeAllDataTypesTest.java   |   4 +-
 .../adapter/geode/simple/BookMasterRegionTest.java |   2 +-
 .../adapter/kafka/KafkaMessageEnumerator.java      |   5 +-
 .../calcite/adapter/kafka/KafkaRowConverter.java   |   8 +-
 .../adapter/kafka/KafkaRowConverterImpl.java       |  11 +-
 .../adapter/kafka/KafkaRowConverterTest.java       |  12 +-
 .../apache/calcite/linq4j/EnumerableDefaults.java  |  39 +-
 .../linq4j/tree/ArrayLengthRecordField.java        |   2 +-
 .../apache/calcite/linq4j/tree/BlockBuilder.java   |   6 +-
 .../apache/calcite/linq4j/tree/BlockStatement.java |   4 +-
 .../linq4j/tree/ConstructorDeclaration.java        |   4 +-
 .../linq4j/tree/DeterministicCodeOptimizer.java    |   2 +-
 .../calcite/linq4j/tree/ForEachStatement.java      |   4 +-
 .../apache/calcite/linq4j/tree/ForStatement.java   |   4 +-
 .../calcite/linq4j/tree/FunctionExpression.java    |   4 +-
 .../calcite/linq4j/tree/MethodCallExpression.java  |   4 +-
 .../calcite/linq4j/tree/NewArrayExpression.java    |   4 +-
 .../apache/calcite/linq4j/tree/NewExpression.java  |   4 +-
 .../calcite/linq4j/tree/OptimizeShuttle.java       |   5 +-
 .../org/apache/calcite/linq4j/tree/Primitive.java  |   4 +-
 .../calcite/linq4j/tree/ReflectedPseudoField.java  |   3 +-
 .../calcite/linq4j/MemoryEnumerableTest.java       |   2 +-
 .../calcite/linq4j/test/BlockBuilderBase.java      |   2 +-
 .../calcite/linq4j/test/BlockBuilderTest.java      |  13 +-
 .../calcite/linq4j/test/CorrelateJoinTest.java     |   2 +-
 .../calcite/linq4j/test/DeterministicTest.java     |   8 +-
 .../calcite/linq4j/test/JoinPreserveOrderTest.java |   4 +-
 .../calcite/adapter/mongodb/MongoEnumerator.java   |   4 +-
 .../apache/calcite/adapter/mongodb/MongoRules.java |  61 +-
 .../mongodb/MongoToEnumerableConverterRule.java    |  24 +-
 .../calcite/adapter/mongodb/MongoAdapterTest.java  |   2 +-
 .../apache/calcite/adapter/pig/PigAggregate.java   |  11 +-
 .../org/apache/calcite/adapter/pig/PigFilter.java  |   6 +-
 .../org/apache/calcite/adapter/pig/PigRules.java   |  74 +-
 .../adapter/pig/PigToEnumerableConverterRule.java  |  12 +-
 .../calcite/test/PigRelBuilderStyleTest.java       |  23 +-
 .../org/apache/calcite/piglet/PigConverter.java    |   3 +-
 .../org/apache/calcite/piglet/PigRelBuilder.java   |   6 +-
 .../org/apache/calcite/piglet/PigRelExVisitor.java |   8 +-
 .../calcite/piglet/PigRelOpInnerVisitor.java       |   1 +
 .../org/apache/calcite/piglet/PigRelOpVisitor.java |   6 +-
 .../org/apache/calcite/piglet/PigRelSqlUdfs.java   |   8 +-
 .../calcite/piglet/PigRelToSqlConverter.java       |   1 +
 .../calcite/piglet/PigToSqlAggregateRule.java      |  97 ++-
 .../calcite/piglet/PigUserDefinedFunction.java     |   4 +-
 .../calcite/chinook/ChosenCustomerEmail.java       |   2 +-
 .../org/apache/calcite/chinook/CodesFunction.java  |   2 +-
 .../apache/calcite/chinook/ConnectionFactory.java  |   4 +-
 .../apache/calcite/chinook/EnvironmentFairy.java   |   2 +-
 .../calcite/chinook/StringConcatFunction.java      |   2 +-
 .../RemotePreparedStatementParametersTest.java     |   3 +-
 .../apache/calcite/adapter/redis/package-info.java |   6 +-
 site/_docs/howto.md                                |  73 ++
 site/_docs/reference.md                            |   8 +-
 site/_docs/tutorial.md                             |  50 +-
 .../spark/EnumerableToSparkConverterRule.java      |  56 --
 .../adapter/spark/JdbcToSparkConverterRule.java    |  20 +-
 .../calcite/adapter/spark/SparkHandlerImpl.java    |   2 +-
 .../apache/calcite/adapter/spark/SparkRules.java   |  98 ++-
 .../calcite/adapter/splunk/SplunkPushDownRule.java |  96 ++-
 src/main/config/checkstyle/checker.xml             |  15 +
 .../enumerable/CodeGenerationBenchmark.java        |   2 +-
 .../calcite/adapter/enumerable/package-info.java   |   2 +-
 .../benchmarks/DefaultDirectedGraphBenchmark.java  |   3 +-
 .../apache/calcite/benchmarks/ParserBenchmark.java |   2 +-
 .../calcite/benchmarks/PreconditionTest.java       |   2 +-
 .../apache/calcite/benchmarks/package-info.java    |   2 +-
 612 files changed, 13522 insertions(+), 9682 deletions(-)
 create mode 100644 core/src/main/java/org/apache/calcite/plan/RelRule.java
 delete mode 100644 core/src/main/java/org/apache/calcite/rel/rules/AbstractMaterializedViewRule.java
 create mode 100644 core/src/main/java/org/apache/calcite/rel/rules/AggregateProjectStarTableRule.java
 delete mode 100644 core/src/main/java/org/apache/calcite/rel/rules/ProjectSortTransposeRule.java
 delete mode 100644 core/src/main/java/org/apache/calcite/sql/SqlExecutableStatement.java
 copy core/src/main/java/org/apache/calcite/sql/{SqlOperandCountRange.java => SqlTableFunction.java} (64%)
 create mode 100644 core/src/main/java/org/apache/calcite/sql/fun/SqlGeoFunctions.java
 create mode 100644 core/src/main/java/org/apache/calcite/sql/fun/SqlIntervalOperator.java
 delete mode 100644 core/src/test/java/org/apache/calcite/test/RelMdPercentageOriginalRowsTest.java
 create mode 100644 core/src/test/java/org/apache/calcite/test/StatesTableFunction.java
 rename {example/csv/src/main/java/org/apache/calcite/adapter/csv => file/src/main/java/org/apache/calcite/adapter/file}/CsvEnumerator.java (81%)
 rename {example/csv/src/main/java/org/apache/calcite/adapter/csv => file/src/main/java/org/apache/calcite/adapter/file}/CsvFieldType.java (97%)
 copy {example/csv/src/main/java/org/apache/calcite/adapter/csv => file/src/main/java/org/apache/calcite/adapter/file}/CsvProjectTableScanRule.java (74%)
 rename {example/csv/src/main/java/org/apache/calcite/adapter/csv => file/src/main/java/org/apache/calcite/adapter/file}/CsvStreamReader.java (99%)
 copy {example/csv/src/main/java/org/apache/calcite/adapter/csv => file/src/main/java/org/apache/calcite/adapter/file}/CsvTable.java (94%)
 copy {example/csv/src/main/java/org/apache/calcite/adapter/csv => file/src/main/java/org/apache/calcite/adapter/file}/CsvTableFactory.java (89%)
 copy {example/csv/src/main/java/org/apache/calcite/adapter/csv => file/src/main/java/org/apache/calcite/adapter/file}/CsvTableScan.java (97%)
 copy {example/csv/src/main/java/org/apache/calcite/adapter/csv => file/src/main/java/org/apache/calcite/adapter/file}/CsvTranslatableTable.java (88%)
 copy example/csv/src/main/java/org/apache/calcite/adapter/csv/CsvRules.java => file/src/main/java/org/apache/calcite/adapter/file/FileRules.java (79%)
 rename {example/csv/src/main/java/org/apache/calcite/adapter/csv => file/src/main/java/org/apache/calcite/adapter/file}/JsonEnumerator.java (99%)
 rename {example/csv/src/main/java/org/apache/calcite/adapter/csv => file/src/main/java/org/apache/calcite/adapter/file}/JsonScannableTable.java (97%)
 rename {example/csv/src/main/java/org/apache/calcite/adapter/csv => file/src/main/java/org/apache/calcite/adapter/file}/JsonTable.java (94%)
 copy example/csv/src/test/java/org/apache/calcite/test/CsvTest.java => file/src/test/java/org/apache/calcite/adapter/file/FileAdapterTest.java (59%)
 create mode 100644 file/src/test/java/org/apache/calcite/adapter/file/FileAdapterTests.java
 delete mode 100644 file/src/test/java/org/apache/calcite/adapter/file/SqlTest.java
 copy {example/csv => file}/src/test/resources/bug.json (93%)
 copy {example/csv => file}/src/test/resources/bug.yaml (100%)
 copy {example/csv => file}/src/test/resources/bug/ARCHERS.json (100%)
 copy file/src/test/resources/{sales-json => bug}/DATE.csv (100%)
 copy {example/csv => file}/src/test/resources/bug/LONG_EMPS.csv (100%)
 copy {example/csv => file}/src/test/resources/bug/WACKY_COLUMN_NAMES.csv (100%)
 copy {example/csv => file}/src/test/resources/filterable-model.json (94%)
 create mode 100644 file/src/test/resources/geo/states.json
 copy {example/csv => file}/src/test/resources/model-with-custom-table.json (94%)
 copy {example/csv => file}/src/test/resources/model.json (93%)
 copy file/src/test/resources/{sales-csv => sales}/DEPTS.csv (100%)
 copy file/src/test/resources/{sales-csv => sales}/EMPS.csv.gz (100%)
 copy {example/csv => file}/src/test/resources/smart.json (95%)
 delete mode 100644 spark/src/main/java/org/apache/calcite/adapter/spark/EnumerableToSparkConverterRule.java


[calcite] 07/09: [CALCITE-4128] Remove dependency of File adapter on Example CSV adapter

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

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

commit 9b678f1d7ffe030711d86f475c3a9e9e95bacb8f
Author: Julian Hyde <jh...@apache.org>
AuthorDate: Sun Jul 19 17:23:08 2020 -0700

    [CALCITE-4128] Remove dependency of File adapter on Example CSV adapter
    
    Move some classes from 'example/csv' to 'file', and make CSV adapter
    depend on File adapter. Copy some other classes (so that we can keep the
    'example/csv' versions simple, and add functionality to the 'file'
    adapter).
    
    Rename SqlTest to FileAdapterTest, factor utility methods into
    FileAdapterTests, and copy in a bunch of tests from example/csv/.../CsvTest.
---
 example/csv/build.gradle.kts                       |   1 +
 .../calcite/adapter/csv/CsvFilterableTable.java    |   7 +-
 .../calcite/adapter/csv/CsvScannableTable.java     |   7 +-
 .../org/apache/calcite/adapter/csv/CsvSchema.java  |   1 +
 .../calcite/adapter/csv/CsvSchemaFactory.java      |   4 -
 .../adapter/csv/CsvStreamScannableTable.java       |   7 +-
 .../org/apache/calcite/adapter/csv/CsvTable.java   |   2 +
 .../apache/calcite/adapter/csv/CsvTableScan.java   |   3 +-
 .../calcite/adapter/csv/CsvTranslatableTable.java  |   9 +-
 file/build.gradle.kts                              |   4 +-
 .../calcite/adapter/file}/CsvEnumerator.java       |  85 +-
 .../apache/calcite/adapter/file}/CsvFieldType.java |   4 +-
 .../adapter/file/CsvProjectTableScanRule.java      |  83 ++
 .../calcite/adapter/file}/CsvStreamReader.java     |   2 +-
 .../org/apache/calcite/adapter/file}/CsvTable.java |   5 +-
 ...FileSchemaFactory.java => CsvTableFactory.java} |  41 +-
 .../apache/calcite/adapter/file}/CsvTableScan.java |   8 +-
 .../adapter/file}/CsvTranslatableTable.java        |  18 +-
 .../apache/calcite/adapter/file/FileRules.java}    |  17 +-
 .../apache/calcite/adapter/file/FileSchema.java    |  55 +-
 .../calcite/adapter/file/FileSchemaFactory.java    |  20 +-
 .../calcite/adapter/file}/JsonEnumerator.java      |   2 +-
 .../calcite/adapter/file}/JsonScannableTable.java  |   2 +-
 .../apache/calcite/adapter/file}/JsonTable.java    |   4 +-
 .../calcite/adapter/file/FileAdapterTest.java      | 890 +++++++++++++++++++++
 .../calcite/adapter/file/FileAdapterTests.java     | 222 +++++
 .../org/apache/calcite/adapter/file/SqlTest.java   | 419 ----------
 .../test/resources/bug.json}                       |  24 +-
 file/src/test/resources/bug.yaml                   |  24 +
 .../test/resources/bug/ARCHERS.json}               |  37 +-
 file/src/test/resources/bug/DATE.csv               |   9 +
 file/src/test/resources/bug/LONG_EMPS.csv          |   6 +
 file/src/test/resources/bug/WACKY_COLUMN_NAMES.csv |   6 +
 .../test/resources/filterable-model.json}          |  29 +-
 .../test/resources/model-with-custom-table.json}   |  30 +-
 .../test/resources/model.json}                     |  26 +-
 file/src/test/resources/sales/DEPTS.csv            |   4 +
 file/src/test/resources/sales/EMPS.csv.gz          | Bin 0 -> 262 bytes
 .../test/resources/smart.json}                     |  34 +-
 39 files changed, 1523 insertions(+), 628 deletions(-)

diff --git a/example/csv/build.gradle.kts b/example/csv/build.gradle.kts
index c2ac70f..ca5ea27 100644
--- a/example/csv/build.gradle.kts
+++ b/example/csv/build.gradle.kts
@@ -21,6 +21,7 @@ val sqllineClasspath by configurations.creating {
 
 dependencies {
     api(project(":core"))
+    api(project(":file"))
     api(project(":linq4j"))
 
     implementation("com.fasterxml.jackson.core:jackson-core")
diff --git a/example/csv/src/main/java/org/apache/calcite/adapter/csv/CsvFilterableTable.java b/example/csv/src/main/java/org/apache/calcite/adapter/csv/CsvFilterableTable.java
index f45e4c4..c9cae65 100644
--- a/example/csv/src/main/java/org/apache/calcite/adapter/csv/CsvFilterableTable.java
+++ b/example/csv/src/main/java/org/apache/calcite/adapter/csv/CsvFilterableTable.java
@@ -17,6 +17,8 @@
 package org.apache.calcite.adapter.csv;
 
 import org.apache.calcite.DataContext;
+import org.apache.calcite.adapter.file.CsvEnumerator;
+import org.apache.calcite.adapter.file.CsvFieldType;
 import org.apache.calcite.linq4j.AbstractEnumerable;
 import org.apache.calcite.linq4j.Enumerable;
 import org.apache.calcite.linq4j.Enumerator;
@@ -27,6 +29,7 @@ import org.apache.calcite.rex.RexLiteral;
 import org.apache.calcite.rex.RexNode;
 import org.apache.calcite.schema.FilterableTable;
 import org.apache.calcite.sql.SqlKind;
+import org.apache.calcite.util.ImmutableIntList;
 import org.apache.calcite.util.Source;
 
 import java.util.List;
@@ -53,12 +56,12 @@ public class CsvFilterableTable extends CsvTable
     final List<CsvFieldType> fieldTypes = getFieldTypes(root.getTypeFactory());
     final String[] filterValues = new String[fieldTypes.size()];
     filters.removeIf(filter -> addFilter(filter, filterValues));
-    final int[] fields = CsvEnumerator.identityList(fieldTypes.size());
+    final List<Integer> fields = ImmutableIntList.identity(fieldTypes.size());
     final AtomicBoolean cancelFlag = DataContext.Variable.CANCEL_FLAG.get(root);
     return new AbstractEnumerable<Object[]>() {
       public Enumerator<Object[]> enumerator() {
         return new CsvEnumerator<>(source, cancelFlag, false, filterValues,
-            new CsvEnumerator.ArrayRowConverter(fieldTypes, fields));
+            CsvEnumerator.arrayConverter(fieldTypes, fields, false));
       }
     };
   }
diff --git a/example/csv/src/main/java/org/apache/calcite/adapter/csv/CsvScannableTable.java b/example/csv/src/main/java/org/apache/calcite/adapter/csv/CsvScannableTable.java
index 237373d..bdc2af0 100644
--- a/example/csv/src/main/java/org/apache/calcite/adapter/csv/CsvScannableTable.java
+++ b/example/csv/src/main/java/org/apache/calcite/adapter/csv/CsvScannableTable.java
@@ -17,11 +17,14 @@
 package org.apache.calcite.adapter.csv;
 
 import org.apache.calcite.DataContext;
+import org.apache.calcite.adapter.file.CsvEnumerator;
+import org.apache.calcite.adapter.file.CsvFieldType;
 import org.apache.calcite.linq4j.AbstractEnumerable;
 import org.apache.calcite.linq4j.Enumerable;
 import org.apache.calcite.linq4j.Enumerator;
 import org.apache.calcite.rel.type.RelProtoDataType;
 import org.apache.calcite.schema.ScannableTable;
+import org.apache.calcite.util.ImmutableIntList;
 import org.apache.calcite.util.Source;
 
 import java.util.List;
@@ -46,12 +49,12 @@ public class CsvScannableTable extends CsvTable
 
   public Enumerable<Object[]> scan(DataContext root) {
     final List<CsvFieldType> fieldTypes = getFieldTypes(root.getTypeFactory());
-    final int[] fields = CsvEnumerator.identityList(fieldTypes.size());
+    final List<Integer> fields = ImmutableIntList.identity(fieldTypes.size());
     final AtomicBoolean cancelFlag = DataContext.Variable.CANCEL_FLAG.get(root);
     return new AbstractEnumerable<Object[]>() {
       public Enumerator<Object[]> enumerator() {
         return new CsvEnumerator<>(source, cancelFlag, false, null,
-            new CsvEnumerator.ArrayRowConverter(fieldTypes, fields));
+            CsvEnumerator.arrayConverter(fieldTypes, fields, false));
       }
     };
   }
diff --git a/example/csv/src/main/java/org/apache/calcite/adapter/csv/CsvSchema.java b/example/csv/src/main/java/org/apache/calcite/adapter/csv/CsvSchema.java
index c5780a1..addefd3 100644
--- a/example/csv/src/main/java/org/apache/calcite/adapter/csv/CsvSchema.java
+++ b/example/csv/src/main/java/org/apache/calcite/adapter/csv/CsvSchema.java
@@ -16,6 +16,7 @@
  */
 package org.apache.calcite.adapter.csv;
 
+import org.apache.calcite.adapter.file.JsonScannableTable;
 import org.apache.calcite.schema.Table;
 import org.apache.calcite.schema.impl.AbstractSchema;
 import org.apache.calcite.util.Source;
diff --git a/example/csv/src/main/java/org/apache/calcite/adapter/csv/CsvSchemaFactory.java b/example/csv/src/main/java/org/apache/calcite/adapter/csv/CsvSchemaFactory.java
index 624c6e8..cadeb59 100644
--- a/example/csv/src/main/java/org/apache/calcite/adapter/csv/CsvSchemaFactory.java
+++ b/example/csv/src/main/java/org/apache/calcite/adapter/csv/CsvSchemaFactory.java
@@ -33,10 +33,6 @@ import java.util.Map;
  */
 @SuppressWarnings("UnusedDeclaration")
 public class CsvSchemaFactory implements SchemaFactory {
-  /** Name of the column that is implicitly created in a CSV stream table
-   * to hold the data arrival time. */
-  static final String ROWTIME_COLUMN_NAME = "ROWTIME";
-
   /** Public singleton, per factory contract. */
   public static final CsvSchemaFactory INSTANCE = new CsvSchemaFactory();
 
diff --git a/example/csv/src/main/java/org/apache/calcite/adapter/csv/CsvStreamScannableTable.java b/example/csv/src/main/java/org/apache/calcite/adapter/csv/CsvStreamScannableTable.java
index c5b9497..8285473 100644
--- a/example/csv/src/main/java/org/apache/calcite/adapter/csv/CsvStreamScannableTable.java
+++ b/example/csv/src/main/java/org/apache/calcite/adapter/csv/CsvStreamScannableTable.java
@@ -17,6 +17,8 @@
 package org.apache.calcite.adapter.csv;
 
 import org.apache.calcite.DataContext;
+import org.apache.calcite.adapter.file.CsvEnumerator;
+import org.apache.calcite.adapter.file.CsvFieldType;
 import org.apache.calcite.linq4j.AbstractEnumerable;
 import org.apache.calcite.linq4j.Enumerable;
 import org.apache.calcite.linq4j.Enumerator;
@@ -24,6 +26,7 @@ import org.apache.calcite.rel.type.RelProtoDataType;
 import org.apache.calcite.schema.ScannableTable;
 import org.apache.calcite.schema.StreamableTable;
 import org.apache.calcite.schema.Table;
+import org.apache.calcite.util.ImmutableIntList;
 import org.apache.calcite.util.Source;
 
 import java.util.List;
@@ -52,12 +55,12 @@ public class CsvStreamScannableTable extends CsvScannableTable
 
   public Enumerable<Object[]> scan(DataContext root) {
     final List<CsvFieldType> fieldTypes = getFieldTypes(root.getTypeFactory());
-    final int[] fields = CsvEnumerator.identityList(fieldTypes.size());
+    final List<Integer> fields = ImmutableIntList.identity(fieldTypes.size());
     final AtomicBoolean cancelFlag = DataContext.Variable.CANCEL_FLAG.get(root);
     return new AbstractEnumerable<Object[]>() {
       public Enumerator<Object[]> enumerator() {
         return new CsvEnumerator<>(source, cancelFlag, true, null,
-            new CsvEnumerator.ArrayRowConverter(fieldTypes, fields, true));
+            CsvEnumerator.arrayConverter(fieldTypes, fields, true));
       }
     };
   }
diff --git a/example/csv/src/main/java/org/apache/calcite/adapter/csv/CsvTable.java b/example/csv/src/main/java/org/apache/calcite/adapter/csv/CsvTable.java
index d0f6396..8f7cc78 100644
--- a/example/csv/src/main/java/org/apache/calcite/adapter/csv/CsvTable.java
+++ b/example/csv/src/main/java/org/apache/calcite/adapter/csv/CsvTable.java
@@ -16,6 +16,8 @@
  */
 package org.apache.calcite.adapter.csv;
 
+import org.apache.calcite.adapter.file.CsvEnumerator;
+import org.apache.calcite.adapter.file.CsvFieldType;
 import org.apache.calcite.adapter.java.JavaTypeFactory;
 import org.apache.calcite.rel.type.RelDataType;
 import org.apache.calcite.rel.type.RelDataTypeFactory;
diff --git a/example/csv/src/main/java/org/apache/calcite/adapter/csv/CsvTableScan.java b/example/csv/src/main/java/org/apache/calcite/adapter/csv/CsvTableScan.java
index ed0a885..f591e29 100644
--- a/example/csv/src/main/java/org/apache/calcite/adapter/csv/CsvTableScan.java
+++ b/example/csv/src/main/java/org/apache/calcite/adapter/csv/CsvTableScan.java
@@ -21,6 +21,7 @@ import org.apache.calcite.adapter.enumerable.EnumerableRel;
 import org.apache.calcite.adapter.enumerable.EnumerableRelImplementor;
 import org.apache.calcite.adapter.enumerable.PhysType;
 import org.apache.calcite.adapter.enumerable.PhysTypeImpl;
+import org.apache.calcite.adapter.file.JsonTable;
 import org.apache.calcite.linq4j.tree.Blocks;
 import org.apache.calcite.linq4j.tree.Expressions;
 import org.apache.calcite.linq4j.tree.Primitive;
@@ -44,7 +45,7 @@ import java.util.List;
 /**
  * Relational expression representing a scan of a CSV file.
  *
- * <p>Like any table scan, it serves as a leaf node of a query tree.</p>
+ * <p>Like any table scan, it serves as a leaf node of a query tree.
  */
 public class CsvTableScan extends TableScan implements EnumerableRel {
   final CsvTranslatableTable csvTable;
diff --git a/example/csv/src/main/java/org/apache/calcite/adapter/csv/CsvTranslatableTable.java b/example/csv/src/main/java/org/apache/calcite/adapter/csv/CsvTranslatableTable.java
index 7a4e13c..e183200 100644
--- a/example/csv/src/main/java/org/apache/calcite/adapter/csv/CsvTranslatableTable.java
+++ b/example/csv/src/main/java/org/apache/calcite/adapter/csv/CsvTranslatableTable.java
@@ -17,6 +17,7 @@
 package org.apache.calcite.adapter.csv;
 
 import org.apache.calcite.DataContext;
+import org.apache.calcite.adapter.file.CsvEnumerator;
 import org.apache.calcite.linq4j.AbstractEnumerable;
 import org.apache.calcite.linq4j.Enumerable;
 import org.apache.calcite.linq4j.Enumerator;
@@ -30,6 +31,7 @@ import org.apache.calcite.schema.QueryableTable;
 import org.apache.calcite.schema.SchemaPlus;
 import org.apache.calcite.schema.Schemas;
 import org.apache.calcite.schema.TranslatableTable;
+import org.apache.calcite.util.ImmutableIntList;
 import org.apache.calcite.util.Source;
 
 import java.lang.reflect.Type;
@@ -49,9 +51,8 @@ public class CsvTranslatableTable extends CsvTable
     return "CsvTranslatableTable";
   }
 
-  /** Returns an enumerable over a given projection of the fields.
-   *
-   * <p>Called from generated code. */
+  /** Returns an enumerable over a given projection of the fields. */
+  @SuppressWarnings("unused") // called from generated code
   public Enumerable<Object> project(final DataContext root,
       final int[] fields) {
     final AtomicBoolean cancelFlag = DataContext.Variable.CANCEL_FLAG.get(root);
@@ -61,7 +62,7 @@ public class CsvTranslatableTable extends CsvTable
             source,
             cancelFlag,
             getFieldTypes(root.getTypeFactory()),
-            fields);
+            ImmutableIntList.of(fields));
       }
     };
   }
diff --git a/file/build.gradle.kts b/file/build.gradle.kts
index 1610a03..53555a8 100644
--- a/file/build.gradle.kts
+++ b/file/build.gradle.kts
@@ -16,12 +16,14 @@
  */
 dependencies {
     api(project(":core"))
-    api(project(":example:csv"))
     api(project(":linq4j"))
 
     implementation("com.google.guava:guava")
     implementation("com.joestelmach:natty")
+    implementation("net.sf.opencsv:opencsv")
     implementation("org.apache.calcite.avatica:avatica-core")
+    implementation("commons-io:commons-io")
+    implementation("org.apache.commons:commons-lang3")
     implementation("org.jsoup:jsoup")
 
     testImplementation(project(":core", "testClasses"))
diff --git a/example/csv/src/main/java/org/apache/calcite/adapter/csv/CsvEnumerator.java b/file/src/main/java/org/apache/calcite/adapter/file/CsvEnumerator.java
similarity index 81%
rename from example/csv/src/main/java/org/apache/calcite/adapter/csv/CsvEnumerator.java
rename to file/src/main/java/org/apache/calcite/adapter/file/CsvEnumerator.java
index cdcc967..091a4a1 100644
--- a/example/csv/src/main/java/org/apache/calcite/adapter/csv/CsvEnumerator.java
+++ b/file/src/main/java/org/apache/calcite/adapter/file/CsvEnumerator.java
@@ -14,13 +14,15 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.apache.calcite.adapter.csv;
+package org.apache.calcite.adapter.file;
 
 import org.apache.calcite.adapter.java.JavaTypeFactory;
 import org.apache.calcite.avatica.util.DateTimeUtils;
 import org.apache.calcite.linq4j.Enumerator;
 import org.apache.calcite.rel.type.RelDataType;
 import org.apache.calcite.sql.type.SqlTypeName;
+import org.apache.calcite.util.ImmutableIntList;
+import org.apache.calcite.util.ImmutableNullableList;
 import org.apache.calcite.util.Pair;
 import org.apache.calcite.util.Source;
 
@@ -41,9 +43,9 @@ import java.util.concurrent.atomic.AtomicBoolean;
  *
  * @param <E> Row type
  */
-class CsvEnumerator<E> implements Enumerator<E> {
+public class CsvEnumerator<E> implements Enumerator<E> {
   private final CSVReader reader;
-  private final String[] filterValues;
+  private final List<String> filterValues;
   private final AtomicBoolean cancelFlag;
   private final RowConverter<E> rowConverter;
   private E current;
@@ -60,23 +62,19 @@ class CsvEnumerator<E> implements Enumerator<E> {
         FastDateFormat.getInstance("yyyy-MM-dd HH:mm:ss", gmt);
   }
 
-  CsvEnumerator(Source source, AtomicBoolean cancelFlag,
-      List<CsvFieldType> fieldTypes) {
-    this(source, cancelFlag, fieldTypes, identityList(fieldTypes.size()));
-  }
-
-  CsvEnumerator(Source source, AtomicBoolean cancelFlag,
-      List<CsvFieldType> fieldTypes, int[] fields) {
+  public CsvEnumerator(Source source, AtomicBoolean cancelFlag,
+      List<CsvFieldType> fieldTypes, List<Integer> fields) {
     //noinspection unchecked
     this(source, cancelFlag, false, null,
         (RowConverter<E>) converter(fieldTypes, fields));
   }
 
-  CsvEnumerator(Source source, AtomicBoolean cancelFlag, boolean stream,
+  public CsvEnumerator(Source source, AtomicBoolean cancelFlag, boolean stream,
       String[] filterValues, RowConverter<E> rowConverter) {
     this.cancelFlag = cancelFlag;
     this.rowConverter = rowConverter;
-    this.filterValues = filterValues;
+    this.filterValues = filterValues == null ? null
+        : ImmutableNullableList.copyOf(filterValues);
     try {
       if (stream) {
         this.reader = new CsvStreamReader(source);
@@ -90,15 +88,20 @@ class CsvEnumerator<E> implements Enumerator<E> {
   }
 
   private static RowConverter<?> converter(List<CsvFieldType> fieldTypes,
-      int[] fields) {
-    if (fields.length == 1) {
-      final int field = fields[0];
+      List<Integer> fields) {
+    if (fields.size() == 1) {
+      final int field = fields.get(0);
       return new SingleColumnRowConverter(fieldTypes.get(field), field);
     } else {
-      return new ArrayRowConverter(fieldTypes, fields);
+      return arrayConverter(fieldTypes, fields, false);
     }
   }
 
+  public static RowConverter<Object[]> arrayConverter(
+      List<CsvFieldType> fieldTypes, List<Integer> fields, boolean stream) {
+    return new ArrayRowConverter(fieldTypes, fields, stream);
+  }
+
   /** Deduces the names and types of a table's columns by reading the first line
    * of a CSV file. */
   static RelDataType deduceRowType(JavaTypeFactory typeFactory, Source source,
@@ -108,12 +111,12 @@ class CsvEnumerator<E> implements Enumerator<E> {
 
   /** Deduces the names and types of a table's columns by reading the first line
   * of a CSV file. */
-  static RelDataType deduceRowType(JavaTypeFactory typeFactory, Source source,
-      List<CsvFieldType> fieldTypes, Boolean stream) {
+  public static RelDataType deduceRowType(JavaTypeFactory typeFactory,
+      Source source, List<CsvFieldType> fieldTypes, Boolean stream) {
     final List<RelDataType> types = new ArrayList<>();
     final List<String> names = new ArrayList<>();
     if (stream) {
-      names.add(CsvSchemaFactory.ROWTIME_COLUMN_NAME);
+      names.add(FileSchemaFactory.ROWTIME_COLUMN_NAME);
       types.add(typeFactory.createSqlType(SqlTypeName.TIMESTAMP));
     }
     try (CSVReader reader = openCsv(source)) {
@@ -161,7 +164,7 @@ class CsvEnumerator<E> implements Enumerator<E> {
     return typeFactory.createStructType(Pair.zip(names, types));
   }
 
-  public static CSVReader openCsv(Source source) throws IOException {
+  static CSVReader openCsv(Source source) throws IOException {
     Objects.requireNonNull(source, "source");
     return new CSVReader(source.reader());
   }
@@ -193,7 +196,7 @@ class CsvEnumerator<E> implements Enumerator<E> {
         }
         if (filterValues != null) {
           for (int i = 0; i < strings.length; i++) {
-            String filterValue = filterValues[i];
+            String filterValue = filterValues.get(i);
             if (filterValue != null) {
               if (!filterValue.equals(strings[i])) {
                 continue outer;
@@ -222,7 +225,7 @@ class CsvEnumerator<E> implements Enumerator<E> {
   }
 
   /** Returns an array of integers {0, ..., n - 1}. */
-  static int[] identityList(int n) {
+  public static int[] identityList(int n) {
     int[] integers = new int[n];
     for (int i = 0; i < n; i++) {
       integers[i] = i;
@@ -315,20 +318,16 @@ class CsvEnumerator<E> implements Enumerator<E> {
 
   /** Array row converter. */
   static class ArrayRowConverter extends RowConverter<Object[]> {
-    private final CsvFieldType[] fieldTypes;
-    private final int[] fields;
-    // whether the row to convert is from a stream
+    /** Field types. List must not be null, but any element may be null. */
+    private final List<CsvFieldType> fieldTypes;
+    private final ImmutableIntList fields;
+    /** Whether the row to convert is from a stream. */
     private final boolean stream;
 
-    ArrayRowConverter(List<CsvFieldType> fieldTypes, int[] fields) {
-      this.fieldTypes = fieldTypes.toArray(new CsvFieldType[0]);
-      this.fields = fields;
-      this.stream = false;
-    }
-
-    ArrayRowConverter(List<CsvFieldType> fieldTypes, int[] fields, boolean stream) {
-      this.fieldTypes = fieldTypes.toArray(new CsvFieldType[0]);
-      this.fields = fields;
+    ArrayRowConverter(List<CsvFieldType> fieldTypes, List<Integer> fields,
+        boolean stream) {
+      this.fieldTypes = ImmutableNullableList.copyOf(fieldTypes);
+      this.fields = ImmutableIntList.copyOf(fields);
       this.stream = stream;
     }
 
@@ -341,27 +340,27 @@ class CsvEnumerator<E> implements Enumerator<E> {
     }
 
     public Object[] convertNormalRow(String[] strings) {
-      final Object[] objects = new Object[fields.length];
-      for (int i = 0; i < fields.length; i++) {
-        int field = fields[i];
-        objects[i] = convert(fieldTypes[field], strings[field]);
+      final Object[] objects = new Object[fields.size()];
+      for (int i = 0; i < fields.size(); i++) {
+        int field = fields.get(i);
+        objects[i] = convert(fieldTypes.get(field), strings[field]);
       }
       return objects;
     }
 
     public Object[] convertStreamRow(String[] strings) {
-      final Object[] objects = new Object[fields.length + 1];
+      final Object[] objects = new Object[fields.size() + 1];
       objects[0] = System.currentTimeMillis();
-      for (int i = 0; i < fields.length; i++) {
-        int field = fields[i];
-        objects[i + 1] = convert(fieldTypes[field], strings[field]);
+      for (int i = 0; i < fields.size(); i++) {
+        int field = fields.get(i);
+        objects[i + 1] = convert(fieldTypes.get(field), strings[field]);
       }
       return objects;
     }
   }
 
   /** Single column row converter. */
-  private static class SingleColumnRowConverter extends RowConverter {
+  private static class SingleColumnRowConverter extends RowConverter<Object> {
     private final CsvFieldType fieldType;
     private final int fieldIndex;
 
diff --git a/example/csv/src/main/java/org/apache/calcite/adapter/csv/CsvFieldType.java b/file/src/main/java/org/apache/calcite/adapter/file/CsvFieldType.java
similarity index 97%
rename from example/csv/src/main/java/org/apache/calcite/adapter/csv/CsvFieldType.java
rename to file/src/main/java/org/apache/calcite/adapter/file/CsvFieldType.java
index 3e50e59..9f1d28b 100644
--- a/example/csv/src/main/java/org/apache/calcite/adapter/csv/CsvFieldType.java
+++ b/file/src/main/java/org/apache/calcite/adapter/file/CsvFieldType.java
@@ -14,7 +14,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.apache.calcite.adapter.csv;
+package org.apache.calcite.adapter.file;
 
 import org.apache.calcite.adapter.java.JavaTypeFactory;
 import org.apache.calcite.linq4j.tree.Primitive;
@@ -30,7 +30,7 @@ import java.util.Map;
  * of type {@link #STRING}. But specifying the field type in the header row
  * makes it easier to write SQL.</p>
  */
-enum CsvFieldType {
+public enum CsvFieldType {
   STRING(String.class, "string"),
   BOOLEAN(Primitive.BOOLEAN),
   BYTE(Primitive.BYTE),
diff --git a/file/src/main/java/org/apache/calcite/adapter/file/CsvProjectTableScanRule.java b/file/src/main/java/org/apache/calcite/adapter/file/CsvProjectTableScanRule.java
new file mode 100644
index 0000000..4d5416c
--- /dev/null
+++ b/file/src/main/java/org/apache/calcite/adapter/file/CsvProjectTableScanRule.java
@@ -0,0 +1,83 @@
+/*
+ * 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.adapter.file;
+
+import org.apache.calcite.plan.RelOptRuleCall;
+import org.apache.calcite.plan.RelRule;
+import org.apache.calcite.rel.logical.LogicalProject;
+import org.apache.calcite.rex.RexInputRef;
+import org.apache.calcite.rex.RexNode;
+
+import java.util.List;
+
+/**
+ * Planner rule that projects from a {@link CsvTableScan} scan just the columns
+ * needed to satisfy a projection. If the projection's expressions are trivial,
+ * the projection is removed.
+ *
+ * @see FileRules#PROJECT_SCAN
+ */
+public class CsvProjectTableScanRule
+    extends RelRule<CsvProjectTableScanRule.Config> {
+
+  /** Creates a CsvProjectTableScanRule. */
+  protected CsvProjectTableScanRule(Config config) {
+    super(config);
+  }
+
+  @Override public void onMatch(RelOptRuleCall call) {
+    final LogicalProject project = call.rel(0);
+    final CsvTableScan scan = call.rel(1);
+    int[] fields = getProjectFields(project.getProjects());
+    if (fields == null) {
+      // Project contains expressions more complex than just field references.
+      return;
+    }
+    call.transformTo(
+        new CsvTableScan(
+            scan.getCluster(),
+            scan.getTable(),
+            scan.csvTable,
+            fields));
+  }
+
+  private int[] getProjectFields(List<RexNode> exps) {
+    final int[] fields = new int[exps.size()];
+    for (int i = 0; i < exps.size(); i++) {
+      final RexNode exp = exps.get(i);
+      if (exp instanceof RexInputRef) {
+        fields[i] = ((RexInputRef) exp).getIndex();
+      } else {
+        return null; // not a simple projection
+      }
+    }
+    return fields;
+  }
+
+  /** Rule configuration. */
+  public interface Config extends RelRule.Config {
+    Config DEFAULT = EMPTY
+        .withOperandSupplier(b0 ->
+            b0.operand(LogicalProject.class).oneInput(b1 ->
+                b1.operand(CsvTableScan.class).noInputs()))
+        .as(Config.class);
+
+    @Override default CsvProjectTableScanRule toRule() {
+      return new CsvProjectTableScanRule(this);
+    }
+  }
+}
diff --git a/example/csv/src/main/java/org/apache/calcite/adapter/csv/CsvStreamReader.java b/file/src/main/java/org/apache/calcite/adapter/file/CsvStreamReader.java
similarity index 99%
rename from example/csv/src/main/java/org/apache/calcite/adapter/csv/CsvStreamReader.java
rename to file/src/main/java/org/apache/calcite/adapter/file/CsvStreamReader.java
index e47b3f4..633079a 100644
--- a/example/csv/src/main/java/org/apache/calcite/adapter/csv/CsvStreamReader.java
+++ b/file/src/main/java/org/apache/calcite/adapter/file/CsvStreamReader.java
@@ -14,7 +14,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.apache.calcite.adapter.csv;
+package org.apache.calcite.adapter.file;
 
 import org.apache.calcite.util.Source;
 
diff --git a/example/csv/src/main/java/org/apache/calcite/adapter/csv/CsvTable.java b/file/src/main/java/org/apache/calcite/adapter/file/CsvTable.java
similarity index 94%
copy from example/csv/src/main/java/org/apache/calcite/adapter/csv/CsvTable.java
copy to file/src/main/java/org/apache/calcite/adapter/file/CsvTable.java
index d0f6396..20f0023 100644
--- a/example/csv/src/main/java/org/apache/calcite/adapter/csv/CsvTable.java
+++ b/file/src/main/java/org/apache/calcite/adapter/file/CsvTable.java
@@ -14,7 +14,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.apache.calcite.adapter.csv;
+package org.apache.calcite.adapter.file;
 
 import org.apache.calcite.adapter.java.JavaTypeFactory;
 import org.apache.calcite.rel.type.RelDataType;
@@ -28,6 +28,9 @@ import java.util.List;
 
 /**
  * Base class for table that reads CSV files.
+ *
+ * <p>Copied from {@code CsvFilterableTable} in demo CSV adapter,
+ * with more advanced features.
  */
 public abstract class CsvTable extends AbstractTable {
   protected final Source source;
diff --git a/file/src/main/java/org/apache/calcite/adapter/file/FileSchemaFactory.java b/file/src/main/java/org/apache/calcite/adapter/file/CsvTableFactory.java
similarity index 53%
copy from file/src/main/java/org/apache/calcite/adapter/file/FileSchemaFactory.java
copy to file/src/main/java/org/apache/calcite/adapter/file/CsvTableFactory.java
index a20d1e6..75e992b 100644
--- a/file/src/main/java/org/apache/calcite/adapter/file/FileSchemaFactory.java
+++ b/file/src/main/java/org/apache/calcite/adapter/file/CsvTableFactory.java
@@ -17,40 +17,37 @@
 package org.apache.calcite.adapter.file;
 
 import org.apache.calcite.model.ModelHandler;
-import org.apache.calcite.schema.Schema;
-import org.apache.calcite.schema.SchemaFactory;
+import org.apache.calcite.rel.type.RelDataType;
+import org.apache.calcite.rel.type.RelDataTypeImpl;
+import org.apache.calcite.rel.type.RelProtoDataType;
 import org.apache.calcite.schema.SchemaPlus;
+import org.apache.calcite.schema.TableFactory;
+import org.apache.calcite.util.Source;
+import org.apache.calcite.util.Sources;
 
 import java.io.File;
-import java.util.List;
 import java.util.Map;
 
 /**
- * Factory that creates a {@link FileSchema}.
+ * Factory that creates a {@link CsvTranslatableTable}.
  *
- * <p>Allows a custom schema to be included in a model.json file.
- * See <a href="http://calcite.apache.org/docs/file_adapter.html">File adapter</a>.
+ * <p>Allows a file-based table to be included in a model.json file, even in a
+ * schema that is not based upon {@link FileSchema}.
  */
 @SuppressWarnings("UnusedDeclaration")
-public class FileSchemaFactory implements SchemaFactory {
+public class CsvTableFactory implements TableFactory<CsvTable> {
   // public constructor, per factory contract
-  public FileSchemaFactory() {
+  public CsvTableFactory() {
   }
 
-  public Schema create(SchemaPlus parentSchema, String name,
-      Map<String, Object> operand) {
-    @SuppressWarnings("unchecked") List<Map<String, Object>> tables =
-        (List) operand.get("tables");
-    final File baseDirectory =
+  public CsvTable create(SchemaPlus schema, String name,
+      Map<String, Object> operand, RelDataType rowType) {
+    String fileName = (String) operand.get("file");
+    final File base =
         (File) operand.get(ModelHandler.ExtraOperand.BASE_DIRECTORY.camelName);
-    File directoryFile = baseDirectory;
-    final String directory = (String) operand.get("directory");
-    if (baseDirectory != null && directory != null) {
-      directoryFile = new File(directory);
-      if (!directoryFile.isAbsolute()) {
-        directoryFile = new File(baseDirectory, directory);
-      }
-    }
-    return new FileSchema(parentSchema, name, directoryFile, tables);
+    final Source source = Sources.file(base, fileName);
+    final RelProtoDataType protoRowType =
+        rowType != null ? RelDataTypeImpl.proto(rowType) : null;
+    return new CsvTranslatableTable(source, protoRowType);
   }
 }
diff --git a/example/csv/src/main/java/org/apache/calcite/adapter/csv/CsvTableScan.java b/file/src/main/java/org/apache/calcite/adapter/file/CsvTableScan.java
similarity index 97%
copy from example/csv/src/main/java/org/apache/calcite/adapter/csv/CsvTableScan.java
copy to file/src/main/java/org/apache/calcite/adapter/file/CsvTableScan.java
index ed0a885..8d6eb87 100644
--- a/example/csv/src/main/java/org/apache/calcite/adapter/csv/CsvTableScan.java
+++ b/file/src/main/java/org/apache/calcite/adapter/file/CsvTableScan.java
@@ -14,7 +14,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.apache.calcite.adapter.csv;
+package org.apache.calcite.adapter.file;
 
 import org.apache.calcite.adapter.enumerable.EnumerableConvention;
 import org.apache.calcite.adapter.enumerable.EnumerableRel;
@@ -44,11 +44,11 @@ import java.util.List;
 /**
  * Relational expression representing a scan of a CSV file.
  *
- * <p>Like any table scan, it serves as a leaf node of a query tree.</p>
+ * <p>Like any table scan, it serves as a leaf node of a query tree.
  */
 public class CsvTableScan extends TableScan implements EnumerableRel {
   final CsvTranslatableTable csvTable;
-  final int[] fields;
+  private final int[] fields;
 
   protected CsvTableScan(RelOptCluster cluster, RelOptTable table,
       CsvTranslatableTable csvTable, int[] fields) {
@@ -80,7 +80,7 @@ public class CsvTableScan extends TableScan implements EnumerableRel {
   }
 
   @Override public void register(RelOptPlanner planner) {
-    planner.addRule(CsvRules.PROJECT_SCAN);
+    planner.addRule(FileRules.PROJECT_SCAN);
   }
 
   @Override public RelOptCost computeSelfCost(RelOptPlanner planner,
diff --git a/example/csv/src/main/java/org/apache/calcite/adapter/csv/CsvTranslatableTable.java b/file/src/main/java/org/apache/calcite/adapter/file/CsvTranslatableTable.java
similarity index 88%
copy from example/csv/src/main/java/org/apache/calcite/adapter/csv/CsvTranslatableTable.java
copy to file/src/main/java/org/apache/calcite/adapter/file/CsvTranslatableTable.java
index 7a4e13c..9f86c53 100644
--- a/example/csv/src/main/java/org/apache/calcite/adapter/csv/CsvTranslatableTable.java
+++ b/file/src/main/java/org/apache/calcite/adapter/file/CsvTranslatableTable.java
@@ -14,7 +14,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.apache.calcite.adapter.csv;
+package org.apache.calcite.adapter.file;
 
 import org.apache.calcite.DataContext;
 import org.apache.calcite.linq4j.AbstractEnumerable;
@@ -30,6 +30,7 @@ import org.apache.calcite.schema.QueryableTable;
 import org.apache.calcite.schema.SchemaPlus;
 import org.apache.calcite.schema.Schemas;
 import org.apache.calcite.schema.TranslatableTable;
+import org.apache.calcite.util.ImmutableIntList;
 import org.apache.calcite.util.Source;
 
 import java.lang.reflect.Type;
@@ -37,6 +38,9 @@ import java.util.concurrent.atomic.AtomicBoolean;
 
 /**
  * Table based on a CSV file.
+ *
+ * <p>Copied from {@code CsvTranslatableTable} in demo CSV adapter,
+ * with more advanced features.
  */
 public class CsvTranslatableTable extends CsvTable
     implements QueryableTable, TranslatableTable {
@@ -49,19 +53,15 @@ public class CsvTranslatableTable extends CsvTable
     return "CsvTranslatableTable";
   }
 
-  /** Returns an enumerable over a given projection of the fields.
-   *
-   * <p>Called from generated code. */
+  /** Returns an enumerable over a given projection of the fields. */
+  @SuppressWarnings("unused") // called from generated code
   public Enumerable<Object> project(final DataContext root,
       final int[] fields) {
     final AtomicBoolean cancelFlag = DataContext.Variable.CANCEL_FLAG.get(root);
     return new AbstractEnumerable<Object>() {
       public Enumerator<Object> enumerator() {
-        return new CsvEnumerator<>(
-            source,
-            cancelFlag,
-            getFieldTypes(root.getTypeFactory()),
-            fields);
+        return new CsvEnumerator<>(source, cancelFlag,
+            getFieldTypes(root.getTypeFactory()), ImmutableIntList.of(fields));
       }
     };
   }
diff --git a/file/build.gradle.kts b/file/src/main/java/org/apache/calcite/adapter/file/FileRules.java
similarity index 66%
copy from file/build.gradle.kts
copy to file/src/main/java/org/apache/calcite/adapter/file/FileRules.java
index 1610a03..9c7e228 100644
--- a/file/build.gradle.kts
+++ b/file/src/main/java/org/apache/calcite/adapter/file/FileRules.java
@@ -14,15 +14,14 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-dependencies {
-    api(project(":core"))
-    api(project(":example:csv"))
-    api(project(":linq4j"))
+package org.apache.calcite.adapter.file;
 
-    implementation("com.google.guava:guava")
-    implementation("com.joestelmach:natty")
-    implementation("org.apache.calcite.avatica:avatica-core")
-    implementation("org.jsoup:jsoup")
+/** Planner rules relating to the File adapter. */
+public abstract class FileRules {
+  private FileRules() {}
 
-    testImplementation(project(":core", "testClasses"))
+  /** Rule that matches a {@link org.apache.calcite.rel.core.Project} on
+   * a {@link CsvTableScan} and pushes down projects if possible. */
+  public static final CsvProjectTableScanRule PROJECT_SCAN =
+      CsvProjectTableScanRule.Config.DEFAULT.toRule();
 }
diff --git a/file/src/main/java/org/apache/calcite/adapter/file/FileSchema.java b/file/src/main/java/org/apache/calcite/adapter/file/FileSchema.java
index 74a4003..4fb5622 100644
--- a/file/src/main/java/org/apache/calcite/adapter/file/FileSchema.java
+++ b/file/src/main/java/org/apache/calcite/adapter/file/FileSchema.java
@@ -16,8 +16,6 @@
  */
 package org.apache.calcite.adapter.file;
 
-import org.apache.calcite.adapter.csv.CsvFilterableTable;
-import org.apache.calcite.adapter.csv.JsonScannableTable;
 import org.apache.calcite.schema.SchemaPlus;
 import org.apache.calcite.schema.Table;
 import org.apache.calcite.schema.impl.AbstractSchema;
@@ -47,11 +45,12 @@ class FileSchema extends AbstractSchema {
    * @param parentSchema  Parent schema
    * @param name          Schema name
    * @param baseDirectory Base directory to look for relative files, or null
-   * @param tables        List containing HTML table identifiers
+   * @param tables        List containing HTML table identifiers, or null
    */
   FileSchema(SchemaPlus parentSchema, String name, File baseDirectory,
       List<Map<String, Object>> tables) {
-    this.tables = ImmutableList.copyOf(tables);
+    this.tables = tables == null ? ImmutableList.of()
+        : ImmutableList.copyOf(tables);
     this.baseDirectory = baseDirectory;
   }
 
@@ -91,29 +90,31 @@ class FileSchema extends AbstractSchema {
 
     // Look for files in the directory ending in ".csv", ".csv.gz", ".json",
     // ".json.gz".
-    final Source baseSource = Sources.of(baseDirectory);
-    File[] files = baseDirectory.listFiles((dir, name) -> {
-      final String nameSansGz = trim(name, ".gz");
-      return nameSansGz.endsWith(".csv")
-          || nameSansGz.endsWith(".json");
-    });
-    if (files == null) {
-      System.out.println("directory " + baseDirectory + " not found");
-      files = new File[0];
-    }
-    // Build a map from table name to table; each file becomes a table.
-    for (File file : files) {
-      Source source = Sources.of(file);
-      Source sourceSansGz = source.trim(".gz");
-      final Source sourceSansJson = sourceSansGz.trimOrNull(".json");
-      if (sourceSansJson != null) {
-        addTable(builder, source, sourceSansJson.relative(baseSource).path(),
-            null);
+    if (baseDirectory != null) {
+      final Source baseSource = Sources.of(baseDirectory);
+      File[] files = baseDirectory.listFiles((dir, name) -> {
+        final String nameSansGz = trim(name, ".gz");
+        return nameSansGz.endsWith(".csv")
+            || nameSansGz.endsWith(".json");
+      });
+      if (files == null) {
+        System.out.println("directory " + baseDirectory + " not found");
+        files = new File[0];
       }
-      final Source sourceSansCsv = sourceSansGz.trimOrNull(".csv");
-      if (sourceSansCsv != null) {
-        addTable(builder, source, sourceSansCsv.relative(baseSource).path(),
-            null);
+      // Build a map from table name to table; each file becomes a table.
+      for (File file : files) {
+        Source source = Sources.of(file);
+        Source sourceSansGz = source.trim(".gz");
+        final Source sourceSansJson = sourceSansGz.trimOrNull(".json");
+        if (sourceSansJson != null) {
+          addTable(builder, source, sourceSansJson.relative(baseSource).path(),
+              null);
+        }
+        final Source sourceSansCsv = sourceSansGz.trimOrNull(".csv");
+        if (sourceSansCsv != null) {
+          addTable(builder, source, sourceSansCsv.relative(baseSource).path(),
+              null);
+        }
       }
     }
 
@@ -145,7 +146,7 @@ class FileSchema extends AbstractSchema {
     }
     final Source sourceSansCsv = sourceSansGz.trimOrNull(".csv");
     if (sourceSansCsv != null) {
-      final Table table = new CsvFilterableTable(source, null);
+      final Table table = new CsvTranslatableTable(source, null);
       builder.put(Util.first(tableName, sourceSansCsv.path()), table);
       return true;
     }
diff --git a/file/src/main/java/org/apache/calcite/adapter/file/FileSchemaFactory.java b/file/src/main/java/org/apache/calcite/adapter/file/FileSchemaFactory.java
index a20d1e6..f0cc654 100644
--- a/file/src/main/java/org/apache/calcite/adapter/file/FileSchemaFactory.java
+++ b/file/src/main/java/org/apache/calcite/adapter/file/FileSchemaFactory.java
@@ -33,8 +33,14 @@ import java.util.Map;
  */
 @SuppressWarnings("UnusedDeclaration")
 public class FileSchemaFactory implements SchemaFactory {
-  // public constructor, per factory contract
-  public FileSchemaFactory() {
+  /** Public singleton, per factory contract. */
+  public static final FileSchemaFactory INSTANCE = new FileSchemaFactory();
+
+  /** Name of the column that is implicitly created in a CSV stream table
+   * to hold the data arrival time. */
+  static final String ROWTIME_COLUMN_NAME = "ROWTIME";
+
+  private FileSchemaFactory() {
   }
 
   public Schema create(SchemaPlus parentSchema, String name,
@@ -43,11 +49,15 @@ public class FileSchemaFactory implements SchemaFactory {
         (List) operand.get("tables");
     final File baseDirectory =
         (File) operand.get(ModelHandler.ExtraOperand.BASE_DIRECTORY.camelName);
-    File directoryFile = baseDirectory;
     final String directory = (String) operand.get("directory");
-    if (baseDirectory != null && directory != null) {
+    File directoryFile = null;
+    if (directory != null) {
       directoryFile = new File(directory);
-      if (!directoryFile.isAbsolute()) {
+    }
+    if (baseDirectory != null) {
+      if (directoryFile == null) {
+        directoryFile = baseDirectory;
+      } else if (!directoryFile.isAbsolute()) {
         directoryFile = new File(baseDirectory, directory);
       }
     }
diff --git a/example/csv/src/main/java/org/apache/calcite/adapter/csv/JsonEnumerator.java b/file/src/main/java/org/apache/calcite/adapter/file/JsonEnumerator.java
similarity index 99%
rename from example/csv/src/main/java/org/apache/calcite/adapter/csv/JsonEnumerator.java
rename to file/src/main/java/org/apache/calcite/adapter/file/JsonEnumerator.java
index 0e8f172..7fd0640 100644
--- a/example/csv/src/main/java/org/apache/calcite/adapter/csv/JsonEnumerator.java
+++ b/file/src/main/java/org/apache/calcite/adapter/file/JsonEnumerator.java
@@ -14,7 +14,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.apache.calcite.adapter.csv;
+package org.apache.calcite.adapter.file;
 
 import org.apache.calcite.linq4j.Enumerator;
 import org.apache.calcite.linq4j.Linq4j;
diff --git a/example/csv/src/main/java/org/apache/calcite/adapter/csv/JsonScannableTable.java b/file/src/main/java/org/apache/calcite/adapter/file/JsonScannableTable.java
similarity index 97%
rename from example/csv/src/main/java/org/apache/calcite/adapter/csv/JsonScannableTable.java
rename to file/src/main/java/org/apache/calcite/adapter/file/JsonScannableTable.java
index 0632fdf..2210b7e 100644
--- a/example/csv/src/main/java/org/apache/calcite/adapter/csv/JsonScannableTable.java
+++ b/file/src/main/java/org/apache/calcite/adapter/file/JsonScannableTable.java
@@ -14,7 +14,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.apache.calcite.adapter.csv;
+package org.apache.calcite.adapter.file;
 
 import org.apache.calcite.DataContext;
 import org.apache.calcite.linq4j.AbstractEnumerable;
diff --git a/example/csv/src/main/java/org/apache/calcite/adapter/csv/JsonTable.java b/file/src/main/java/org/apache/calcite/adapter/file/JsonTable.java
similarity index 94%
rename from example/csv/src/main/java/org/apache/calcite/adapter/csv/JsonTable.java
rename to file/src/main/java/org/apache/calcite/adapter/file/JsonTable.java
index 049881e..5fade31 100644
--- a/example/csv/src/main/java/org/apache/calcite/adapter/csv/JsonTable.java
+++ b/file/src/main/java/org/apache/calcite/adapter/file/JsonTable.java
@@ -14,9 +14,9 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.apache.calcite.adapter.csv;
+package org.apache.calcite.adapter.file;
 
-import org.apache.calcite.adapter.csv.JsonEnumerator.JsonDataConverter;
+import org.apache.calcite.adapter.file.JsonEnumerator.JsonDataConverter;
 import org.apache.calcite.rel.type.RelDataType;
 import org.apache.calcite.rel.type.RelDataTypeFactory;
 import org.apache.calcite.schema.Statistic;
diff --git a/file/src/test/java/org/apache/calcite/adapter/file/FileAdapterTest.java b/file/src/test/java/org/apache/calcite/adapter/file/FileAdapterTest.java
new file mode 100644
index 0000000..5682b77
--- /dev/null
+++ b/file/src/test/java/org/apache/calcite/adapter/file/FileAdapterTest.java
@@ -0,0 +1,890 @@
+/*
+ * 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.adapter.file;
+
+import org.apache.calcite.jdbc.CalciteConnection;
+import org.apache.calcite.schema.Schema;
+import org.apache.calcite.sql2rel.SqlToRelConverter;
+import org.apache.calcite.util.TestUtil;
+
+import com.google.common.collect.ImmutableMap;
+
+import org.junit.jupiter.api.Disabled;
+import org.junit.jupiter.api.Test;
+import org.junit.jupiter.api.extension.ExtendWith;
+
+import java.sql.Connection;
+import java.sql.Date;
+import java.sql.DriverManager;
+import java.sql.PreparedStatement;
+import java.sql.ResultSet;
+import java.sql.ResultSetMetaData;
+import java.sql.SQLException;
+import java.sql.Statement;
+import java.sql.Time;
+import java.sql.Timestamp;
+import java.sql.Types;
+import java.util.Properties;
+import java.util.function.Consumer;
+
+import static org.apache.calcite.adapter.file.FileAdapterTests.sql;
+
+import static org.hamcrest.CoreMatchers.equalTo;
+import static org.hamcrest.CoreMatchers.is;
+import static org.hamcrest.CoreMatchers.isA;
+import static org.hamcrest.MatcherAssert.assertThat;
+import static org.junit.jupiter.api.Assertions.assertEquals;
+
+/**
+ * System test of the Calcite file adapter, which can read and parse
+ * HTML tables over HTTP, and also read CSV and JSON files from the filesystem.
+ */
+@ExtendWith(RequiresNetworkExtension.class)
+class FileAdapterTest {
+  /** Reads from a local file and checks the result. */
+  @Test void testFileSelect() {
+    final String sql = "select H1 from T1 where H0 = 'R1C0'";
+    sql("testModel", sql).returns("H1=R1C1").ok();
+  }
+
+  /** Reads from a local file without table headers &lt;TH&gt; and checks the
+   * result. */
+  @Test @RequiresNetwork void testNoThSelect() {
+    final String sql = "select \"col1\" from T1_NO_TH where \"col0\" like 'R0%'";
+    sql("testModel", sql).returns("col1=R0C1").ok();
+  }
+
+  /** Reads from a local file - finds larger table even without &lt;TH&gt;
+   * elements. */
+  @Test void testFindBiggerNoTh() {
+    final String sql = "select \"col4\" from TABLEX2 where \"col0\" like 'R1%'";
+    sql("testModel", sql).returns("col4=R1C4").ok();
+  }
+
+  /** Reads from a URL and checks the result. */
+  @Disabled("[CALCITE-1789] Wikipedia format change breaks file adapter test")
+  @Test @RequiresNetwork void testUrlSelect() {
+    final String sql = "select \"State\", \"Statehood\" from \"States_as_of\"\n"
+        + "where \"State\" = 'California'";
+    sql("wiki", sql).returns("State=California; Statehood=1850-09-09").ok();
+  }
+
+  /** Reads the EMPS table. */
+  @Test void testSalesEmps() {
+    final String sql = "select * from sales.emps";
+    sql("sales", sql)
+        .returns("EMPNO=100; NAME=Fred; DEPTNO=30",
+            "EMPNO=110; NAME=Eric; DEPTNO=20",
+            "EMPNO=110; NAME=John; DEPTNO=40",
+            "EMPNO=120; NAME=Wilma; DEPTNO=20",
+            "EMPNO=130; NAME=Alice; DEPTNO=40")
+        .ok();
+  }
+
+  /** Reads the DEPTS table. */
+  @Test void testSalesDepts() {
+    final String sql = "select * from sales.depts";
+    sql("sales", sql)
+        .returns("DEPTNO=10; NAME=Sales",
+            "DEPTNO=20; NAME=Marketing",
+            "DEPTNO=30; NAME=Accounts")
+        .ok();
+  }
+
+  /** Reads the DEPTS table from the CSV schema. */
+  @Test void testCsvSalesDepts() {
+    final String sql = "select * from sales.depts";
+    sql("sales-csv", sql)
+        .returns("DEPTNO=10; NAME=Sales",
+            "DEPTNO=20; NAME=Marketing",
+            "DEPTNO=30; NAME=Accounts")
+        .ok();
+  }
+
+  /** Reads the EMPS table from the CSV schema. */
+  @Test void testCsvSalesEmps() {
+    final String sql = "select * from sales.emps";
+    final String[] lines = {
+        "EMPNO=100; NAME=Fred; DEPTNO=10; GENDER=; CITY=; EMPID=30; AGE=25; SLACKER=true; MANAGER=false; JOINEDAT=1996-08-03",
+        "EMPNO=110; NAME=Eric; DEPTNO=20; GENDER=M; CITY=San Francisco; EMPID=3; AGE=80; SLACKER=null; MANAGER=false; JOINEDAT=2001-01-01",
+        "EMPNO=110; NAME=John; DEPTNO=40; GENDER=M; CITY=Vancouver; EMPID=2; AGE=null; SLACKER=false; MANAGER=true; JOINEDAT=2002-05-03",
+        "EMPNO=120; NAME=Wilma; DEPTNO=20; GENDER=F; CITY=; EMPID=1; AGE=5; SLACKER=null; MANAGER=true; JOINEDAT=2005-09-07",
+        "EMPNO=130; NAME=Alice; DEPTNO=40; GENDER=F; CITY=Vancouver; EMPID=2; AGE=null; SLACKER=false; MANAGER=true; JOINEDAT=2007-01-01",
+    };
+    sql("sales-csv", sql).returns(lines).ok();
+  }
+
+  /** Reads the HEADER_ONLY table from the CSV schema. The CSV file has one
+   * line - the column headers - but no rows of data. */
+  @Test void testCsvSalesHeaderOnly() {
+    final String sql = "select * from sales.header_only";
+    sql("sales-csv", sql).returns().ok();
+  }
+
+  /** Reads the EMPTY table from the CSV schema. The CSV file has no lines,
+   * therefore the table has a system-generated column called
+   * "EmptyFileHasNoColumns". */
+  @Test void testCsvSalesEmpty() {
+    final String sql = "select * from sales.\"EMPTY\"";
+    sql("sales-csv", sql)
+        .checking(FileAdapterTest::checkEmpty)
+        .ok();
+  }
+
+  private static void checkEmpty(ResultSet resultSet) {
+    try {
+      final ResultSetMetaData metaData = resultSet.getMetaData();
+      assertThat(metaData.getColumnCount(), is(1));
+      assertThat(metaData.getColumnName(1), is("EmptyFileHasNoColumns"));
+      assertThat(metaData.getColumnType(1), is(Types.BOOLEAN));
+      String actual = FileAdapterTests.toString(resultSet);
+      assertThat(actual, is(""));
+    } catch (SQLException e) {
+      throw TestUtil.rethrow(e);
+    }
+  }
+
+  /** Test case for
+   * <a href="https://issues.apache.org/jira/browse/CALCITE-1754">[CALCITE-1754]
+   * In Csv adapter, convert DATE and TIME values to int, and TIMESTAMP values
+   * to long</a>. */
+  @Test void testCsvGroupByTimestampAdd() {
+    final String sql = "select count(*) as c,\n"
+        + "  {fn timestampadd(SQL_TSI_DAY, 1, JOINEDAT) } as t\n"
+        + "from EMPS group by {fn timestampadd(SQL_TSI_DAY, 1, JOINEDAT ) } ";
+    sql("sales-csv", sql)
+        .returnsUnordered("C=1; T=1996-08-04",
+            "C=1; T=2002-05-04",
+            "C=1; T=2005-09-08",
+            "C=1; T=2007-01-02",
+            "C=1; T=2001-01-02")
+        .ok();
+    final String sql2 = "select count(*) as c,\n"
+        + "  {fn timestampadd(SQL_TSI_MONTH, 1, JOINEDAT) } as t\n"
+        + "from EMPS group by {fn timestampadd(SQL_TSI_MONTH, 1, JOINEDAT ) } ";
+    sql("sales-csv", sql2)
+        .returnsUnordered("C=1; T=2002-06-03",
+            "C=1; T=2005-10-07",
+            "C=1; T=2007-02-01",
+            "C=1; T=2001-02-01",
+            "C=1; T=1996-09-03").ok();
+    final String sql3 = "select\n"
+        + " distinct {fn timestampadd(SQL_TSI_MONTH, 1, JOINEDAT) } as t\n"
+        + "from EMPS";
+    sql("sales-csv", sql3)
+        .returnsUnordered("T=2002-06-03",
+            "T=2005-10-07",
+            "T=2007-02-01",
+            "T=2001-02-01",
+            "T=1996-09-03").ok();
+  }
+
+  /** Reads the DEPTS table from the JSON schema. */
+  @Test void testJsonSalesDepts() {
+    final String sql = "select * from sales.depts";
+    sql("sales-json", sql)
+        .returns("DEPTNO=10; NAME=Sales",
+            "DEPTNO=20; NAME=Marketing",
+            "DEPTNO=30; NAME=Accounts")
+        .ok();
+  }
+
+  /** Reads the EMPS table from the JSON schema. */
+  @Test void testJsonSalesEmps() {
+    final String sql = "select * from sales.emps";
+    final String[] lines = {
+        "EMPNO=100; NAME=Fred; DEPTNO=10; GENDER=; CITY=; EMPID=30; AGE=25; SLACKER=true; MANAGER=false; JOINEDAT=1996-08-03",
+        "EMPNO=110; NAME=Eric; DEPTNO=20; GENDER=M; CITY=San Francisco; EMPID=3; AGE=80; SLACKER=null; MANAGER=false; JOINEDAT=2001-01-01",
+        "EMPNO=110; NAME=John; DEPTNO=40; GENDER=M; CITY=Vancouver; EMPID=2; AGE=null; SLACKER=false; MANAGER=true; JOINEDAT=2002-05-03",
+        "EMPNO=120; NAME=Wilma; DEPTNO=20; GENDER=F; CITY=; EMPID=1; AGE=5; SLACKER=null; MANAGER=true; JOINEDAT=2005-09-07",
+        "EMPNO=130; NAME=Alice; DEPTNO=40; GENDER=F; CITY=Vancouver; EMPID=2; AGE=null; SLACKER=false; MANAGER=true; JOINEDAT=2007-01-01",
+    };
+    sql("sales-json", sql).returns(lines).ok();
+  }
+
+  /** Reads the EMPTY table from the JSON schema. The JSON file has no lines,
+   * therefore the table has a system-generated column called
+   * "EmptyFileHasNoColumns". */
+  @Test void testJsonSalesEmpty() {
+    final String sql = "select * from sales.\"EMPTY\"";
+    sql("sales-json", sql)
+        .checking(FileAdapterTest::checkEmpty)
+        .ok();
+  }
+
+  /** Test returns the result of two json file joins. */
+  @Test void testJsonJoinOnString() {
+    final String sql = "select emps.EMPNO, emps.NAME, depts.deptno from emps\n"
+        + "join depts on emps.deptno = depts.deptno";
+    final String[] lines = {
+        "EMPNO=100; NAME=Fred; DEPTNO=10",
+        "EMPNO=110; NAME=Eric; DEPTNO=20",
+        "EMPNO=120; NAME=Wilma; DEPTNO=20",
+    };
+    sql("sales-json", sql).returns(lines).ok();
+  }
+
+  /** The folder contains both JSON files and CSV files joins. */
+  @Test void testJsonWithCsvJoin() {
+    final String sql = "select emps.empno,\n"
+        + " NAME,\n"
+        + " \"DATE\".JOINEDAT\n"
+        + " from \"DATE\"\n"
+        + "join emps on emps.empno = \"DATE\".EMPNO\n"
+        + "order by empno, name, joinedat limit 3";
+    final String[] lines = {
+        "EMPNO=100; NAME=Fred; JOINEDAT=1996-08-03",
+        "EMPNO=110; NAME=Eric; JOINEDAT=2001-01-01",
+        "EMPNO=110; NAME=Eric; JOINEDAT=2002-05-03",
+    };
+    sql("sales-json", sql)
+        .returns(lines)
+        .ok();
+  }
+
+  /** Tests an inline schema with a non-existent directory. */
+  @Test void testBadDirectory() throws SQLException {
+    Properties info = new Properties();
+    info.put("model",
+        "inline:"
+            + "{\n"
+            + "  version: '1.0',\n"
+            + "   schemas: [\n"
+            + "     {\n"
+            + "       type: 'custom',\n"
+            + "       name: 'bad',\n"
+            + "       factory: 'org.apache.calcite.adapter.file.FileSchemaFactory',\n"
+            + "       operand: {\n"
+            + "         directory: '/does/not/exist'\n"
+            + "       }\n"
+            + "     }\n"
+            + "   ]\n"
+            + "}");
+
+    Connection connection =
+        DriverManager.getConnection("jdbc:calcite:", info);
+    // must print "directory ... not found" to stdout, but not fail
+    ResultSet tables =
+        connection.getMetaData().getTables(null, null, null, null);
+    tables.next();
+    tables.close();
+    connection.close();
+  }
+
+  /**
+   * Reads from a table.
+   */
+  @Test void testSelect() {
+    sql("model", "select * from EMPS").ok();
+  }
+
+  @Test void testSelectSingleProjectGz() {
+    sql("smart", "select name from EMPS").ok();
+  }
+
+  @Test void testSelectSingleProject() {
+    sql("smart", "select name from DEPTS").ok();
+  }
+
+  /** Test case for
+   * <a href="https://issues.apache.org/jira/browse/CALCITE-898">[CALCITE-898]
+   * Type inference multiplying Java long by SQL INTEGER</a>. */
+  @Test void testSelectLongMultiplyInteger() {
+    final String sql = "select empno * 3 as e3\n"
+        + "from long_emps where empno = 100";
+
+    sql("bug", sql).checking(resultSet -> {
+      try {
+        assertThat(resultSet.next(), is(true));
+        Long o = (Long) resultSet.getObject(1);
+        assertThat(o, is(300L));
+        assertThat(resultSet.next(), is(false));
+      } catch (SQLException e) {
+        throw TestUtil.rethrow(e);
+      }
+    }).ok();
+  }
+
+  @Test void testCustomTable() {
+    sql("model-with-custom-table", "select * from CUSTOM_TABLE.EMPS").ok();
+  }
+
+  @Test void testPushDownProject() {
+    final String sql = "explain plan for select * from EMPS";
+    final String expected = "PLAN=CsvTableScan(table=[[SALES, EMPS]], "
+        + "fields=[[0, 1, 2, 3, 4, 5, 6, 7, 8, 9]])\n";
+    sql("smart", sql).returns(expected).ok();
+  }
+
+  @Test void testPushDownProject2() {
+    sql("smart", "explain plan for select name, empno from EMPS")
+        .returns("PLAN=CsvTableScan(table=[[SALES, EMPS]], fields=[[1, 0]])\n")
+        .ok();
+    // make sure that it works...
+    sql("smart", "select name, empno from EMPS")
+        .returns("NAME=Fred; EMPNO=100",
+            "NAME=Eric; EMPNO=110",
+            "NAME=John; EMPNO=110",
+            "NAME=Wilma; EMPNO=120",
+            "NAME=Alice; EMPNO=130")
+        .ok();
+  }
+
+  @Test void testPushDownProjectAggregate() {
+    final String sql = "explain plan for\n"
+        + "select gender, count(*) from EMPS group by gender";
+    final String expected = "PLAN="
+        + "EnumerableAggregate(group=[{0}], EXPR$1=[COUNT()])\n"
+        + "  CsvTableScan(table=[[SALES, EMPS]], fields=[[3]])\n";
+    sql("smart", sql).returns(expected).ok();
+  }
+
+  @Test void testPushDownProjectAggregateWithFilter() {
+    final String sql = "explain plan for\n"
+        + "select max(empno) from EMPS where gender='F'";
+    final String expected = "PLAN="
+        + "EnumerableAggregate(group=[{}], EXPR$0=[MAX($0)])\n"
+        + "  EnumerableCalc(expr#0..1=[{inputs}], expr#2=['F':VARCHAR], "
+        + "expr#3=[=($t1, $t2)], proj#0..1=[{exprs}], $condition=[$t3])\n"
+        + "    CsvTableScan(table=[[SALES, EMPS]], fields=[[0, 3]])\n";
+    sql("smart", sql).returns(expected).ok();
+  }
+
+  @Test void testPushDownProjectAggregateNested() {
+    final String sql = "explain plan for\n"
+        + "select gender, max(qty)\n"
+        + "from (\n"
+        + "  select name, gender, count(*) qty\n"
+        + "  from EMPS\n"
+        + "  group by name, gender) t\n"
+        + "group by gender";
+    final String expected = "PLAN="
+        + "EnumerableAggregate(group=[{1}], EXPR$1=[MAX($2)])\n"
+        + "  EnumerableAggregate(group=[{0, 1}], QTY=[COUNT()])\n"
+        + "    CsvTableScan(table=[[SALES, EMPS]], fields=[[1, 3]])\n";
+    sql("smart", sql).returns(expected).ok();
+  }
+
+  @Test void testFilterableSelect() {
+    sql("filterable-model", "select name from EMPS").ok();
+  }
+
+  @Test void testFilterableSelectStar() {
+    sql("filterable-model", "select * from EMPS").ok();
+  }
+
+  /** Filter that can be fully handled by CsvFilterableTable. */
+  @Test void testFilterableWhere() {
+    final String sql =
+        "select empno, gender, name from EMPS where name = 'John'";
+    sql("filterable-model", sql)
+        .returns("EMPNO=110; GENDER=M; NAME=John").ok();
+  }
+
+  /** Filter that can be partly handled by CsvFilterableTable. */
+  @Test void testFilterableWhere2() {
+    final String sql = "select empno, gender, name from EMPS\n"
+        + " where gender = 'F' and empno > 125";
+    sql("filterable-model", sql)
+        .returns("EMPNO=130; GENDER=F; NAME=Alice").ok();
+  }
+
+  /** Filter that can be slightly handled by CsvFilterableTable. */
+  @Test void testFilterableWhere3() {
+    final String sql = "select empno, gender, name from EMPS\n"
+        + " where gender <> 'M' and empno > 125";
+    sql("filterable-model", sql)
+        .returns("EMPNO=130; GENDER=F; NAME=Alice")
+        .ok();
+  }
+
+  /** Test case for
+   * <a href="https://issues.apache.org/jira/browse/CALCITE-2272">[CALCITE-2272]
+   * Incorrect result for {@code name like '%E%' and city not like '%W%'}</a>.
+   */
+  @Test void testFilterableWhereWithNot1() {
+    sql("filterable-model",
+        "select name, empno from EMPS "
+            + "where name like '%E%' and city not like '%W%' ")
+        .returns("NAME=Eric; EMPNO=110")
+        .ok();
+  }
+
+  /** Similar to {@link #testFilterableWhereWithNot1()};
+   * But use the same column. */
+  @Test void testFilterableWhereWithNot2() {
+    sql("filterable-model",
+        "select name, empno from EMPS "
+            + "where name like '%i%' and name not like '%W%' ")
+        .returns("NAME=Eric; EMPNO=110",
+            "NAME=Alice; EMPNO=130")
+        .ok();
+  }
+
+  @Test void testJson() {
+    final String sql = "select * from archers\n";
+    final String[] lines = {
+        "id=19990101; dow=Friday; longDate=New Years Day; title=Tractor trouble.; "
+            + "characters=[Alice, Bob, Xavier]; script=Julian Hyde; summary=; "
+            + "lines=[Bob's tractor got stuck in a field., "
+            + "Alice and Xavier hatch a plan to surprise Charlie.]",
+        "id=19990103; dow=Sunday; longDate=Sunday 3rd January; "
+            + "title=Charlie's surprise.; characters=[Alice, Zebedee, Charlie, Xavier]; "
+            + "script=William Shakespeare; summary=; "
+            + "lines=[Charlie is very surprised by Alice and Xavier's surprise plan.]",
+    };
+    sql("bug", sql)
+        .returns(lines)
+        .ok();
+  }
+
+  @Test void testJoinOnString() {
+    final String sql = "select * from emps\n"
+        + "join depts on emps.name = depts.name";
+    sql("smart", sql).ok();
+  }
+
+  @Test void testWackyColumns() {
+    final String sql = "select * from wacky_column_names where false";
+    sql("bug", sql).returns().ok();
+
+    final String sql2 = "select \"joined at\", \"naME\"\n"
+        + "from wacky_column_names\n"
+        + "where \"2gender\" = 'F'";
+    sql("bug", sql2)
+        .returns("joined at=2005-09-07; naME=Wilma",
+            "joined at=2007-01-01; naME=Alice")
+        .ok();
+  }
+
+  /** Test case for
+   * <a href="https://issues.apache.org/jira/browse/CALCITE-1754">[CALCITE-1754]
+   * In Csv adapter, convert DATE and TIME values to int, and TIMESTAMP values
+   * to long</a>. */
+  @Test void testGroupByTimestampAdd() {
+    final String sql = "select count(*) as c,\n"
+        + "  {fn timestampadd(SQL_TSI_DAY, 1, JOINEDAT) } as t\n"
+        + "from EMPS group by {fn timestampadd(SQL_TSI_DAY, 1, JOINEDAT ) } ";
+    sql("model", sql)
+        .returnsUnordered("C=1; T=1996-08-04",
+            "C=1; T=2002-05-04",
+            "C=1; T=2005-09-08",
+            "C=1; T=2007-01-02",
+            "C=1; T=2001-01-02")
+        .ok();
+
+    final String sql2 = "select count(*) as c,\n"
+        + "  {fn timestampadd(SQL_TSI_MONTH, 1, JOINEDAT) } as t\n"
+        + "from EMPS group by {fn timestampadd(SQL_TSI_MONTH, 1, JOINEDAT ) } ";
+    sql("model", sql2)
+        .returnsUnordered("C=1; T=2002-06-03",
+            "C=1; T=2005-10-07",
+            "C=1; T=2007-02-01",
+            "C=1; T=2001-02-01",
+            "C=1; T=1996-09-03")
+        .ok();
+  }
+
+  @Test void testUnionGroupByWithoutGroupKey() {
+    final String sql = "select count(*) as c1 from EMPS group by NAME\n"
+        + "union\n"
+        + "select count(*) as c1 from EMPS group by NAME";
+    sql("model", sql).ok();
+  }
+
+  @Test void testBoolean() {
+    sql("smart", "select empno, slacker from emps where slacker")
+        .returns("EMPNO=100; SLACKER=true").ok();
+  }
+
+  @Test void testReadme() {
+    final String sql = "SELECT d.name, COUNT(*) cnt"
+        + " FROM emps AS e"
+        + " JOIN depts AS d ON e.deptno = d.deptno"
+        + " GROUP BY d.name";
+    sql("smart", sql)
+        .returns("NAME=Sales; CNT=1", "NAME=Marketing; CNT=2").ok();
+  }
+
+  /** Test case for
+   * <a href="https://issues.apache.org/jira/browse/CALCITE-824">[CALCITE-824]
+   * Type inference when converting IN clause to semijoin</a>. */
+  @Test void testInToSemiJoinWithCast() {
+    // Note that the IN list needs at least 20 values to trigger the rewrite
+    // to a semijoin. Try it both ways.
+    final String sql = "SELECT e.name\n"
+        + "FROM emps AS e\n"
+        + "WHERE cast(e.empno as bigint) in ";
+    final int threshold = SqlToRelConverter.DEFAULT_IN_SUB_QUERY_THRESHOLD;
+    sql("smart", sql + range(130, threshold - 5))
+        .returns("NAME=Alice").ok();
+    sql("smart", sql + range(130, threshold))
+        .returns("NAME=Alice").ok();
+    sql("smart", sql + range(130, threshold + 1000))
+        .returns("NAME=Alice").ok();
+  }
+
+  /** Test case for
+   * <a href="https://issues.apache.org/jira/browse/CALCITE-1051">[CALCITE-1051]
+   * Underflow exception due to scaling IN clause literals</a>. */
+  @Test void testInToSemiJoinWithoutCast() {
+    final String sql = "SELECT e.name\n"
+        + "FROM emps AS e\n"
+        + "WHERE e.empno in "
+        + range(130, SqlToRelConverter.DEFAULT_IN_SUB_QUERY_THRESHOLD);
+    sql("smart", sql).returns("NAME=Alice").ok();
+  }
+
+  private String range(int first, int count) {
+    final StringBuilder sb = new StringBuilder();
+    for (int i = 0; i < count; i++) {
+      sb.append(i == 0 ? "(" : ", ").append(first + i);
+    }
+    return sb.append(')').toString();
+  }
+
+  @Test void testDateType() throws SQLException {
+    Properties info = new Properties();
+    info.put("model", FileAdapterTests.jsonPath("bug"));
+
+    try (Connection connection =
+             DriverManager.getConnection("jdbc:calcite:", info)) {
+      ResultSet res = connection.getMetaData().getColumns(null, null,
+          "DATE", "JOINEDAT");
+      res.next();
+      assertEquals(res.getInt("DATA_TYPE"), Types.DATE);
+
+      res = connection.getMetaData().getColumns(null, null,
+          "DATE", "JOINTIME");
+      res.next();
+      assertEquals(res.getInt("DATA_TYPE"), Types.TIME);
+
+      res = connection.getMetaData().getColumns(null, null,
+          "DATE", "JOINTIMES");
+      res.next();
+      assertEquals(res.getInt("DATA_TYPE"), Types.TIMESTAMP);
+
+      Statement statement = connection.createStatement();
+      ResultSet resultSet = statement.executeQuery(
+          "select \"JOINEDAT\", \"JOINTIME\", \"JOINTIMES\" from \"DATE\" where EMPNO = 100");
+      resultSet.next();
+
+      // date
+      assertEquals(Date.class, resultSet.getDate(1).getClass());
+      assertEquals(Date.valueOf("1996-08-03"), resultSet.getDate(1));
+
+      // time
+      assertEquals(Time.class, resultSet.getTime(2).getClass());
+      assertEquals(Time.valueOf("00:01:02"), resultSet.getTime(2));
+
+      // timestamp
+      assertEquals(Timestamp.class, resultSet.getTimestamp(3).getClass());
+      assertEquals(Timestamp.valueOf("1996-08-03 00:01:02"),
+          resultSet.getTimestamp(3));
+    }
+  }
+
+  /** Test case for
+   * <a href="https://issues.apache.org/jira/browse/CALCITE-1072">[CALCITE-1072]
+   * CSV adapter incorrectly parses TIMESTAMP values after noon</a>. */
+  @Test void testDateType2() throws SQLException {
+    Properties info = new Properties();
+    info.put("model", FileAdapterTests.jsonPath("bug"));
+
+    try (Connection connection =
+             DriverManager.getConnection("jdbc:calcite:", info)) {
+      Statement statement = connection.createStatement();
+      final String sql = "select * from \"DATE\"\n"
+          + "where EMPNO >= 140 and EMPNO < 200";
+      ResultSet resultSet = statement.executeQuery(sql);
+      int n = 0;
+      while (resultSet.next()) {
+        ++n;
+        final int empId = resultSet.getInt(1);
+        final String date = resultSet.getString(2);
+        final String time = resultSet.getString(3);
+        final String timestamp = resultSet.getString(4);
+        assertThat(date, is("2015-12-31"));
+        switch (empId) {
+        case 140:
+          assertThat(time, is("07:15:56"));
+          assertThat(timestamp, is("2015-12-31 07:15:56"));
+          break;
+        case 150:
+          assertThat(time, is("13:31:21"));
+          assertThat(timestamp, is("2015-12-31 13:31:21"));
+          break;
+        default:
+          throw new AssertionError();
+        }
+      }
+      assertThat(n, is(2));
+      resultSet.close();
+      statement.close();
+    }
+  }
+
+  /** Test case for
+   * <a href="https://issues.apache.org/jira/browse/CALCITE-1673">[CALCITE-1673]
+   * Query with ORDER BY or GROUP BY on TIMESTAMP column throws
+   * CompileException</a>. */
+  @Test void testTimestampGroupBy() throws SQLException {
+    Properties info = new Properties();
+    info.put("model", FileAdapterTests.jsonPath("bug"));
+    // Use LIMIT to ensure that results are deterministic without ORDER BY
+    final String sql = "select \"EMPNO\", \"JOINTIMES\"\n"
+        + "from (select * from \"DATE\" limit 1)\n"
+        + "group by \"EMPNO\",\"JOINTIMES\"";
+    try (Connection connection =
+             DriverManager.getConnection("jdbc:calcite:", info);
+         Statement statement = connection.createStatement();
+         ResultSet resultSet = statement.executeQuery(sql)) {
+      assertThat(resultSet.next(), is(true));
+      final Timestamp timestamp = resultSet.getTimestamp(2);
+      assertThat(timestamp, isA(Timestamp.class));
+      // Note: This logic is time zone specific, but the same time zone is
+      // used in the CSV adapter and this test, so they should cancel out.
+      assertThat(timestamp, is(Timestamp.valueOf("1996-08-03 00:01:02.0")));
+    }
+  }
+
+  /** As {@link #testTimestampGroupBy()} but with ORDER BY. */
+  @Test void testTimestampOrderBy() throws SQLException {
+    Properties info = new Properties();
+    info.put("model", FileAdapterTests.jsonPath("bug"));
+    final String sql = "select \"EMPNO\",\"JOINTIMES\" from \"DATE\"\n"
+        + "order by \"JOINTIMES\"";
+    try (Connection connection =
+             DriverManager.getConnection("jdbc:calcite:", info);
+         Statement statement = connection.createStatement();
+         ResultSet resultSet = statement.executeQuery(sql)) {
+      assertThat(resultSet.next(), is(true));
+      final Timestamp timestamp = resultSet.getTimestamp(2);
+      assertThat(timestamp, is(Timestamp.valueOf("1996-08-03 00:01:02")));
+    }
+  }
+
+  /** As {@link #testTimestampGroupBy()} but with ORDER BY as well as GROUP
+   * BY. */
+  @Test void testTimestampGroupByAndOrderBy() throws SQLException {
+    Properties info = new Properties();
+    info.put("model", FileAdapterTests.jsonPath("bug"));
+    final String sql = "select \"EMPNO\", \"JOINTIMES\" from \"DATE\"\n"
+        + "group by \"EMPNO\",\"JOINTIMES\" order by \"JOINTIMES\"";
+    try (Connection connection =
+             DriverManager.getConnection("jdbc:calcite:", info);
+         Statement statement = connection.createStatement();
+         ResultSet resultSet = statement.executeQuery(sql)) {
+      assertThat(resultSet.next(), is(true));
+      final Timestamp timestamp = resultSet.getTimestamp(2);
+      assertThat(timestamp, is(Timestamp.valueOf("1996-08-03 00:01:02")));
+    }
+  }
+
+  /** Test case for
+   * <a href="https://issues.apache.org/jira/browse/CALCITE-1031">[CALCITE-1031]
+   * In prepared statement, CsvScannableTable.scan is called twice</a>. To see
+   * the bug, place a breakpoint in CsvScannableTable.scan, and note that it is
+   * called twice. It should only be called once. */
+  @Test void testPrepared() throws SQLException {
+    final Properties properties = new Properties();
+    properties.setProperty("caseSensitive", "true");
+    try (Connection connection =
+             DriverManager.getConnection("jdbc:calcite:", properties)) {
+      final CalciteConnection calciteConnection = connection.unwrap(
+          CalciteConnection.class);
+
+      final Schema schema =
+          FileSchemaFactory.INSTANCE
+              .create(calciteConnection.getRootSchema(), null,
+                  ImmutableMap.of("directory",
+                      FileAdapterTests.resourcePath("sales-csv"), "flavor", "scannable"));
+      calciteConnection.getRootSchema().add("TEST", schema);
+      final String sql = "select * from \"TEST\".\"DEPTS\" where \"NAME\" = ?";
+      final PreparedStatement statement2 =
+          calciteConnection.prepareStatement(sql);
+
+      statement2.setString(1, "Sales");
+      final ResultSet resultSet1 = statement2.executeQuery();
+      Consumer<ResultSet> expect = FileAdapterTests.expect("DEPTNO=10; NAME=Sales");
+      expect.accept(resultSet1);
+    }
+  }
+
+  /** Test case for
+   * <a href="https://issues.apache.org/jira/browse/CALCITE-1054">[CALCITE-1054]
+   * NPE caused by wrong code generation for Timestamp fields</a>. */
+  @Test void testFilterOnNullableTimestamp() throws Exception {
+    Properties info = new Properties();
+    info.put("model", FileAdapterTests.jsonPath("bug"));
+
+    try (Connection connection =
+             DriverManager.getConnection("jdbc:calcite:", info)) {
+      final Statement statement = connection.createStatement();
+
+      // date
+      final String sql1 = "select JOINEDAT from \"DATE\"\n"
+          + "where JOINEDAT < {d '2000-01-01'}\n"
+          + "or JOINEDAT >= {d '2017-01-01'}";
+      final ResultSet joinedAt = statement.executeQuery(sql1);
+      assertThat(joinedAt.next(), is(true));
+      assertThat(joinedAt.getDate(1), is(Date.valueOf("1996-08-03")));
+
+      // time
+      final String sql2 = "select JOINTIME from \"DATE\"\n"
+          + "where JOINTIME >= {t '07:00:00'}\n"
+          + "and JOINTIME < {t '08:00:00'}";
+      final ResultSet joinTime = statement.executeQuery(sql2);
+      assertThat(joinTime.next(), is(true));
+      assertThat(joinTime.getTime(1), is(Time.valueOf("07:15:56")));
+
+      // timestamp
+      final String sql3 = "select JOINTIMES,\n"
+          + "  {fn timestampadd(SQL_TSI_DAY, 1, JOINTIMES)}\n"
+          + "from \"DATE\"\n"
+          + "where (JOINTIMES >= {ts '2003-01-01 00:00:00'}\n"
+          + "and JOINTIMES < {ts '2006-01-01 00:00:00'})\n"
+          + "or (JOINTIMES >= {ts '2003-01-01 00:00:00'}\n"
+          + "and JOINTIMES < {ts '2007-01-01 00:00:00'})";
+      final ResultSet joinTimes = statement.executeQuery(sql3);
+      assertThat(joinTimes.next(), is(true));
+      assertThat(joinTimes.getTimestamp(1),
+          is(Timestamp.valueOf("2005-09-07 00:00:00")));
+      assertThat(joinTimes.getTimestamp(2),
+          is(Timestamp.valueOf("2005-09-08 00:00:00")));
+
+      final String sql4 = "select JOINTIMES, extract(year from JOINTIMES)\n"
+          + "from \"DATE\"";
+      final ResultSet joinTimes2 = statement.executeQuery(sql4);
+      assertThat(joinTimes2.next(), is(true));
+      assertThat(joinTimes2.getTimestamp(1),
+          is(Timestamp.valueOf("1996-08-03 00:01:02")));
+    }
+  }
+
+  /** Test case for
+   * <a href="https://issues.apache.org/jira/browse/CALCITE-1118">[CALCITE-1118]
+   * NullPointerException in EXTRACT with WHERE ... IN clause if field has null
+   * value</a>. */
+  @Test void testFilterOnNullableTimestamp2() throws Exception {
+    Properties info = new Properties();
+    info.put("model", FileAdapterTests.jsonPath("bug"));
+
+    try (Connection connection =
+             DriverManager.getConnection("jdbc:calcite:", info)) {
+      final Statement statement = connection.createStatement();
+      final String sql1 = "select extract(year from JOINTIMES)\n"
+          + "from \"DATE\"\n"
+          + "where extract(year from JOINTIMES) in (2006, 2007)";
+      final ResultSet joinTimes = statement.executeQuery(sql1);
+      assertThat(joinTimes.next(), is(true));
+      assertThat(joinTimes.getInt(1), is(2007));
+
+      final String sql2 = "select extract(year from JOINTIMES),\n"
+          + "  count(0) from \"DATE\"\n"
+          + "where extract(year from JOINTIMES) between 2007 and 2016\n"
+          + "group by extract(year from JOINTIMES)";
+      final ResultSet joinTimes2 = statement.executeQuery(sql2);
+      assertThat(joinTimes2.next(), is(true));
+      assertThat(joinTimes2.getInt(1), is(2007));
+      assertThat(joinTimes2.getLong(2), is(1L));
+      assertThat(joinTimes2.next(), is(true));
+      assertThat(joinTimes2.getInt(1), is(2015));
+      assertThat(joinTimes2.getLong(2), is(2L));
+    }
+  }
+
+  /** Test case for
+   * <a href="https://issues.apache.org/jira/browse/CALCITE-1427">[CALCITE-1427]
+   * Code generation incorrect (does not compile) for DATE, TIME and TIMESTAMP
+   * fields</a>. */
+  @Test void testNonNullFilterOnDateType() throws SQLException {
+    Properties info = new Properties();
+    info.put("model", FileAdapterTests.jsonPath("bug"));
+
+    try (Connection connection =
+             DriverManager.getConnection("jdbc:calcite:", info)) {
+      final Statement statement = connection.createStatement();
+
+      // date
+      final String sql1 = "select JOINEDAT from \"DATE\"\n"
+          + "where JOINEDAT is not null";
+      final ResultSet joinedAt = statement.executeQuery(sql1);
+      assertThat(joinedAt.next(), is(true));
+      assertThat(joinedAt.getDate(1).getClass(), equalTo(Date.class));
+      assertThat(joinedAt.getDate(1), is(Date.valueOf("1996-08-03")));
+
+      // time
+      final String sql2 = "select JOINTIME from \"DATE\"\n"
+          + "where JOINTIME is not null";
+      final ResultSet joinTime = statement.executeQuery(sql2);
+      assertThat(joinTime.next(), is(true));
+      assertThat(joinTime.getTime(1).getClass(), equalTo(Time.class));
+      assertThat(joinTime.getTime(1), is(Time.valueOf("00:01:02")));
+
+      // timestamp
+      final String sql3 = "select JOINTIMES from \"DATE\"\n"
+          + "where JOINTIMES is not null";
+      final ResultSet joinTimes = statement.executeQuery(sql3);
+      assertThat(joinTimes.next(), is(true));
+      assertThat(joinTimes.getTimestamp(1).getClass(),
+          equalTo(Timestamp.class));
+      assertThat(joinTimes.getTimestamp(1),
+          is(Timestamp.valueOf("1996-08-03 00:01:02")));
+    }
+  }
+
+  /** Test case for
+   * <a href="https://issues.apache.org/jira/browse/CALCITE-1427">[CALCITE-1427]
+   * Code generation incorrect (does not compile) for DATE, TIME and TIMESTAMP
+   * fields</a>. */
+  @Test void testGreaterThanFilterOnDateType() throws SQLException {
+    Properties info = new Properties();
+    info.put("model", FileAdapterTests.jsonPath("bug"));
+
+    try (Connection connection =
+             DriverManager.getConnection("jdbc:calcite:", info)) {
+      final Statement statement = connection.createStatement();
+
+      // date
+      final String sql1 = "select JOINEDAT from \"DATE\"\n"
+          + "where JOINEDAT > {d '1990-01-01'}";
+      final ResultSet joinedAt = statement.executeQuery(sql1);
+      assertThat(joinedAt.next(), is(true));
+      assertThat(joinedAt.getDate(1).getClass(), equalTo(Date.class));
+      assertThat(joinedAt.getDate(1), is(Date.valueOf("1996-08-03")));
+
+      // time
+      final String sql2 = "select JOINTIME from \"DATE\"\n"
+          + "where JOINTIME > {t '00:00:00'}";
+      final ResultSet joinTime = statement.executeQuery(sql2);
+      assertThat(joinTime.next(), is(true));
+      assertThat(joinTime.getTime(1).getClass(), equalTo(Time.class));
+      assertThat(joinTime.getTime(1), is(Time.valueOf("00:01:02")));
+
+      // timestamp
+      final String sql3 = "select JOINTIMES from \"DATE\"\n"
+          + "where JOINTIMES > {ts '1990-01-01 00:00:00'}";
+      final ResultSet joinTimes = statement.executeQuery(sql3);
+      assertThat(joinTimes.next(), is(true));
+      assertThat(joinTimes.getTimestamp(1).getClass(),
+          equalTo(Timestamp.class));
+      assertThat(joinTimes.getTimestamp(1),
+          is(Timestamp.valueOf("1996-08-03 00:01:02")));
+    }
+  }
+}
diff --git a/file/src/test/java/org/apache/calcite/adapter/file/FileAdapterTests.java b/file/src/test/java/org/apache/calcite/adapter/file/FileAdapterTests.java
new file mode 100644
index 0000000..1162516
--- /dev/null
+++ b/file/src/test/java/org/apache/calcite/adapter/file/FileAdapterTests.java
@@ -0,0 +1,222 @@
+/*
+ * 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.adapter.file;
+
+import org.apache.calcite.util.Sources;
+import org.apache.calcite.util.TestUtil;
+import org.apache.calcite.util.Util;
+
+import com.google.common.collect.Ordering;
+
+import java.io.PrintStream;
+import java.sql.Connection;
+import java.sql.DriverManager;
+import java.sql.ResultSet;
+import java.sql.ResultSetMetaData;
+import java.sql.SQLException;
+import java.sql.Statement;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.List;
+import java.util.Properties;
+import java.util.function.Consumer;
+
+import static org.junit.jupiter.api.Assertions.assertEquals;
+
+/** Helpers for test suite of the File adapter. */
+abstract class FileAdapterTests {
+  private FileAdapterTests() {
+  }
+
+  static Fluent sql(String model, String sql) {
+    return new Fluent(model, sql, FileAdapterTests::output);
+  }
+
+  /** Returns a function that checks the contents of a result set against an
+   * expected string. */
+  static Consumer<ResultSet> expect(final String... expected) {
+    return resultSet -> {
+      try {
+        final List<String> lines = new ArrayList<>();
+        collect(lines, resultSet);
+        assertEquals(Arrays.asList(expected), lines);
+      } catch (SQLException e) {
+        throw TestUtil.rethrow(e);
+      }
+    };
+  }
+
+  /** Returns a function that checks the contents of a result set against an
+   * expected string. */
+  private static Consumer<ResultSet> expectUnordered(String... expected) {
+    final List<String> expectedLines =
+        Ordering.natural().immutableSortedCopy(Arrays.asList(expected));
+    return resultSet -> {
+      try {
+        final List<String> lines = new ArrayList<>();
+        collect(lines, resultSet);
+        Collections.sort(lines);
+        assertEquals(expectedLines, lines);
+      } catch (SQLException e) {
+        throw TestUtil.rethrow(e);
+      }
+    };
+  }
+
+  private static void collect(List<String> result, ResultSet resultSet)
+      throws SQLException {
+    final StringBuilder buf = new StringBuilder();
+    while (resultSet.next()) {
+      buf.setLength(0);
+      int n = resultSet.getMetaData().getColumnCount();
+      String sep = "";
+      for (int i = 1; i <= n; i++) {
+        buf.append(sep)
+            .append(resultSet.getMetaData().getColumnLabel(i))
+            .append("=")
+            .append(resultSet.getString(i));
+        sep = "; ";
+      }
+      result.add(Util.toLinux(buf.toString()));
+    }
+  }
+
+  static String toString(ResultSet resultSet) throws SQLException {
+    StringBuilder buf = new StringBuilder();
+    while (resultSet.next()) {
+      int n = resultSet.getMetaData().getColumnCount();
+      String sep = "";
+      for (int i = 1; i <= n; i++) {
+        buf.append(sep)
+            .append(resultSet.getMetaData().getColumnLabel(i))
+            .append("=")
+            .append(resultSet.getObject(i));
+        sep = "; ";
+      }
+      buf.append("\n");
+    }
+    return buf.toString();
+  }
+
+  static void checkSql(String sql, String model, Consumer<ResultSet> fn)
+      throws SQLException {
+    Connection connection = null;
+    Statement statement = null;
+    try {
+      Properties info = new Properties();
+      info.put("model", jsonPath(model));
+      connection = DriverManager.getConnection("jdbc:calcite:", info);
+      statement = connection.createStatement();
+      final ResultSet resultSet =
+          statement.executeQuery(
+              sql);
+      fn.accept(resultSet);
+    } finally {
+      close(connection, statement);
+    }
+  }
+
+  static String jsonPath(String model) {
+    return resourcePath(model + ".json");
+  }
+
+  static String resourcePath(String path) {
+    return Sources.of(FileAdapterTest.class.getResource("/" + path)).file().getAbsolutePath();
+  }
+
+  private static void output(ResultSet resultSet, PrintStream out)
+      throws SQLException {
+    final ResultSetMetaData metaData = resultSet.getMetaData();
+    final int columnCount = metaData.getColumnCount();
+    while (resultSet.next()) {
+      for (int i = 1;; i++) {
+        out.print(resultSet.getString(i));
+        if (i < columnCount) {
+          out.print(", ");
+        } else {
+          out.println();
+          break;
+        }
+      }
+    }
+  }
+
+  private static void output(ResultSet resultSet) {
+    try {
+      output(resultSet, System.out);
+    } catch (SQLException e) {
+      throw TestUtil.rethrow(e);
+    }
+  }
+
+  static void close(Connection connection, Statement statement) {
+    if (statement != null) {
+      try {
+        statement.close();
+      } catch (SQLException e) {
+        // ignore
+      }
+    }
+    if (connection != null) {
+      try {
+        connection.close();
+      } catch (SQLException e) {
+        // ignore
+      }
+    }
+  }
+
+  /** Fluent API to perform test actions. */
+  static class Fluent {
+    private final String model;
+    private final String sql;
+    private final Consumer<ResultSet> expect;
+
+    Fluent(String model, String sql, Consumer<ResultSet> expect) {
+      this.model = model;
+      this.sql = sql;
+      this.expect = expect;
+    }
+
+    /** Runs the test. */
+    Fluent ok() {
+      try {
+        checkSql(sql, model, expect);
+        return this;
+      } catch (SQLException e) {
+        throw TestUtil.rethrow(e);
+      }
+    }
+
+    /** Assigns a function to call to test whether output is correct. */
+    Fluent checking(Consumer<ResultSet> expect) {
+      return new Fluent(model, sql, expect);
+    }
+
+    /** Sets the rows that are expected to be returned from the SQL query. */
+    Fluent returns(String... expectedLines) {
+      return checking(expect(expectedLines));
+    }
+
+    /** Sets the rows that are expected to be returned from the SQL query,
+     * in no particular order. */
+    Fluent returnsUnordered(String... expectedLines) {
+      return checking(expectUnordered(expectedLines));
+    }
+  }
+}
diff --git a/file/src/test/java/org/apache/calcite/adapter/file/SqlTest.java b/file/src/test/java/org/apache/calcite/adapter/file/SqlTest.java
deleted file mode 100644
index 444f225..0000000
--- a/file/src/test/java/org/apache/calcite/adapter/file/SqlTest.java
+++ /dev/null
@@ -1,419 +0,0 @@
-/*
- * 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.adapter.file;
-
-import org.apache.calcite.util.Sources;
-import org.apache.calcite.util.TestUtil;
-import org.apache.calcite.util.Util;
-
-import com.google.common.collect.Ordering;
-
-import org.junit.jupiter.api.Disabled;
-import org.junit.jupiter.api.Test;
-import org.junit.jupiter.api.extension.ExtendWith;
-
-import java.sql.Connection;
-import java.sql.DriverManager;
-import java.sql.ResultSet;
-import java.sql.SQLException;
-import java.sql.Statement;
-import java.sql.Types;
-import java.util.ArrayList;
-import java.util.Arrays;
-import java.util.Collections;
-import java.util.List;
-import java.util.Properties;
-import java.util.function.Function;
-
-import static org.hamcrest.CoreMatchers.is;
-import static org.hamcrest.MatcherAssert.assertThat;
-import static org.junit.jupiter.api.Assertions.assertEquals;
-
-/**
- * System test of the Calcite file adapter, which can also read and parse
- * HTML tables over HTTP.
- */
-@ExtendWith(RequiresNetworkExtension.class)
-class SqlTest {
-  // helper functions
-
-  private Fluent sql(String model, String sql) {
-    return new Fluent(model, sql, input -> {
-      throw new AssertionError();
-    });
-  }
-
-  private static Function<ResultSet, Void> expect(String... expectedLines) {
-    final StringBuilder b = new StringBuilder();
-    for (String s : expectedLines) {
-      b.append(s).append('\n');
-    }
-    final String expected = b.toString();
-    return resultSet -> {
-      try {
-        String actual = toString(resultSet);
-        if (!expected.equals(actual)) {
-          System.out.println("Assertion failure:");
-          System.out.println("\tExpected: '" + expected + "'");
-          System.out.println("\tActual: '" + actual + "'");
-        }
-        assertEquals(expected, actual);
-      } catch (SQLException e) {
-        throw TestUtil.rethrow(e);
-      }
-      return null;
-    };
-  }
-
-  /** Returns a function that checks the contents of a result set against an
-   * expected string. */
-  private static Function<ResultSet, Void> expectUnordered(String... expected) {
-    final List<String> expectedLines =
-        Ordering.natural().immutableSortedCopy(Arrays.asList(expected));
-    return resultSet -> {
-      try {
-        final List<String> lines = new ArrayList<>();
-        SqlTest.collect(lines, resultSet);
-        Collections.sort(lines);
-        assertEquals(expectedLines, lines);
-      } catch (SQLException e) {
-        throw TestUtil.rethrow(e);
-      }
-      return null;
-    };
-  }
-
-  private static void collect(List<String> result, ResultSet resultSet)
-      throws SQLException {
-    final StringBuilder buf = new StringBuilder();
-    while (resultSet.next()) {
-      buf.setLength(0);
-      int n = resultSet.getMetaData().getColumnCount();
-      String sep = "";
-      for (int i = 1; i <= n; i++) {
-        buf.append(sep)
-            .append(resultSet.getMetaData().getColumnLabel(i))
-            .append("=")
-            .append(resultSet.getString(i));
-        sep = "; ";
-      }
-      result.add(Util.toLinux(buf.toString()));
-    }
-  }
-
-  private void checkSql(String sql, String model, Function<ResultSet, Void> fn)
-      throws SQLException {
-    Connection connection = null;
-    Statement statement = null;
-    try {
-      Properties info = new Properties();
-      info.put("model",
-          Sources.of(SqlTest.class.getResource("/" + model + ".json")).path());
-      connection = DriverManager.getConnection("jdbc:calcite:", info);
-      statement = connection.createStatement();
-      final ResultSet resultSet = statement.executeQuery(sql);
-      fn.apply(resultSet);
-    } finally {
-      close(connection, statement);
-    }
-  }
-
-  private static String toString(ResultSet resultSet) throws SQLException {
-    StringBuilder buf = new StringBuilder();
-    while (resultSet.next()) {
-      int n = resultSet.getMetaData().getColumnCount();
-      String sep = "";
-      for (int i = 1; i <= n; i++) {
-        buf.append(sep)
-            .append(resultSet.getMetaData().getColumnLabel(i))
-            .append("=")
-            .append(resultSet.getObject(i));
-        sep = "; ";
-      }
-      buf.append("\n");
-    }
-    return buf.toString();
-  }
-
-  private void close(Connection connection, Statement statement) {
-    if (statement != null) {
-      try {
-        statement.close();
-      } catch (SQLException e) {
-        // ignore
-      }
-    }
-    if (connection != null) {
-      try {
-        connection.close();
-      } catch (SQLException e) {
-        // ignore
-      }
-    }
-  }
-
-  // tests
-
-  /** Reads from a local file and checks the result. */
-  @Test void testFileSelect() throws SQLException {
-    final String sql = "select H1 from T1 where H0 = 'R1C0'";
-    sql("testModel", sql).returns("H1=R1C1").ok();
-  }
-
-  /** Reads from a local file without table headers &lt;TH&gt; and checks the
-   * result. */
-  @Test @RequiresNetwork public void testNoThSelect() throws SQLException {
-    final String sql = "select \"col1\" from T1_NO_TH where \"col0\" like 'R0%'";
-    sql("testModel", sql).returns("col1=R0C1").ok();
-  }
-
-  /** Reads from a local file - finds larger table even without &lt;TH&gt;
-   * elements. */
-  @Test void testFindBiggerNoTh() throws SQLException {
-    final String sql = "select \"col4\" from TABLEX2 where \"col0\" like 'R1%'";
-    sql("testModel", sql).returns("col4=R1C4").ok();
-  }
-
-  /** Reads from a URL and checks the result. */
-  @Disabled("[CALCITE-1789] Wikipedia format change breaks file adapter test")
-  @Test @RequiresNetwork public void testUrlSelect() throws SQLException {
-    final String sql = "select \"State\", \"Statehood\" from \"States_as_of\"\n"
-        + "where \"State\" = 'California'";
-    sql("wiki", sql).returns("State=California; Statehood=1850-09-09").ok();
-  }
-
-  /** Reads the EMPS table. */
-  @Test void testSalesEmps() throws SQLException {
-    final String sql = "select * from sales.emps";
-    sql("sales", sql)
-        .returns("EMPNO=100; NAME=Fred; DEPTNO=30",
-            "EMPNO=110; NAME=Eric; DEPTNO=20",
-            "EMPNO=110; NAME=John; DEPTNO=40",
-            "EMPNO=120; NAME=Wilma; DEPTNO=20",
-            "EMPNO=130; NAME=Alice; DEPTNO=40")
-        .ok();
-  }
-
-  /** Reads the DEPTS table. */
-  @Test void testSalesDepts() throws SQLException {
-    final String sql = "select * from sales.depts";
-    sql("sales", sql)
-        .returns("DEPTNO=10; NAME=Sales",
-            "DEPTNO=20; NAME=Marketing",
-            "DEPTNO=30; NAME=Accounts")
-        .ok();
-  }
-
-  /** Reads the DEPTS table from the CSV schema. */
-  @Test void testCsvSalesDepts() throws SQLException {
-    final String sql = "select * from sales.depts";
-    sql("sales-csv", sql)
-        .returns("DEPTNO=10; NAME=Sales",
-            "DEPTNO=20; NAME=Marketing",
-            "DEPTNO=30; NAME=Accounts")
-        .ok();
-  }
-
-  /** Reads the EMPS table from the CSV schema. */
-  @Test void testCsvSalesEmps() throws SQLException {
-    final String sql = "select * from sales.emps";
-    final String[] lines = {
-        "EMPNO=100; NAME=Fred; DEPTNO=10; GENDER=; CITY=; EMPID=30; AGE=25; SLACKER=true; MANAGER=false; JOINEDAT=1996-08-03",
-        "EMPNO=110; NAME=Eric; DEPTNO=20; GENDER=M; CITY=San Francisco; EMPID=3; AGE=80; SLACKER=null; MANAGER=false; JOINEDAT=2001-01-01",
-        "EMPNO=110; NAME=John; DEPTNO=40; GENDER=M; CITY=Vancouver; EMPID=2; AGE=null; SLACKER=false; MANAGER=true; JOINEDAT=2002-05-03",
-        "EMPNO=120; NAME=Wilma; DEPTNO=20; GENDER=F; CITY=; EMPID=1; AGE=5; SLACKER=null; MANAGER=true; JOINEDAT=2005-09-07",
-        "EMPNO=130; NAME=Alice; DEPTNO=40; GENDER=F; CITY=Vancouver; EMPID=2; AGE=null; SLACKER=false; MANAGER=true; JOINEDAT=2007-01-01",
-    };
-    sql("sales-csv", sql).returns(lines).ok();
-  }
-
-  /** Reads the HEADER_ONLY table from the CSV schema. The CSV file has one
-   * line - the column headers - but no rows of data. */
-  @Test void testCsvSalesHeaderOnly() throws SQLException {
-    final String sql = "select * from sales.header_only";
-    sql("sales-csv", sql).returns().ok();
-  }
-
-  /** Reads the EMPTY table from the CSV schema. The CSV file has no lines,
-   * therefore the table has a system-generated column called
-   * "EmptyFileHasNoColumns". */
-  @Test void testCsvSalesEmpty() throws SQLException {
-    final String sql = "select * from sales.\"EMPTY\"";
-    checkSql(sql, "sales-csv", resultSet -> {
-      try {
-        assertThat(resultSet.getMetaData().getColumnCount(), is(1));
-        assertThat(resultSet.getMetaData().getColumnName(1),
-            is("EmptyFileHasNoColumns"));
-        assertThat(resultSet.getMetaData().getColumnType(1),
-            is(Types.BOOLEAN));
-        String actual = toString(resultSet);
-        assertThat(actual, is(""));
-      } catch (SQLException e) {
-        throw TestUtil.rethrow(e);
-      }
-      return null;
-    });
-  }
-
-  /** Test case for
-   * <a href="https://issues.apache.org/jira/browse/CALCITE-1754">[CALCITE-1754]
-   * In Csv adapter, convert DATE and TIME values to int, and TIMESTAMP values
-   * to long</a>. */
-  @Test void testCsvGroupByTimestampAdd() throws SQLException {
-    final String sql = "select count(*) as c,\n"
-        + "  {fn timestampadd(SQL_TSI_DAY, 1, JOINEDAT) } as t\n"
-        + "from EMPS group by {fn timestampadd(SQL_TSI_DAY, 1, JOINEDAT ) } ";
-    sql("sales-csv", sql)
-        .returnsUnordered("C=1; T=1996-08-04",
-            "C=1; T=2002-05-04",
-            "C=1; T=2005-09-08",
-            "C=1; T=2007-01-02",
-            "C=1; T=2001-01-02")
-        .ok();
-    final String sql2 = "select count(*) as c,\n"
-        + "  {fn timestampadd(SQL_TSI_MONTH, 1, JOINEDAT) } as t\n"
-        + "from EMPS group by {fn timestampadd(SQL_TSI_MONTH, 1, JOINEDAT ) } ";
-    sql("sales-csv", sql2)
-        .returnsUnordered("C=1; T=2002-06-03",
-            "C=1; T=2005-10-07",
-            "C=1; T=2007-02-01",
-            "C=1; T=2001-02-01",
-            "C=1; T=1996-09-03").ok();
-    final String sql3 = "select\n"
-        + " distinct {fn timestampadd(SQL_TSI_MONTH, 1, JOINEDAT) } as t\n"
-        + "from EMPS";
-    sql("sales-csv", sql3)
-        .returnsUnordered("T=2002-06-03",
-            "T=2005-10-07",
-            "T=2007-02-01",
-            "T=2001-02-01",
-            "T=1996-09-03").ok();
-  }
-
-  /** Fluent API to perform test actions. */
-  private class Fluent {
-    private final String model;
-    private final String sql;
-    private final Function<ResultSet, Void> expect;
-
-    Fluent(String model, String sql, Function<ResultSet, Void> expect) {
-      this.model = model;
-      this.sql = sql;
-      this.expect = expect;
-    }
-
-    /** Runs the test. */
-    Fluent ok() {
-      try {
-        checkSql(sql, model, expect);
-        return this;
-      } catch (SQLException e) {
-        throw TestUtil.rethrow(e);
-      }
-    }
-
-    /** Assigns a function to call to test whether output is correct. */
-    Fluent checking(Function<ResultSet, Void> expect) {
-      return new Fluent(model, sql, expect);
-    }
-
-    /** Sets the rows that are expected to be returned from the SQL query. */
-    Fluent returns(String... expectedLines) {
-      return checking(expect(expectedLines));
-    }
-
-    /** Sets the rows that are expected to be returned from the SQL query,
-     * in no particular order. */
-    Fluent returnsUnordered(String... expectedLines) {
-      return checking(expectUnordered(expectedLines));
-    }
-  }
-
-  /** Reads the DEPTS table from the JSON schema. */
-  @Test void testJsonSalesDepts() throws SQLException {
-    final String sql = "select * from sales.depts";
-    sql("sales-json", sql)
-        .returns("DEPTNO=10; NAME=Sales",
-            "DEPTNO=20; NAME=Marketing",
-            "DEPTNO=30; NAME=Accounts")
-        .ok();
-  }
-
-  /** Reads the EMPS table from the JSON schema. */
-  @Test void testJsonSalesEmps() throws SQLException {
-    final String sql = "select * from sales.emps";
-    final String[] lines = {
-        "EMPNO=100; NAME=Fred; DEPTNO=10; GENDER=; CITY=; EMPID=30; AGE=25; SLACKER=true; MANAGER=false; JOINEDAT=1996-08-03",
-        "EMPNO=110; NAME=Eric; DEPTNO=20; GENDER=M; CITY=San Francisco; EMPID=3; AGE=80; SLACKER=null; MANAGER=false; JOINEDAT=2001-01-01",
-        "EMPNO=110; NAME=John; DEPTNO=40; GENDER=M; CITY=Vancouver; EMPID=2; AGE=null; SLACKER=false; MANAGER=true; JOINEDAT=2002-05-03",
-        "EMPNO=120; NAME=Wilma; DEPTNO=20; GENDER=F; CITY=; EMPID=1; AGE=5; SLACKER=null; MANAGER=true; JOINEDAT=2005-09-07",
-        "EMPNO=130; NAME=Alice; DEPTNO=40; GENDER=F; CITY=Vancouver; EMPID=2; AGE=null; SLACKER=false; MANAGER=true; JOINEDAT=2007-01-01",
-    };
-    sql("sales-json", sql).returns(lines).ok();
-  }
-
-  /** Reads the EMPTY table from the JSON schema. The JSON file has no lines,
-   * therefore the table has a system-generated column called
-   * "EmptyFileHasNoColumns". */
-  @Test void testJsonSalesEmpty() throws SQLException {
-    final String sql = "select * from sales.\"EMPTY\"";
-    checkSql(sql, "sales-json", resultSet -> {
-      try {
-        assertThat(resultSet.getMetaData().getColumnCount(), is(1));
-        assertThat(resultSet.getMetaData().getColumnName(1),
-            is("EmptyFileHasNoColumns"));
-        assertThat(resultSet.getMetaData().getColumnType(1),
-            is(Types.BOOLEAN));
-        String actual = toString(resultSet);
-        assertThat(actual, is(""));
-      } catch (SQLException e) {
-        throw TestUtil.rethrow(e);
-      }
-      return null;
-    });
-  }
-
-  /** Test returns the result of two json file joins. */
-  @Test void testJsonJoinOnString() {
-    final String sql = "select emps.EMPNO, emps.NAME, depts.deptno from emps\n"
-        + "join depts on emps.deptno = depts.deptno";
-    final String[] lines = {
-        "EMPNO=100; NAME=Fred; DEPTNO=10",
-        "EMPNO=110; NAME=Eric; DEPTNO=20",
-        "EMPNO=120; NAME=Wilma; DEPTNO=20",
-    };
-    sql("sales-json", sql).returns(lines).ok();
-  }
-
-  /** The folder contains both JSON files and CSV files joins. */
-  @Test void testJsonWithCsvJoin() {
-    final String sql = "select emps.empno,\n"
-        + " NAME,\n"
-        + " \"DATE\".JOINEDAT\n"
-        + " from \"DATE\"\n"
-        + "join emps on emps.empno = \"DATE\".EMPNO\n"
-        + "order by empno, name, joinedat limit 3";
-    final String[] lines = {
-        "EMPNO=100; NAME=Fred; JOINEDAT=1996-08-03",
-        "EMPNO=110; NAME=Eric; JOINEDAT=2001-01-01",
-        "EMPNO=110; NAME=Eric; JOINEDAT=2002-05-03",
-    };
-    sql("sales-json", sql)
-        .returns(lines)
-        .ok();
-  }
-}
diff --git a/file/build.gradle.kts b/file/src/test/resources/bug.json
similarity index 69%
copy from file/build.gradle.kts
copy to file/src/test/resources/bug.json
index 1610a03..46d0cb4 100644
--- a/file/build.gradle.kts
+++ b/file/src/test/resources/bug.json
@@ -14,15 +14,17 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-dependencies {
-    api(project(":core"))
-    api(project(":example:csv"))
-    api(project(":linq4j"))
-
-    implementation("com.google.guava:guava")
-    implementation("com.joestelmach:natty")
-    implementation("org.apache.calcite.avatica:avatica-core")
-    implementation("org.jsoup:jsoup")
-
-    testImplementation(project(":core", "testClasses"))
+{
+  "version": "1.0",
+  "defaultSchema": "BUG",
+  "schemas": [
+    {
+      "name": "BUG",
+      "type": "custom",
+      "factory": "org.apache.calcite.adapter.file.FileSchemaFactory",
+      "operand": {
+        "directory": "bug"
+      }
+    }
+  ]
 }
diff --git a/file/src/test/resources/bug.yaml b/file/src/test/resources/bug.yaml
new file mode 100644
index 0000000..fd50d5b
--- /dev/null
+++ b/file/src/test/resources/bug.yaml
@@ -0,0 +1,24 @@
+#
+# Licensed to the Apache Software Foundation (ASF) under one or more
+# contributor license agreements.  See the NOTICE file distributed with
+# this work for additional information regarding copyright ownership.
+# The ASF licenses this file to you under the Apache License, Version 2.0
+# (the "License"); you may not use this file except in compliance with
+# the License.  You may obtain a copy of the License at
+#
+# http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing, software
+# distributed under the License is distributed on an "AS IS" BASIS,
+# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+# See the License for the specific language governing permissions and
+# limitations under the License.
+#
+version: 1.0
+defaultSchema: BUG
+schemas:
+- name: BUG
+  type: custom
+  factory: org.apache.calcite.adapter.csv.CsvSchemaFactory
+  operand:
+    directory: bug
diff --git a/file/build.gradle.kts b/file/src/test/resources/bug/ARCHERS.json
similarity index 56%
copy from file/build.gradle.kts
copy to file/src/test/resources/bug/ARCHERS.json
index 1610a03..0e36759 100644
--- a/file/build.gradle.kts
+++ b/file/src/test/resources/bug/ARCHERS.json
@@ -14,15 +14,30 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-dependencies {
-    api(project(":core"))
-    api(project(":example:csv"))
-    api(project(":linq4j"))
-
-    implementation("com.google.guava:guava")
-    implementation("com.joestelmach:natty")
-    implementation("org.apache.calcite.avatica:avatica-core")
-    implementation("org.jsoup:jsoup")
-
-    testImplementation(project(":core", "testClasses"))
+[
+{
+  "id": "19990101",
+  "dow": "Friday",
+  "longDate": "New Years Day",
+  "title": "Tractor trouble.",
+  "characters": [ "Alice", "Bob", "Xavier" ],
+  "script": "Julian Hyde",
+  "summary": "",
+  "lines": [
+    "Bob's tractor got stuck in a field.",
+    "Alice and Xavier hatch a plan to surprise Charlie."
+  ]
+},
+{
+  "id": "19990103",
+  "dow": "Sunday",
+  "longDate": "Sunday 3rd January",
+  "title": "Charlie's surprise.",
+  "characters": [ "Alice", "Zebedee", "Charlie", "Xavier" ],
+  "script": "William Shakespeare",
+  "summary": "",
+  "lines": [
+    "Charlie is very surprised by Alice and Xavier's surprise plan."
+  ]
 }
+]
diff --git a/file/src/test/resources/bug/DATE.csv b/file/src/test/resources/bug/DATE.csv
new file mode 100644
index 0000000..2999baf
--- /dev/null
+++ b/file/src/test/resources/bug/DATE.csv
@@ -0,0 +1,9 @@
+EMPNO:int,JOINEDAT:date,JOINTIME:time,JOINTIMES:timestamp
+100,"1996-08-03","00:01:02","1996-08-03 00:01:02"
+110,"2001-01-01","00:00:00","2001-01-01 00:00:00"
+110,"2002-05-03","00:00:00","2002-05-03 00:00:00"
+120,"2005-09-07","00:00:00","2005-09-07 00:00:00"
+130,"2007-01-01","00:00:00","2007-01-01 00:00:00"
+140,"2015-12-31","07:15:56","2015-12-31 07:15:56"
+150,"2015-12-31","13:31:21","2015-12-31 13:31:21"
+200,,,
diff --git a/file/src/test/resources/bug/LONG_EMPS.csv b/file/src/test/resources/bug/LONG_EMPS.csv
new file mode 100644
index 0000000..f69e0c5
--- /dev/null
+++ b/file/src/test/resources/bug/LONG_EMPS.csv
@@ -0,0 +1,6 @@
+EMPNO:long,NAME:string,DEPTNO:int,GENDER:string,CITY:string,EMPID:int,AGE:int,SLACKER:boolean,MANAGER:boolean,JOINEDAT:date
+100,"Fred",10,,,30,25,true,false,"1996-08-03"
+110,"Eric",20,"M","San Francisco",3,80,,false,"2001-01-01"
+110,"John",40,"M","Vancouver",2,,false,true,"2002-05-03"
+120,"Wilma",20,"F",,1,5,,true,"2005-09-07"
+130,"Alice",40,"F","Vancouver",2,,false,true,"2007-01-01"
diff --git a/file/src/test/resources/bug/WACKY_COLUMN_NAMES.csv b/file/src/test/resources/bug/WACKY_COLUMN_NAMES.csv
new file mode 100644
index 0000000..453d961
--- /dev/null
+++ b/file/src/test/resources/bug/WACKY_COLUMN_NAMES.csv
@@ -0,0 +1,6 @@
+EMPNO:int,naME:string,DEPTNO:Integer,2gender:string,CITY:string,EMPID:int,AGE:int,SLACKER:boolean,MANAGER:boolean,joined at:date
+100,"Fred",10,,,30,25,true,false,"1996-08-03"
+110,"Eric",20,"M","San Francisco",3,80,,false,"2001-01-01"
+110,"John",40,"M","Vancouver",2,,false,true,"2002-05-03"
+120,"Wilma",20,"F",,1,5,,true,"2005-09-07"
+130,"Alice",40,"F","Vancouver",2,,false,true,"2007-01-01"
diff --git a/file/build.gradle.kts b/file/src/test/resources/filterable-model.json
similarity index 62%
copy from file/build.gradle.kts
copy to file/src/test/resources/filterable-model.json
index 1610a03..abc2ab6 100644
--- a/file/build.gradle.kts
+++ b/file/src/test/resources/filterable-model.json
@@ -13,16 +13,23 @@
  * 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.
+ *
+ * A JSON model of a Calcite schema that is similar to model.json,
+ * except that it produces tables that implement FilterableTable.
+ * These tables can implement their own simple filtering.
  */
-dependencies {
-    api(project(":core"))
-    api(project(":example:csv"))
-    api(project(":linq4j"))
-
-    implementation("com.google.guava:guava")
-    implementation("com.joestelmach:natty")
-    implementation("org.apache.calcite.avatica:avatica-core")
-    implementation("org.jsoup:jsoup")
-
-    testImplementation(project(":core", "testClasses"))
+{
+  "version": "1.0",
+  "defaultSchema": "SALES",
+  "schemas": [
+    {
+      "name": "SALES",
+      "type": "custom",
+      "factory": "org.apache.calcite.adapter.file.FileSchemaFactory",
+      "operand": {
+        "directory": "sales",
+        "flavor": "FILTERABLE"
+      }
+    }
+  ]
 }
diff --git a/file/build.gradle.kts b/file/src/test/resources/model-with-custom-table.json
similarity index 66%
copy from file/build.gradle.kts
copy to file/src/test/resources/model-with-custom-table.json
index 1610a03..19aa032 100644
--- a/file/build.gradle.kts
+++ b/file/src/test/resources/model-with-custom-table.json
@@ -14,15 +14,23 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-dependencies {
-    api(project(":core"))
-    api(project(":example:csv"))
-    api(project(":linq4j"))
-
-    implementation("com.google.guava:guava")
-    implementation("com.joestelmach:natty")
-    implementation("org.apache.calcite.avatica:avatica-core")
-    implementation("org.jsoup:jsoup")
-
-    testImplementation(project(":core", "testClasses"))
+{
+  "version": "1.0",
+  "defaultSchema": "CUSTOM_TABLE",
+  "schemas": [
+    {
+      "name": "CUSTOM_TABLE",
+      "tables": [
+        {
+          "name": "EMPS",
+          "type": "custom",
+          "factory": "org.apache.calcite.adapter.file.CsvTableFactory",
+          "operand": {
+            "file": "sales/EMPS.csv.gz",
+            "flavor": "scannable"
+          }
+        }
+      ]
+    }
+  ]
 }
diff --git a/file/build.gradle.kts b/file/src/test/resources/model.json
similarity index 69%
copy from file/build.gradle.kts
copy to file/src/test/resources/model.json
index 1610a03..826ca1a 100644
--- a/file/build.gradle.kts
+++ b/file/src/test/resources/model.json
@@ -13,16 +13,20 @@
  * 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.
+ *
+ * A JSON model of a simple Calcite schema.
  */
-dependencies {
-    api(project(":core"))
-    api(project(":example:csv"))
-    api(project(":linq4j"))
-
-    implementation("com.google.guava:guava")
-    implementation("com.joestelmach:natty")
-    implementation("org.apache.calcite.avatica:avatica-core")
-    implementation("org.jsoup:jsoup")
-
-    testImplementation(project(":core", "testClasses"))
+{
+  "version": "1.0",
+  "defaultSchema": "SALES",
+  "schemas": [
+    {
+      "name": "SALES",
+      "type": "custom",
+      "factory": "org.apache.calcite.adapter.file.FileSchemaFactory",
+      "operand": {
+        "directory": "sales"
+      }
+    }
+  ]
 }
diff --git a/file/src/test/resources/sales/DEPTS.csv b/file/src/test/resources/sales/DEPTS.csv
new file mode 100644
index 0000000..628f2d8
--- /dev/null
+++ b/file/src/test/resources/sales/DEPTS.csv
@@ -0,0 +1,4 @@
+DEPTNO:int,NAME:string
+10,"Sales"
+20,"Marketing"
+30,"Accounts"
diff --git a/file/src/test/resources/sales/EMPS.csv.gz b/file/src/test/resources/sales/EMPS.csv.gz
new file mode 100644
index 0000000..294bad4
Binary files /dev/null and b/file/src/test/resources/sales/EMPS.csv.gz differ
diff --git a/file/build.gradle.kts b/file/src/test/resources/smart.json
similarity index 52%
copy from file/build.gradle.kts
copy to file/src/test/resources/smart.json
index 1610a03..9b86761 100644
--- a/file/build.gradle.kts
+++ b/file/src/test/resources/smart.json
@@ -13,16 +13,28 @@
  * 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.
+ *
+ * A JSON model of a Calcite schema that is similar to model.json,
+ * except that it produces tables that implement FilterableTable.
+ * These tables can implement their own simple filtering.
+ *
+ * A JSON model of a Calcite schema that is similar to model.json,
+ * except that it produces tables that implement
+ * TranslatableTable. These tables are translated to a CsvTableScan
+ * relational expression which participates in query planning.
  */
-dependencies {
-    api(project(":core"))
-    api(project(":example:csv"))
-    api(project(":linq4j"))
-
-    implementation("com.google.guava:guava")
-    implementation("com.joestelmach:natty")
-    implementation("org.apache.calcite.avatica:avatica-core")
-    implementation("org.jsoup:jsoup")
-
-    testImplementation(project(":core", "testClasses"))
+{
+  "version": "1.0",
+  "defaultSchema": "SALES",
+  "schemas": [
+    {
+      "name": "SALES",
+      "type": "custom",
+      "factory": "org.apache.calcite.adapter.file.FileSchemaFactory",
+      "operand": {
+        "directory": "sales",
+        "flavor": "TRANSLATABLE"
+      }
+    }
+  ]
 }


[calcite] 04/09: [CALCITE-4080] Allow character literals as column aliases, if SqlConformance.allowCharLiteralAlias()

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

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

commit 04314987373f0d7dd84b7b4e8df3243eef7f8f68
Author: Julian Hyde <jh...@apache.org>
AuthorDate: Mon Jul 20 23:49:16 2020 -0700

    [CALCITE-4080] Allow character literals as column aliases, if SqlConformance.allowCharLiteralAlias()
    
    This is for compatibility with MySQL, Microsoft SQL Server,
    and BigQuery dialects of SQL.
    
    Close apache/calcite#2087
---
 core/src/main/codegen/templates/Parser.jj          |  39 +++++-
 .../apache/calcite/runtime/CalciteResource.java    |   3 +
 .../sql/validate/SqlAbstractConformance.java       |   4 +
 .../calcite/sql/validate/SqlConformance.java       |  19 +++
 .../calcite/sql/validate/SqlConformanceEnum.java   |  14 ++
 .../calcite/runtime/CalciteResource.properties     |   1 +
 .../apache/calcite/sql/parser/SqlParserTest.java   | 142 ++++++++++++++++++++-
 7 files changed, 215 insertions(+), 7 deletions(-)

diff --git a/core/src/main/codegen/templates/Parser.jj b/core/src/main/codegen/templates/Parser.jj
index b317cb3..1d8465e 100644
--- a/core/src/main/codegen/templates/Parser.jj
+++ b/core/src/main/codegen/templates/Parser.jj
@@ -1828,7 +1828,15 @@ SqlNode SelectItem() :
     e = SelectExpression()
     [
         [ <AS> ]
-        id = SimpleIdentifier() {
+        (
+            id = SimpleIdentifier()
+        |
+            // Mute the warning about ambiguity between alias and continued
+            // string literal.
+            LOOKAHEAD(1)
+            id = SimpleIdentifierFromStringLiteral()
+        )
+        {
             e = SqlStdOperatorTable.AS.createCall(span().end(e), e, id);
         }
     ]
@@ -4117,6 +4125,12 @@ SqlNode StringLiteral() :
         }
     }
     (
+        // The grammar is ambiguous when a continued literals and a character
+        // string alias are both possible. For example, in
+        //   SELECT x'01'\n'ab'
+        // we prefer that 'ab' continues the literal, and is not an alias.
+        // The following LOOKAHEAD mutes the warning about ambiguity.
+        LOOKAHEAD(1)
         <QUOTED_STRING>
         {
             try {
@@ -4166,6 +4180,12 @@ SqlNode StringLiteral() :
         nfrags++;
     }
     (
+        // The grammar is ambiguous when a continued literals and a character
+        // string alias are both possible. For example, in
+        //   SELECT 'taxi'\n'cab'
+        // we prefer that 'cab' continues the literal, and is not an alias.
+        // The following LOOKAHEAD mutes the warning about ambiguity.
+        LOOKAHEAD(1)
         <QUOTED_STRING>
         {
             p = SqlParserUtil.parseString(token.image);
@@ -4679,6 +4699,23 @@ SqlIdentifier SimpleIdentifier() :
 }
 
 /**
+ * Parses a character literal as an SqlIdentifier.
+ * Only valid for column aliases in certain dialects.
+ */
+SqlIdentifier SimpleIdentifierFromStringLiteral() :
+{
+}
+{
+    <QUOTED_STRING> {
+        if (!this.conformance.allowCharLiteralAlias()) {
+            throw SqlUtil.newContextException(getPos(), RESOURCE.charLiteralAliasNotValid());
+        }
+        final String s = SqlParserUtil.parseString(token.image);
+        return new SqlIdentifier(s, getPos());
+    }
+}
+
+/**
  * Parses a comma-separated list of simple identifiers.
  */
 void SimpleIdentifierCommaList(List<SqlNode> list) :
diff --git a/core/src/main/java/org/apache/calcite/runtime/CalciteResource.java b/core/src/main/java/org/apache/calcite/runtime/CalciteResource.java
index d932321..53fe755 100644
--- a/core/src/main/java/org/apache/calcite/runtime/CalciteResource.java
+++ b/core/src/main/java/org/apache/calcite/runtime/CalciteResource.java
@@ -283,6 +283,9 @@ public interface CalciteResource {
   @BaseMessage("Table or column alias must be a simple identifier")
   ExInst<SqlValidatorException> aliasMustBeSimpleIdentifier();
 
+  @BaseMessage("Expecting alias, found character literal")
+  ExInst<SqlValidatorException> charLiteralAliasNotValid();
+
   @BaseMessage("List of column aliases must have same degree as table; table has {0,number,#} columns {1}, whereas alias list has {2,number,#} columns")
   ExInst<SqlValidatorException> aliasListDegree(int a0, String a1, int a2);
 
diff --git a/core/src/main/java/org/apache/calcite/sql/validate/SqlAbstractConformance.java b/core/src/main/java/org/apache/calcite/sql/validate/SqlAbstractConformance.java
index 842ee92..531543c 100644
--- a/core/src/main/java/org/apache/calcite/sql/validate/SqlAbstractConformance.java
+++ b/core/src/main/java/org/apache/calcite/sql/validate/SqlAbstractConformance.java
@@ -27,6 +27,10 @@ public abstract class SqlAbstractConformance implements SqlConformance {
     return SqlConformanceEnum.DEFAULT.isLiberal();
   }
 
+  public boolean allowCharLiteralAlias() {
+    return SqlConformanceEnum.DEFAULT.allowCharLiteralAlias();
+  }
+
   public boolean isGroupByAlias() {
     return SqlConformanceEnum.DEFAULT.isGroupByAlias();
   }
diff --git a/core/src/main/java/org/apache/calcite/sql/validate/SqlConformance.java b/core/src/main/java/org/apache/calcite/sql/validate/SqlConformance.java
index dc07d1b..78b7db3 100644
--- a/core/src/main/java/org/apache/calcite/sql/validate/SqlConformance.java
+++ b/core/src/main/java/org/apache/calcite/sql/validate/SqlConformance.java
@@ -68,6 +68,25 @@ public interface SqlConformance {
   boolean isLiberal();
 
   /**
+   * Whether this dialect allows character literals as column aliases.
+   *
+   * <p>For example,
+   *
+   * <blockquote><pre>
+   *   SELECT empno, sal + comm AS 'remuneration'
+   *   FROM Emp</pre></blockquote>
+   *
+   * <p>Among the built-in conformance levels, true in
+   * {@link SqlConformanceEnum#BABEL},
+   * {@link SqlConformanceEnum#BIG_QUERY},
+   * {@link SqlConformanceEnum#LENIENT},
+   * {@link SqlConformanceEnum#MYSQL_5},
+   * {@link SqlConformanceEnum#SQL_SERVER_2008};
+   * false otherwise.
+   */
+  boolean allowCharLiteralAlias();
+
+  /**
    * Whether to allow aliases from the {@code SELECT} clause to be used as
    * column names in the {@code GROUP BY} clause.
    *
diff --git a/core/src/main/java/org/apache/calcite/sql/validate/SqlConformanceEnum.java b/core/src/main/java/org/apache/calcite/sql/validate/SqlConformanceEnum.java
index 3d689a1..ffccfd7 100644
--- a/core/src/main/java/org/apache/calcite/sql/validate/SqlConformanceEnum.java
+++ b/core/src/main/java/org/apache/calcite/sql/validate/SqlConformanceEnum.java
@@ -88,6 +88,20 @@ public enum SqlConformanceEnum implements SqlConformance {
     }
   }
 
+  public boolean allowCharLiteralAlias() {
+    switch (this) {
+    case BABEL:
+    case BIG_QUERY:
+    case LENIENT:
+    case MYSQL_5:
+    case SQL_SERVER_2008:
+      return true;
+    default:
+      return false;
+    }
+  }
+
+
   public boolean isGroupByAlias() {
     switch (this) {
     case BABEL:
diff --git a/core/src/main/resources/org/apache/calcite/runtime/CalciteResource.properties b/core/src/main/resources/org/apache/calcite/runtime/CalciteResource.properties
index 079ef8c..9f4b453 100644
--- a/core/src/main/resources/org/apache/calcite/runtime/CalciteResource.properties
+++ b/core/src/main/resources/org/apache/calcite/runtime/CalciteResource.properties
@@ -97,6 +97,7 @@ NumberLiteralOutOfRange=Numeric literal ''{0}'' out of range
 DateLiteralOutOfRange=Date literal ''{0}'' out of range
 StringFragsOnSameLine=String literal continued on same line
 AliasMustBeSimpleIdentifier=Table or column alias must be a simple identifier
+CharLiteralAliasNotValid=Expecting alias, found character literal
 AliasListDegree=List of column aliases must have same degree as table; table has {0,number,#} columns {1}, whereas alias list has {2,number,#} columns
 AliasListDuplicate=Duplicate name ''{0}'' in column alias list
 JoinRequiresCondition=INNER, LEFT, RIGHT or FULL join requires a condition (NATURAL keyword or ON or USING clause)
diff --git a/core/src/test/java/org/apache/calcite/sql/parser/SqlParserTest.java b/core/src/test/java/org/apache/calcite/sql/parser/SqlParserTest.java
index 603beec..4fa3451 100644
--- a/core/src/test/java/org/apache/calcite/sql/parser/SqlParserTest.java
+++ b/core/src/test/java/org/apache/calcite/sql/parser/SqlParserTest.java
@@ -2123,6 +2123,79 @@ public class SqlParserTest {
             + "WHERE (`x`.`DEPTNO` IN (10, 20))");
   }
 
+  /** Test case for
+   * <a href="https://issues.apache.org/jira/browse/CALCITE-4080">[CALCITE-4080]
+   * Allow character literals as column aliases, if
+   * SqlConformance.allowCharLiteralAlias()</a>. */
+  @Test void testSingleQuotedAlias() {
+    final String expectingAlias = "Expecting alias, found character literal";
+
+    final String sql1 = "select 1 as ^'a b'^ from t";
+    conformance = SqlConformanceEnum.DEFAULT;
+    sql(sql1).fails(expectingAlias);
+    conformance = SqlConformanceEnum.MYSQL_5;
+    final String sql1b = "SELECT 1 AS `a b`\n"
+        + "FROM `T`";
+    sql(sql1).sansCarets().ok(sql1b);
+    conformance = SqlConformanceEnum.BIG_QUERY;
+    sql(sql1).sansCarets().ok(sql1b);
+    conformance = SqlConformanceEnum.SQL_SERVER_2008;
+    sql(sql1).sansCarets().ok(sql1b);
+
+    // valid on MSSQL (alias contains a single quote)
+    final String sql2 = "with t as (select 1 as ^'x''y'^)\n"
+        + "select [x'y] from t as [u]";
+    conformance = SqlConformanceEnum.DEFAULT;
+    quoting = Quoting.BRACKET;
+    sql(sql2).fails(expectingAlias);
+    conformance = SqlConformanceEnum.MYSQL_5;
+    final String sql2b = "WITH `T` AS (SELECT 1 AS `x'y`) (SELECT `x'y`\n"
+        + "FROM `T` AS `u`)";
+    sql(sql2).sansCarets().ok(sql2b);
+    conformance = SqlConformanceEnum.BIG_QUERY;
+    sql(sql2).sansCarets().ok(sql2b);
+    conformance = SqlConformanceEnum.SQL_SERVER_2008;
+    sql(sql2).sansCarets().ok(sql2b);
+
+    // also valid on MSSQL
+    final String sql3 = "with [t] as (select 1 as [x]) select [x] from [t]";
+    final String sql3b = "WITH `t` AS (SELECT 1 AS `x`) (SELECT `x`\n"
+        + "FROM `t`)";
+    conformance = SqlConformanceEnum.DEFAULT;
+    quoting = Quoting.BRACKET;
+    sql(sql3).sansCarets().ok(sql3b);
+    conformance = SqlConformanceEnum.MYSQL_5;
+    sql(sql3).sansCarets().ok(sql3b);
+    conformance = SqlConformanceEnum.BIG_QUERY;
+    sql(sql3).sansCarets().ok(sql3b);
+    conformance = SqlConformanceEnum.SQL_SERVER_2008;
+    sql(sql3).sansCarets().ok(sql3b);
+
+    // char literal as table alias is invalid on MSSQL (and others)
+    final String sql4 = "with t as (select 1 as x) select x from t as ^'u'^";
+    final String sql4b = "(?s)Encountered \"\\\\'u\\\\'\" at .*";
+    conformance = SqlConformanceEnum.DEFAULT;
+    sql(sql4).fails(sql4b);
+    conformance = SqlConformanceEnum.MYSQL_5;
+    sql(sql4).fails(sql4b);
+    conformance = SqlConformanceEnum.BIG_QUERY;
+    sql(sql4).fails(sql4b);
+    conformance = SqlConformanceEnum.SQL_SERVER_2008;
+    sql(sql4).fails(sql4b);
+
+    // char literal as table alias (without AS) is invalid on MSSQL (and others)
+    final String sql5 = "with t as (select 1 as x) select x from t ^'u'^";
+    final String sql5b = "(?s)Encountered \"\\\\'u\\\\'\" at .*";
+    conformance = SqlConformanceEnum.DEFAULT;
+    sql(sql5).fails(sql5b);
+    conformance = SqlConformanceEnum.MYSQL_5;
+    sql(sql5).fails(sql5b);
+    conformance = SqlConformanceEnum.BIG_QUERY;
+    sql(sql5).fails(sql5b);
+    conformance = SqlConformanceEnum.SQL_SERVER_2008;
+    sql(sql5).fails(sql5b);
+  }
+
   @Test void testInList() {
     sql("select * from emp where deptno in (10, 20) and gender = 'F'")
         .ok("SELECT *\n"
@@ -2687,6 +2760,62 @@ public class SqlParserTest {
         .fails("Binary literal string must contain only characters '0' - '9', 'A' - 'F'");
   }
 
+  /** Tests that ambiguity between extended string literals and character string
+   * aliases is always resolved in favor of extended string literals. */
+  @Test void testContinuedLiteralAlias() {
+    final String expectingAlias = "Expecting alias, found character literal";
+
+    // Not ambiguous, because of 'as'.
+    final String sql0 = "select 1 an_alias,\n"
+        + "  x'01'\n"
+        + "  'ab' as x\n"
+        + "from t";
+    final String sql0b = "SELECT 1 AS `AN_ALIAS`, X'01'\n"
+        + "'AB' AS `X`\n"
+        + "FROM `T`";
+    conformance = SqlConformanceEnum.DEFAULT;
+    sql(sql0).ok(sql0b);
+    conformance = SqlConformanceEnum.MYSQL_5;
+    sql(sql0).ok(sql0b);
+    conformance = SqlConformanceEnum.BIG_QUERY;
+    sql(sql0).ok(sql0b);
+
+    // Is 'ab' an alias or is it part of the x'01' 'ab' continued binary string
+    // literal? It's ambiguous, but we prefer the latter.
+    final String sql1 = "select 1 ^'an alias'^,\n"
+        + "  x'01'\n"
+        + "  'ab'\n"
+        + "from t";
+    final String sql1b = "SELECT 1 AS `an alias`, X'01'\n"
+        + "'AB'\n"
+        + "FROM `T`";
+    conformance = SqlConformanceEnum.DEFAULT;
+    sql(sql1).fails(expectingAlias);
+    conformance = SqlConformanceEnum.MYSQL_5;
+    sql(sql1).sansCarets().ok(sql1b);
+    conformance = SqlConformanceEnum.BIG_QUERY;
+    sql(sql1).sansCarets().ok(sql1b);
+
+    // Parser prefers continued character and binary string literals over
+    // character string aliases, regardless of whether the dialect allows
+    // character string aliases.
+    final String sql2 = "select 'continued'\n"
+        + "  'char literal, not alias',\n"
+        + "  x'01'\n"
+        + "  'ab'\n"
+        + "from t";
+    final String sql2b = "SELECT 'continued'\n"
+        + "'char literal, not alias', X'01'\n"
+        + "'AB'\n"
+        + "FROM `T`";
+    conformance = SqlConformanceEnum.DEFAULT;
+    sql(sql2).ok(sql2b);
+    conformance = SqlConformanceEnum.MYSQL_5;
+    sql(sql2).ok(sql2b);
+    conformance = SqlConformanceEnum.BIG_QUERY;
+    sql(sql2).ok(sql2b);
+  }
+
   @Test void testMixedFrom() {
     // REVIEW: Is this syntax even valid?
     sql("select * from a join b using (x), c join d using (y)")
@@ -3960,8 +4089,8 @@ public class SqlParserTest {
 
   @Test void testBitStringNotImplemented() {
     // Bit-string is longer part of the SQL standard. We do not support it.
-    sql("select B^'1011'^ || 'foobar' from (values (true))")
-        .fails("(?s).*Encountered \"\\\\'1011\\\\'\" at line 1, column 9.*");
+    sql("select (B^'1011'^ || 'foobar') from (values (true))")
+        .fails("(?s).*Encountered \"\\\\'1011\\\\'\" at .*");
   }
 
   @Test void testHexAndBinaryString() {
@@ -4040,16 +4169,17 @@ public class SqlParserTest {
   }
 
   @Test void testStringLiteralFails() {
-    sql("select N ^'space'^")
+    sql("select (N ^'space'^)")
         .fails("(?s).*Encountered .*space.* at line 1, column ...*");
-    sql("select _latin1\n^'newline'^")
+    sql("select (_latin1\n^'newline'^)")
         .fails("(?s).*Encountered.*newline.* at line 2, column ...*");
     sql("select ^_unknown-charset''^ from (values(true))")
         .fails("Unknown character set 'unknown-charset'");
 
     // valid syntax, but should give a validator error
-    sql("select N'1' '2' from t")
-        .ok("SELECT _ISO-8859-1'1'\n'2'\n"
+    sql("select (N'1' '2') from t")
+        .ok("SELECT _ISO-8859-1'1'\n"
+            + "'2'\n"
             + "FROM `T`");
   }
 


[calcite] 03/09: [CALCITE-4137] Checkstyle should ensure that every class has a Javadoc comment

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

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

commit a1bdba6d79ac05ef227cdb15b7c044c6e0e54dfa
Author: Julian Hyde <jh...@apache.org>
AuthorDate: Fri Jul 24 18:21:02 2020 -0700

    [CALCITE-4137] Checkstyle should ensure that every class has a Javadoc comment
    
    Checkstyle should ensure that every class, interface, enum, package has
    a Javadoc comment. And that javadoc types (e.g @param and @return) are
    not empty.
    
    Following [CALCITE-2905], remove @PackageMarker from Redis adapter.
---
 .../adapter/cassandra/CassandraEnumerator.java     |   2 +-
 .../calcite/adapter/cassandra/CassandraRules.java  |   3 -
 .../calcite/adapter/cassandra/CassandraSchema.java |   7 +-
 .../adapter/cassandra/CassandraSchemaFactory.java  |   2 +-
 .../calcite/adapter/cassandra/CassandraTable.java  |   2 +-
 .../apache/calcite/test/CassandraExtension.java    |   9 +-
 .../calcite/adapter/enumerable/EnumUtils.java      |   7 +-
 .../calcite/adapter/enumerable/JavaRowFormat.java  |   2 +-
 .../adapter/enumerable/NestedBlockBuilder.java     |   3 +-
 .../adapter/enumerable/NestedBlockBuilderImpl.java |   3 +-
 .../ReflectiveCallNotNullImplementor.java          |   5 +-
 .../adapter/enumerable/RexToLixTranslator.java     |   2 +-
 .../enumerable/impl/AggResetContextImpl.java       |   2 +-
 .../enumerable/impl/AggResultContextImpl.java      |   2 +-
 .../calcite/adapter/java/ReflectiveSchema.java     |   2 +-
 .../calcite/adapter/jdbc/JdbcImplementor.java      |   1 +
 .../org/apache/calcite/adapter/jdbc/JdbcRules.java |   8 +-
 .../apache/calcite/adapter/jdbc/JdbcSchema.java    |   2 +-
 .../calcite/config/CalciteConnectionConfig.java    |  67 +--
 .../calcite/config/CalciteSystemProperty.java      |   5 +-
 .../org/apache/calcite/interpreter/Bindables.java  |   1 +
 .../java/org/apache/calcite/interpreter/Row.java   |   5 +-
 .../org/apache/calcite/model/ModelHandler.java     |   1 +
 .../apache/calcite/plan/AbstractRelOptPlanner.java |   9 +-
 .../java/org/apache/calcite/plan/Convention.java   |   5 +-
 .../org/apache/calcite/plan/RelOptCluster.java     |   1 +
 .../java/org/apache/calcite/plan/RelOptCost.java   |  22 +-
 .../org/apache/calcite/plan/RelOptPlanner.java     |   8 +-
 .../java/org/apache/calcite/plan/RelOptRule.java   |   4 +-
 .../org/apache/calcite/plan/RelOptRuleCall.java    |   2 +-
 .../org/apache/calcite/plan/RelOptRuleOperand.java |   4 +-
 .../java/org/apache/calcite/plan/RelOptRules.java  |   5 +-
 .../java/org/apache/calcite/plan/RelOptTable.java  |   2 +-
 .../java/org/apache/calcite/plan/RelOptUtil.java   |  21 +-
 .../java/org/apache/calcite/plan/RelTraitDef.java  |  10 +-
 .../java/org/apache/calcite/plan/RelTraitSet.java  |   2 +-
 .../org/apache/calcite/plan/TableAccessMap.java    |   4 +-
 .../apache/calcite/plan/VisitorDataContext.java    |   2 +-
 .../org/apache/calcite/plan/hep/HepRelVertex.java  |   2 +-
 .../calcite/plan/volcano/IterativeRuleQueue.java   |   2 +-
 .../org/apache/calcite/plan/volcano/RelSet.java    |   6 +-
 .../org/apache/calcite/plan/volcano/RelSubset.java |  34 +-
 .../calcite/plan/volcano/TopDownRuleDriver.java    |   6 +-
 .../calcite/plan/volcano/VolcanoPlanner.java       |   2 +-
 .../org/apache/calcite/prepare/PlannerImpl.java    |   5 +-
 .../org/apache/calcite/rel/AbstractRelNode.java    |  16 +-
 .../main/java/org/apache/calcite/rel/RelNode.java  |  16 +-
 .../java/org/apache/calcite/rel/RelWriter.java     |   4 +-
 .../org/apache/calcite/rel/convert/Converter.java  |   2 +-
 .../org/apache/calcite/rel/core/Aggregate.java     |   4 +-
 .../org/apache/calcite/rel/core/CorrelationId.java |   2 +-
 .../java/org/apache/calcite/rel/core/Join.java     |   1 +
 .../org/apache/calcite/rel/core/RepeatUnion.java   |   2 +-
 .../calcite/rel/metadata/RelColumnOrigin.java      |  12 +-
 .../calcite/rel/metadata/RelMdPredicates.java      |   1 +
 .../org/apache/calcite/rel/metadata/RelMdUtil.java |  23 +-
 .../apache/calcite/rel/mutable/MutableCalc.java    |   2 +-
 .../calcite/rel/mutable/MutableMultiRel.java       |   2 +-
 .../calcite/rel/rel2sql/RelToSqlConverter.java     |  32 +-
 .../apache/calcite/rel/rel2sql/SqlImplementor.java |   8 +-
 .../rel/rules/AggregateReduceFunctionsRule.java    |   2 +-
 .../rel/rules/JoinProjectTransposeRule.java        |  15 +-
 .../calcite/rel/rules/JoinToMultiJoinRule.java     |  16 +-
 .../org/apache/calcite/rel/rules/LoptJoinTree.java |   4 +-
 .../apache/calcite/rel/rules/LoptMultiJoin.java    | 224 +++++-----
 .../calcite/rel/rules/LoptOptimizeJoinRule.java    |  12 +-
 .../calcite/rel/rules/LoptSemiJoinOptimizer.java   |  10 +-
 .../org/apache/calcite/rel/rules/MultiJoin.java    |  22 +-
 .../apache/calcite/rel/rules/PushProjector.java    |  10 +-
 .../calcite/rel/rules/ReduceDecimalsRule.java      |  36 +-
 .../rules/materialize/MaterializedViewRule.java    |  15 +-
 .../org/apache/calcite/rel/type/RelDataType.java   |  23 +-
 .../calcite/rel/type/RelDataTypeFactory.java       |   4 +-
 .../java/org/apache/calcite/rex/RexBuilder.java    |  15 +-
 .../java/org/apache/calcite/rex/RexLiteral.java    |   6 +-
 .../org/apache/calcite/rex/RexMultisetUtil.java    |   4 +-
 .../org/apache/calcite/rex/RexPatternFieldRef.java |   2 +-
 .../java/org/apache/calcite/rex/RexProgram.java    |   2 +-
 .../org/apache/calcite/rex/RexProgramBuilder.java  |   6 +-
 .../java/org/apache/calcite/rex/RexSimplify.java   |   4 +-
 .../rex/RexSqlReflectiveConvertletTable.java       |   2 +-
 .../org/apache/calcite/rex/RexTableInputRef.java   |   3 +-
 .../main/java/org/apache/calcite/rex/RexUtil.java  |   6 +-
 .../calcite/runtime/CalciteContextException.java   |  15 +-
 .../org/apache/calcite/runtime/Enumerables.java    |   2 +-
 .../apache/calcite/runtime/EnumeratorCursor.java   |   3 +-
 .../main/java/org/apache/calcite/runtime/Hook.java |   3 +
 .../java/org/apache/calcite/runtime/Matcher.java   |   2 +-
 .../java/org/apache/calcite/runtime/Pattern.java   |   6 +-
 .../apache/calcite/runtime/SocketFactoryImpl.java  |  24 +-
 .../org/apache/calcite/runtime/SqlFunctions.java   |   6 +-
 .../calcite/runtime/TrustAllSslSocketFactory.java  |   2 +
 .../java/org/apache/calcite/runtime/Utilities.java |   1 +
 .../schema/impl/ReflectiveFunctionBase.java        |   5 +-
 .../org/apache/calcite/sql/JoinConditionType.java  |  10 +-
 .../java/org/apache/calcite/sql/SqlAccessEnum.java |   2 +-
 .../java/org/apache/calcite/sql/SqlAccessType.java |   4 +-
 .../apache/calcite/sql/SqlBasicTypeNameSpec.java   |   2 +-
 .../apache/calcite/sql/SqlBinaryStringLiteral.java |  12 +-
 .../main/java/org/apache/calcite/sql/SqlCall.java  |   4 +-
 .../apache/calcite/sql/SqlCharStringLiteral.java   |  11 +-
 .../java/org/apache/calcite/sql/SqlCollation.java  |   6 +-
 .../java/org/apache/calcite/sql/SqlDelete.java     |   4 +-
 .../java/org/apache/calcite/sql/SqlExplain.java    |  10 +-
 .../java/org/apache/calcite/sql/SqlFunction.java   |  15 +-
 .../java/org/apache/calcite/sql/SqlInsert.java     |   8 +-
 .../apache/calcite/sql/SqlIntervalQualifier.java   |   2 +-
 .../apache/calcite/sql/SqlJdbcFunctionCall.java    |  10 +-
 .../calcite/sql/SqlJsonConstructorNullClause.java  |   2 +-
 .../main/java/org/apache/calcite/sql/SqlKind.java  | 453 ++++++---------------
 .../java/org/apache/calcite/sql/SqlLiteral.java    |  12 +-
 .../main/java/org/apache/calcite/sql/SqlMerge.java |  25 +-
 .../main/java/org/apache/calcite/sql/SqlNode.java  |   1 +
 .../java/org/apache/calcite/sql/SqlOperator.java   |  27 +-
 .../org/apache/calcite/sql/SqlOperatorBinding.java |  12 +-
 .../java/org/apache/calcite/sql/SqlUpdate.java     |  16 +-
 .../main/java/org/apache/calcite/sql/SqlUtil.java  |  10 +-
 .../java/org/apache/calcite/sql/SqlWindow.java     |   1 +
 .../apache/calcite/sql/SqlWindowTableFunction.java |   6 +-
 .../org/apache/calcite/sql/advise/SqlAdvisor.java  |  45 +-
 .../apache/calcite/sql/advise/SqlAdvisorHint2.java |   2 +-
 .../apache/calcite/sql/advise/SqlSimpleParser.java |  27 +-
 .../calcite/sql/dialect/BigQuerySqlDialect.java    |   6 +-
 .../java/org/apache/calcite/sql/fun/SqlCase.java   |   2 +-
 .../calcite/sql/fun/SqlLiteralChainOperator.java   |   2 +-
 .../calcite/sql/fun/SqlStdOperatorTable.java       |  11 +-
 .../apache/calcite/sql/parser/SqlParserPos.java    |  22 +-
 .../apache/calcite/sql/parser/SqlParserUtil.java   |  23 +-
 .../sql/type/MultisetOperandTypeChecker.java       |   4 +-
 .../org/apache/calcite/sql/type/OperandTypes.java  |  23 +-
 .../org/apache/calcite/sql/type/ReturnTypes.java   |  21 +-
 .../calcite/sql/type/SqlOperandTypeChecker.java    |   4 +-
 .../org/apache/calcite/sql/type/SqlTypeFamily.java |   8 +-
 .../org/apache/calcite/sql/type/SqlTypeName.java   |  16 +-
 .../calcite/sql/type/SqlTypeTransformCascade.java  |   2 +-
 .../apache/calcite/sql/type/SqlTypeTransforms.java |   2 +-
 .../org/apache/calcite/sql/type/SqlTypeUtil.java   | 164 ++------
 .../org/apache/calcite/sql/util/SqlVisitor.java    |   2 +-
 .../sql/validate/AggregatingSelectScope.java       |   2 +-
 .../apache/calcite/sql/validate/SelectScope.java   |   4 +-
 .../apache/calcite/sql/validate/SqlMoniker.java    |   2 +-
 .../apache/calcite/sql/validate/SqlValidator.java  |  16 +-
 .../sql/validate/SqlValidatorCatalogReader.java    |   5 +-
 .../calcite/sql/validate/SqlValidatorImpl.java     |  15 +-
 .../calcite/sql/validate/SqlValidatorScope.java    |   2 +
 .../calcite/sql/validate/SqlValidatorTable.java    |   2 +-
 .../sql/validate/SqlValidatorWithHints.java        |   4 +-
 .../calcite/sql/validate/TableNamespace.java       |   4 +-
 .../sql/validate/implicit/TypeCoercionImpl.java    |   7 +-
 .../calcite/sql2rel/ReflectiveConvertletTable.java |   2 +-
 .../apache/calcite/sql2rel/RelDecorrelator.java    |   2 +-
 .../apache/calcite/sql2rel/SqlToRelConverter.java  |  12 +-
 .../apache/calcite/sql2rel/SubQueryConverter.java  |   4 +-
 .../org/apache/calcite/tools/FrameworkConfig.java  |   2 +-
 .../org/apache/calcite/tools/PigRelBuilder.java    |   4 +-
 .../java/org/apache/calcite/tools/Planner.java     |   1 +
 .../java/org/apache/calcite/tools/RelBuilder.java  |   5 +-
 .../java/org/apache/calcite/util/BlackholeMap.java |   2 +-
 .../src/main/java/org/apache/calcite/util/Bug.java |   3 +-
 .../java/org/apache/calcite/util/CancelFlag.java   |   4 +-
 .../org/apache/calcite/util/ConversionUtil.java    |   4 +-
 .../apache/calcite/util/DateTimeStringUtils.java   |   3 +-
 .../org/apache/calcite/util/ImmutableBeans.java    |   2 +-
 .../java/org/apache/calcite/util/NumberUtil.java   |   6 +-
 .../main/java/org/apache/calcite/util/Sources.java |   7 +-
 .../java/org/apache/calcite/util/StackWriter.java  |  16 +-
 .../main/java/org/apache/calcite/util/Util.java    |  11 +-
 .../java/org/apache/calcite/util/XmlOutput.java    |  29 +-
 .../util/graph/AttributedDirectedGraph.java        |   1 +
 .../util/mapping/AbstractSourceMapping.java        |   4 +-
 .../util/mapping/AbstractTargetMapping.java        |   4 +-
 .../apache/calcite/util/trace/CalciteLogger.java   |  24 +-
 .../calcite/adapter/enumerable/TypeFinderTest.java |   3 +-
 .../calcite/jdbc/CalciteRemoteDriverTest.java      |   4 +-
 .../apache/calcite/jdbc/JavaTypeFactoryTest.java   |   4 +-
 .../materialize/NormalizationTrimFieldTest.java    |  23 +-
 .../org/apache/calcite/plan/RelOptUtilTest.java    |  31 +-
 .../org/apache/calcite/plan/RelWriterTest.java     |  13 +-
 .../apache/calcite/plan/volcano/ComboRuleTest.java |   2 +-
 .../calcite/plan/volcano/TraitConversionTest.java  |   5 +-
 .../calcite/plan/volcano/TraitPropagationTest.java |  14 +-
 .../calcite/plan/volcano/VolcanoPlannerTest.java   |  10 +-
 .../rel/rel2sql/RelToSqlConverterStructsTest.java  |  22 +-
 .../calcite/rel/rel2sql/RelToSqlConverterTest.java |   9 +-
 .../org/apache/calcite/rex/RexBuilderTest.java     |  10 +-
 .../org/apache/calcite/rex/RexProgramTest.java     |   7 +-
 .../org/apache/calcite/rex/RexProgramTestBase.java |   5 +-
 .../runtime/DeterministicAutomatonTest.java        |   2 +-
 .../apache/calcite/sql/parser/SqlParserTest.java   |   9 +-
 .../java/org/apache/calcite/sql/test/SqlTests.java |   4 +-
 .../calcite/sql/type/RelDataTypeSystemTest.java    |   2 +-
 .../calcite/sql2rel/RelFieldTrimmerTest.java       |   1 +
 .../org/apache/calcite/test/BookstoreSchema.java   |  15 +-
 .../org/apache/calcite/test/CalciteAssert.java     |  27 +-
 .../java/org/apache/calcite/test/DiffTestCase.java |   9 +-
 .../org/apache/calcite/test/HierarchySchema.java   |   4 +-
 .../org/apache/calcite/test/JdbcAdapterTest.java   |   5 +-
 .../apache/calcite/test/JdbcFrontLinqBackTest.java |  12 +-
 .../java/org/apache/calcite/test/JdbcTest.java     |  43 +-
 .../calcite/test/LogicalProjectDigestTest.java     |   4 +-
 .../java/org/apache/calcite/test/Matchers.java     |   4 +-
 .../org/apache/calcite/test/MutableRelTest.java    |   2 +-
 .../apache/calcite/test/ReflectiveSchemaTest.java  |  18 +-
 .../org/apache/calcite/test/RelBuilderTest.java    |   6 +-
 .../test/RelMdPercentageOriginalRowsTest.java      |  43 --
 .../org/apache/calcite/test/RelOptRulesTest.java   | 103 +++--
 .../org/apache/calcite/test/RelOptTestBase.java    |   1 +
 .../org/apache/calcite/test/RexShuttleTest.java    |   2 +-
 .../apache/calcite/test/SqlToRelConverterTest.java | 140 +++----
 .../org/apache/calcite/test/SqlValidatorTest.java  |   1 +
 .../org/apache/calcite/test/TypeCoercionTest.java  |   2 +-
 .../org/apache/calcite/test/catalog/Fixture.java   |   2 +-
 .../calcite/test/catalog/MockCatalogReader.java    |   2 +-
 .../test/concurrent/ConcurrentTestCommand.java     |   2 +-
 .../concurrent/ConcurrentTestCommandExecutor.java  |   2 +-
 .../concurrent/ConcurrentTestCommandGenerator.java |  10 +-
 .../concurrent/ConcurrentTestCommandScript.java    |   6 +-
 .../test/concurrent/ConcurrentTestPlugin.java      |   2 +-
 .../EnumerableBatchNestedLoopJoinTest.java         |   2 +-
 .../test/enumerable/EnumerableCalcTest.java        |   2 +-
 .../test/enumerable/EnumerableCorrelateTest.java   |  11 +-
 .../enumerable/EnumerableSortedAggregateTest.java  |   5 +-
 .../test/enumerable/EnumerableUncollectTest.java   |   1 +
 .../calcite/test/fuzzer/RexProgramFuzzyTest.java   |   2 +-
 .../java/org/apache/calcite/tools/PlannerTest.java |   8 +-
 .../java/org/apache/calcite/util/BitSetsTest.java  |   4 +-
 .../test/java/org/apache/calcite/util/Smalls.java  |  12 +-
 .../java/org/apache/calcite/util/SourceTest.java   |   2 +-
 .../java/org/apache/calcite/util/TestUtilTest.java |   2 +-
 .../adapter/druid/BinaryOperatorConversion.java    |   3 +-
 .../adapter/druid/CeilOperatorConversion.java      |   3 +-
 .../calcite/adapter/druid/ComplexMetric.java       |  12 +-
 .../calcite/adapter/druid/DruidDateTimeUtils.java  |   8 +-
 .../calcite/adapter/druid/DruidExpressions.java    |  16 +-
 .../calcite/adapter/druid/DruidJsonFilter.java     |  70 ++--
 .../apache/calcite/adapter/druid/DruidQuery.java   |  75 ++--
 .../apache/calcite/adapter/druid/DruidRules.java   |  10 +-
 .../adapter/druid/DruidSqlCastConverter.java       |   3 +-
 .../adapter/druid/DruidSqlOperatorConverter.java   |   3 +-
 .../apache/calcite/adapter/druid/DruidTable.java   |  13 +-
 .../apache/calcite/adapter/druid/DruidType.java    |  13 +-
 .../adapter/druid/ExtractOperatorConversion.java   |  12 +-
 .../adapter/druid/ExtractionDimensionSpec.java     |   6 +-
 .../adapter/druid/FloorOperatorConversion.java     |   3 +-
 .../adapter/druid/NaryOperatorConverter.java       |   3 +-
 .../adapter/druid/SubstringOperatorConversion.java |   2 +-
 .../adapter/druid/TimeExtractionFunction.java      |   8 +-
 .../druid/UnaryPrefixOperatorConversion.java       |   3 +-
 .../druid/UnarySuffixOperatorConversion.java       |   3 +-
 .../calcite/adapter/druid/VirtualColumn.java       |   2 +-
 .../org/apache/calcite/test/DruidAdapter2IT.java   | 127 +++---
 .../org/apache/calcite/test/DruidAdapterIT.java    | 195 ++++-----
 .../elasticsearch/ElasticsearchConstants.java      |   3 +-
 .../adapter/elasticsearch/ElasticsearchJson.java   |  31 +-
 .../adapter/elasticsearch/ElasticsearchRules.java  |   4 +-
 .../adapter/elasticsearch/ElasticsearchSchema.java |   2 +-
 .../elasticsearch/ElasticsearchSchemaFactory.java  |   3 +-
 .../elasticsearch/ElasticsearchSearchResult.java   |   6 +-
 .../elasticsearch/ElasticsearchTransport.java      |   3 +-
 .../adapter/elasticsearch/PredicateAnalyzer.java   |  26 +-
 .../adapter/elasticsearch/QueryBuilders.java       |   7 +-
 .../calcite/adapter/elasticsearch/Scrolling.java   |   5 +-
 .../adapter/elasticsearch/AggregationTest.java     |  10 +-
 .../adapter/elasticsearch/BooleanLogicTest.java    |   3 +-
 .../elasticsearch/ElasticSearchAdapterTest.java    |  14 +-
 .../elasticsearch/ElasticsearchVersionTest.java    |   2 +-
 .../elasticsearch/EmbeddedElasticsearchNode.java   |  13 +-
 .../elasticsearch/EmbeddedElasticsearchPolicy.java |   1 +
 .../calcite/adapter/elasticsearch/MatchTest.java   |   5 +-
 .../adapter/elasticsearch/Projection2Test.java     |   6 +-
 .../adapter/elasticsearch/ProjectionTest.java      |   2 +-
 .../adapter/elasticsearch/QueryBuildersTest.java   |   3 +-
 .../apache/calcite/test/ElasticsearchChecker.java  |   2 +-
 .../calcite/adapter/file/FileReaderTest.java       |   2 +-
 .../calcite/adapter/geode/rel/GeodeFilter.java     |  14 +-
 .../calcite/adapter/geode/rel/GeodeTable.java      |   2 +-
 .../calcite/adapter/geode/rel/package-info.java    |   2 +-
 .../geode/simple/GeodeSimpleScannableTable.java    |   2 +-
 .../calcite/adapter/geode/simple/package-info.java |   2 +-
 .../calcite/adapter/geode/util/GeodeUtils.java     |   2 +-
 .../calcite/adapter/geode/util/package-info.java   |   2 +-
 .../adapter/geode/rel/GeodeAllDataTypesTest.java   |   4 +-
 .../adapter/geode/simple/BookMasterRegionTest.java |   2 +-
 .../adapter/kafka/KafkaMessageEnumerator.java      |   5 +-
 .../calcite/adapter/kafka/KafkaRowConverter.java   |   8 +-
 .../adapter/kafka/KafkaRowConverterImpl.java       |  11 +-
 .../adapter/kafka/KafkaRowConverterTest.java       |  12 +-
 .../apache/calcite/linq4j/EnumerableDefaults.java  |  39 +-
 .../linq4j/tree/ArrayLengthRecordField.java        |   2 +-
 .../apache/calcite/linq4j/tree/BlockBuilder.java   |   6 +-
 .../apache/calcite/linq4j/tree/BlockStatement.java |   4 +-
 .../linq4j/tree/ConstructorDeclaration.java        |   4 +-
 .../linq4j/tree/DeterministicCodeOptimizer.java    |   2 +-
 .../calcite/linq4j/tree/ForEachStatement.java      |   4 +-
 .../apache/calcite/linq4j/tree/ForStatement.java   |   4 +-
 .../calcite/linq4j/tree/FunctionExpression.java    |   4 +-
 .../calcite/linq4j/tree/MethodCallExpression.java  |   4 +-
 .../calcite/linq4j/tree/NewArrayExpression.java    |   4 +-
 .../apache/calcite/linq4j/tree/NewExpression.java  |   4 +-
 .../calcite/linq4j/tree/OptimizeShuttle.java       |   5 +-
 .../org/apache/calcite/linq4j/tree/Primitive.java  |   4 +-
 .../calcite/linq4j/tree/ReflectedPseudoField.java  |   3 +-
 .../calcite/linq4j/MemoryEnumerableTest.java       |   2 +-
 .../calcite/linq4j/test/BlockBuilderBase.java      |   2 +-
 .../calcite/linq4j/test/BlockBuilderTest.java      |  13 +-
 .../calcite/linq4j/test/CorrelateJoinTest.java     |   2 +-
 .../calcite/linq4j/test/DeterministicTest.java     |   8 +-
 .../calcite/linq4j/test/JoinPreserveOrderTest.java |   4 +-
 .../calcite/adapter/mongodb/MongoEnumerator.java   |   4 +-
 .../calcite/adapter/mongodb/MongoAdapterTest.java  |   2 +-
 .../apache/calcite/adapter/pig/PigAggregate.java   |  11 +-
 .../org/apache/calcite/adapter/pig/PigFilter.java  |   6 +-
 .../org/apache/calcite/piglet/PigRelBuilder.java   |   6 +-
 .../org/apache/calcite/piglet/PigRelExVisitor.java |   8 +-
 .../calcite/piglet/PigRelOpInnerVisitor.java       |   1 +
 .../org/apache/calcite/piglet/PigRelOpVisitor.java |   6 +-
 .../org/apache/calcite/piglet/PigRelSqlUdfs.java   |   8 +-
 .../calcite/piglet/PigRelToSqlConverter.java       |   1 +
 .../calcite/piglet/PigUserDefinedFunction.java     |   4 +-
 .../calcite/chinook/ChosenCustomerEmail.java       |   2 +-
 .../org/apache/calcite/chinook/CodesFunction.java  |   2 +-
 .../apache/calcite/chinook/ConnectionFactory.java  |   4 +-
 .../apache/calcite/chinook/EnvironmentFairy.java   |   2 +-
 .../calcite/chinook/StringConcatFunction.java      |   2 +-
 .../RemotePreparedStatementParametersTest.java     |   3 +-
 .../apache/calcite/adapter/redis/package-info.java |   6 +-
 .../calcite/adapter/spark/SparkHandlerImpl.java    |   2 +-
 src/main/config/checkstyle/checker.xml             |  15 +
 .../enumerable/CodeGenerationBenchmark.java        |   2 +-
 .../calcite/adapter/enumerable/package-info.java   |   2 +-
 .../benchmarks/DefaultDirectedGraphBenchmark.java  |   3 +-
 .../apache/calcite/benchmarks/ParserBenchmark.java |   2 +-
 .../calcite/benchmarks/PreconditionTest.java       |   2 +-
 .../apache/calcite/benchmarks/package-info.java    |   2 +-
 334 files changed, 1747 insertions(+), 2203 deletions(-)

diff --git a/cassandra/src/main/java/org/apache/calcite/adapter/cassandra/CassandraEnumerator.java b/cassandra/src/main/java/org/apache/calcite/adapter/cassandra/CassandraEnumerator.java
index 0413819..a693db6 100644
--- a/cassandra/src/main/java/org/apache/calcite/adapter/cassandra/CassandraEnumerator.java
+++ b/cassandra/src/main/java/org/apache/calcite/adapter/cassandra/CassandraEnumerator.java
@@ -57,7 +57,7 @@ class CassandraEnumerator implements Enumerator<Object> {
     this.fieldTypes = protoRowType.apply(typeFactory).getFieldList();
   }
 
-  /** Produce the next row from the results
+  /** Produces the next row from the results.
    *
    * @return A new row from the results
    */
diff --git a/cassandra/src/main/java/org/apache/calcite/adapter/cassandra/CassandraRules.java b/cassandra/src/main/java/org/apache/calcite/adapter/cassandra/CassandraRules.java
index 6aad2a5..dd76da7 100644
--- a/cassandra/src/main/java/org/apache/calcite/adapter/cassandra/CassandraRules.java
+++ b/cassandra/src/main/java/org/apache/calcite/adapter/cassandra/CassandraRules.java
@@ -205,7 +205,6 @@ public class CassandraRules {
       }
     }
 
-    /** @see org.apache.calcite.rel.convert.ConverterRule */
     @Override public void onMatch(RelOptRuleCall call) {
       LogicalFilter filter = call.rel(0);
       CassandraTableScan scan = call.rel(1);
@@ -370,7 +369,6 @@ public class CassandraRules {
       }
     }
 
-    /** @see org.apache.calcite.rel.convert.ConverterRule */
     @Override public void onMatch(RelOptRuleCall call) {
       final Sort sort = call.rel(0);
       CassandraFilter filter = call.rel(2);
@@ -426,7 +424,6 @@ public class CassandraRules {
         convert(limit.getInput(), CassandraRel.CONVENTION), limit.offset, limit.fetch);
     }
 
-    /** @see org.apache.calcite.rel.convert.ConverterRule */
     @Override public void onMatch(RelOptRuleCall call) {
       final EnumerableLimit limit = call.rel(0);
       final RelNode converted = convert(limit);
diff --git a/cassandra/src/main/java/org/apache/calcite/adapter/cassandra/CassandraSchema.java b/cassandra/src/main/java/org/apache/calcite/adapter/cassandra/CassandraSchema.java
index d023bfa..0bcd4dc 100644
--- a/cassandra/src/main/java/org/apache/calcite/adapter/cassandra/CassandraSchema.java
+++ b/cassandra/src/main/java/org/apache/calcite/adapter/cassandra/CassandraSchema.java
@@ -65,7 +65,7 @@ import java.util.stream.Collectors;
 import java.util.stream.IntStream;
 
 /**
- * Schema mapped onto a Cassandra column family
+ * Schema mapped onto a Cassandra column family.
  */
 public class CassandraSchema extends AbstractSchema {
   final Session session;
@@ -243,7 +243,7 @@ public class CassandraSchema extends AbstractSchema {
   }
 
   /**
-   * Get all primary key columns from the underlying CQL table
+   * Returns all primary key columns from the underlying CQL table.
    *
    * @return A list of field names that are part of the partition and clustering keys
    */
@@ -305,8 +305,7 @@ public class CassandraSchema extends AbstractSchema {
     return keyCollations;
   }
 
-  /** Add all materialized views defined in the schema to this column family
-   */
+  /** Adds all materialized views defined in the schema to this column family. */
   private void addMaterializedViews() {
     // Close the hook use to get us here
     hook.close();
diff --git a/cassandra/src/main/java/org/apache/calcite/adapter/cassandra/CassandraSchemaFactory.java b/cassandra/src/main/java/org/apache/calcite/adapter/cassandra/CassandraSchemaFactory.java
index 87943a8..db64ac2 100644
--- a/cassandra/src/main/java/org/apache/calcite/adapter/cassandra/CassandraSchemaFactory.java
+++ b/cassandra/src/main/java/org/apache/calcite/adapter/cassandra/CassandraSchemaFactory.java
@@ -23,7 +23,7 @@ import org.apache.calcite.schema.SchemaPlus;
 import java.util.Map;
 
 /**
- * Factory that creates a {@link CassandraSchema}
+ * Factory that creates a {@link CassandraSchema}.
  */
 @SuppressWarnings("UnusedDeclaration")
 public class CassandraSchemaFactory implements SchemaFactory {
diff --git a/cassandra/src/main/java/org/apache/calcite/adapter/cassandra/CassandraTable.java b/cassandra/src/main/java/org/apache/calcite/adapter/cassandra/CassandraTable.java
index 8b4112a..ed65284 100644
--- a/cassandra/src/main/java/org/apache/calcite/adapter/cassandra/CassandraTable.java
+++ b/cassandra/src/main/java/org/apache/calcite/adapter/cassandra/CassandraTable.java
@@ -48,7 +48,7 @@ import java.util.List;
 import java.util.Map;
 
 /**
- * Table based on a Cassandra column family
+ * Table based on a Cassandra column family.
  */
 public class CassandraTable extends AbstractQueryableTable
     implements TranslatableTable {
diff --git a/cassandra/src/test/java/org/apache/calcite/test/CassandraExtension.java b/cassandra/src/test/java/org/apache/calcite/test/CassandraExtension.java
index 3ec105a..dc8b510 100644
--- a/cassandra/src/test/java/org/apache/calcite/test/CassandraExtension.java
+++ b/cassandra/src/test/java/org/apache/calcite/test/CassandraExtension.java
@@ -93,9 +93,8 @@ class CassandraExtension implements ParameterResolver, ExecutionCondition {
             .file().getAbsolutePath());
   }
 
-  /**
-   * Register cassandra resource in root context so it can be shared with other tests
-   */
+  /** Registers a Cassandra resource in root context so it can be shared with
+   * other tests. */
   private static CassandraResource getOrCreate(ExtensionContext context) {
     // same cassandra instance should be shared across all extension instances
     return context.getRoot()
@@ -128,7 +127,9 @@ class CassandraExtension implements ParameterResolver, ExecutionCondition {
     return ConditionEvaluationResult.disabled("Cassandra tests disabled");
   }
 
-  private static class CassandraResource implements ExtensionContext.Store.CloseableResource {
+  /** Cassandra resource. */
+  private static class CassandraResource
+      implements ExtensionContext.Store.CloseableResource {
     private final Session session;
     private final Cluster cluster;
 
diff --git a/core/src/main/java/org/apache/calcite/adapter/enumerable/EnumUtils.java b/core/src/main/java/org/apache/calcite/adapter/enumerable/EnumUtils.java
index 83e8089..846e523 100644
--- a/core/src/main/java/org/apache/calcite/adapter/enumerable/EnumUtils.java
+++ b/core/src/main/java/org/apache/calcite/adapter/enumerable/EnumUtils.java
@@ -593,7 +593,7 @@ public class EnumUtils {
   }
 
   /**
-   * Handle decimal type specifically with explicit type conversion
+   * Handles decimal type specifically with explicit type conversion.
    */
   private static Expression convertAssignableType(
       Expression argument, Type targetType) {
@@ -671,7 +671,8 @@ public class EnumUtils {
   }
 
   /**
-   * Match an argument expression to method parameter type with best effort
+   * Matches an argument expression to method parameter type with best effort.
+   *
    * @param argument Argument Expression
    * @param parameter Parameter type
    * @return Converted argument expression that matches the parameter type.
@@ -832,6 +833,7 @@ public class EnumUtils {
     };
   }
 
+  /** Enumerator that converts rows into sessions separated by gaps. */
   private static class SessionizationEnumerator implements Enumerator<Object[]> {
     private final Enumerator<Object[]> inputEnumerator;
     private final int indexOfWatermarkedColumn;
@@ -972,6 +974,7 @@ public class EnumUtils {
     };
   }
 
+  /** Enumerator that computes HOP. */
   private static class HopEnumerator implements Enumerator<Object[]> {
     private final Enumerator<Object[]> inputEnumerator;
     private final int indexOfWatermarkedColumn;
diff --git a/core/src/main/java/org/apache/calcite/adapter/enumerable/JavaRowFormat.java b/core/src/main/java/org/apache/calcite/adapter/enumerable/JavaRowFormat.java
index 00d354f..eb31923 100644
--- a/core/src/main/java/org/apache/calcite/adapter/enumerable/JavaRowFormat.java
+++ b/core/src/main/java/org/apache/calcite/adapter/enumerable/JavaRowFormat.java
@@ -189,7 +189,7 @@ public enum JavaRowFormat {
   },
 
   /**
-   * See {@link org.apache.calcite.interpreter.Row}
+   * See {@link org.apache.calcite.interpreter.Row}.
    */
   ROW {
     @Override Type javaRowClass(JavaTypeFactory typeFactory, RelDataType type) {
diff --git a/core/src/main/java/org/apache/calcite/adapter/enumerable/NestedBlockBuilder.java b/core/src/main/java/org/apache/calcite/adapter/enumerable/NestedBlockBuilder.java
index 9d8c960..d319251 100644
--- a/core/src/main/java/org/apache/calcite/adapter/enumerable/NestedBlockBuilder.java
+++ b/core/src/main/java/org/apache/calcite/adapter/enumerable/NestedBlockBuilder.java
@@ -41,8 +41,7 @@ public interface NestedBlockBuilder {
   void nestBlock(BlockBuilder block);
 
   /**
-   * Returns the current code block
-   * @return current code block
+   * Returns the current code block.
    */
   BlockBuilder currentBlock();
 
diff --git a/core/src/main/java/org/apache/calcite/adapter/enumerable/NestedBlockBuilderImpl.java b/core/src/main/java/org/apache/calcite/adapter/enumerable/NestedBlockBuilderImpl.java
index ea6c516..abc186e 100644
--- a/core/src/main/java/org/apache/calcite/adapter/enumerable/NestedBlockBuilderImpl.java
+++ b/core/src/main/java/org/apache/calcite/adapter/enumerable/NestedBlockBuilderImpl.java
@@ -60,8 +60,7 @@ public class NestedBlockBuilderImpl implements NestedBlockBuilder {
   }
 
   /**
-   * Returns the current code block
-   * @return current code block
+   * Returns the current code block.
    */
   public final BlockBuilder currentBlock() {
     return blocks.get(blocks.size() - 1);
diff --git a/core/src/main/java/org/apache/calcite/adapter/enumerable/ReflectiveCallNotNullImplementor.java b/core/src/main/java/org/apache/calcite/adapter/enumerable/ReflectiveCallNotNullImplementor.java
index d031983..1a97467 100644
--- a/core/src/main/java/org/apache/calcite/adapter/enumerable/ReflectiveCallNotNullImplementor.java
+++ b/core/src/main/java/org/apache/calcite/adapter/enumerable/ReflectiveCallNotNullImplementor.java
@@ -36,8 +36,9 @@ public class ReflectiveCallNotNullImplementor implements NotNullImplementor {
   protected final Method method;
 
   /**
-   * Constructor of {@link ReflectiveCallNotNullImplementor}
-   * @param method method that is used to implement the call
+   * Constructor of {@link ReflectiveCallNotNullImplementor}.
+   *
+   * @param method Method that is used to implement the call
    */
   public ReflectiveCallNotNullImplementor(Method method) {
     this.method = method;
diff --git a/core/src/main/java/org/apache/calcite/adapter/enumerable/RexToLixTranslator.java b/core/src/main/java/org/apache/calcite/adapter/enumerable/RexToLixTranslator.java
index 0e183b3..74318ea 100644
--- a/core/src/main/java/org/apache/calcite/adapter/enumerable/RexToLixTranslator.java
+++ b/core/src/main/java/org/apache/calcite/adapter/enumerable/RexToLixTranslator.java
@@ -1097,7 +1097,7 @@ public class RexToLixTranslator implements RexVisitor<RexToLixTranslator.Result>
 
   /**
    * For {@code PREV} operator, the offset of {@code inputGetter}
-   * should be set first
+   * should be set first.
    */
   private Result implementPrev(RexCall call) {
     final RexNode node = call.getOperands().get(0);
diff --git a/core/src/main/java/org/apache/calcite/adapter/enumerable/impl/AggResetContextImpl.java b/core/src/main/java/org/apache/calcite/adapter/enumerable/impl/AggResetContextImpl.java
index 7578a68..51e6f61 100644
--- a/core/src/main/java/org/apache/calcite/adapter/enumerable/impl/AggResetContextImpl.java
+++ b/core/src/main/java/org/apache/calcite/adapter/enumerable/impl/AggResetContextImpl.java
@@ -26,7 +26,7 @@ import java.util.List;
 
 /**
  * Implementation of
- * {@link org.apache.calcite.adapter.enumerable.AggResetContext}
+ * {@link org.apache.calcite.adapter.enumerable.AggResetContext}.
  */
 public abstract class AggResetContextImpl extends NestedBlockBuilderImpl
     implements AggResetContext {
diff --git a/core/src/main/java/org/apache/calcite/adapter/enumerable/impl/AggResultContextImpl.java b/core/src/main/java/org/apache/calcite/adapter/enumerable/impl/AggResultContextImpl.java
index db97675..ecf03b7 100644
--- a/core/src/main/java/org/apache/calcite/adapter/enumerable/impl/AggResultContextImpl.java
+++ b/core/src/main/java/org/apache/calcite/adapter/enumerable/impl/AggResultContextImpl.java
@@ -27,7 +27,7 @@ import java.util.List;
 
 /**
  * Implementation of
- * {@link org.apache.calcite.adapter.enumerable.AggResultContext}
+ * {@link org.apache.calcite.adapter.enumerable.AggResultContext}.
  */
 public class AggResultContextImpl extends AggResetContextImpl
     implements AggResultContext {
diff --git a/core/src/main/java/org/apache/calcite/adapter/java/ReflectiveSchema.java b/core/src/main/java/org/apache/calcite/adapter/java/ReflectiveSchema.java
index 86d151b..c2f23c0 100644
--- a/core/src/main/java/org/apache/calcite/adapter/java/ReflectiveSchema.java
+++ b/core/src/main/java/org/apache/calcite/adapter/java/ReflectiveSchema.java
@@ -196,7 +196,7 @@ public class ReflectiveSchema
   }
 
   /** Deduces the element type of a collection;
-   * same logic as {@link #toEnumerable} */
+   * same logic as {@link #toEnumerable}. */
   private static Type getElementType(Class clazz) {
     if (clazz.isArray()) {
       return clazz.getComponentType();
diff --git a/core/src/main/java/org/apache/calcite/adapter/jdbc/JdbcImplementor.java b/core/src/main/java/org/apache/calcite/adapter/jdbc/JdbcImplementor.java
index f0df676..eeebc0b 100644
--- a/core/src/main/java/org/apache/calcite/adapter/jdbc/JdbcImplementor.java
+++ b/core/src/main/java/org/apache/calcite/adapter/jdbc/JdbcImplementor.java
@@ -33,6 +33,7 @@ public class JdbcImplementor extends RelToSqlConverter {
     Util.discard(typeFactory);
   }
 
+  // CHECKSTYLE: IGNORE 1
   /** @see #dispatch */
   public Result visit(JdbcTableScan scan) {
     return result(scan.jdbcTable.tableName(),
diff --git a/core/src/main/java/org/apache/calcite/adapter/jdbc/JdbcRules.java b/core/src/main/java/org/apache/calcite/adapter/jdbc/JdbcRules.java
index f7f8ce8..5a46990 100644
--- a/core/src/main/java/org/apache/calcite/adapter/jdbc/JdbcRules.java
+++ b/core/src/main/java/org/apache/calcite/adapter/jdbc/JdbcRules.java
@@ -1088,10 +1088,10 @@ public class JdbcRules {
     }
   }
 
-  /**
-   * Visitor for checking whether part of projection is a user defined function or not
-   */
-  private static class CheckingUserDefinedFunctionVisitor extends RexVisitorImpl<Void> {
+  /** Visitor that checks whether part of a projection is a user-defined
+   * function (UDF). */
+  private static class CheckingUserDefinedFunctionVisitor
+      extends RexVisitorImpl<Void> {
 
     private boolean containsUsedDefinedFunction = false;
 
diff --git a/core/src/main/java/org/apache/calcite/adapter/jdbc/JdbcSchema.java b/core/src/main/java/org/apache/calcite/adapter/jdbc/JdbcSchema.java
index 1011d13..90ce722 100644
--- a/core/src/main/java/org/apache/calcite/adapter/jdbc/JdbcSchema.java
+++ b/core/src/main/java/org/apache/calcite/adapter/jdbc/JdbcSchema.java
@@ -561,7 +561,7 @@ public class JdbcSchema implements Schema {
     }
   }
 
-  /** Do not use */
+  /** Do not use. */
   @Experimental
   public interface Foo
       extends BiFunction<String, String, Iterable<MetaImpl.MetaTable>> {
diff --git a/core/src/main/java/org/apache/calcite/config/CalciteConnectionConfig.java b/core/src/main/java/org/apache/calcite/config/CalciteConnectionConfig.java
index 60dfde4..73f8c90 100644
--- a/core/src/main/java/org/apache/calcite/config/CalciteConnectionConfig.java
+++ b/core/src/main/java/org/apache/calcite/config/CalciteConnectionConfig.java
@@ -29,59 +29,70 @@ import java.util.Properties;
  * classes for system and statement properties. */
 public interface CalciteConnectionConfig extends ConnectionConfig {
   /** Default configuration. */
-  CalciteConnectionConfigImpl DEFAULT = new CalciteConnectionConfigImpl(new Properties());
-  /** @see CalciteConnectionProperty#APPROXIMATE_DISTINCT_COUNT */
+  CalciteConnectionConfigImpl DEFAULT =
+      new CalciteConnectionConfigImpl(new Properties());
+  /** Returns the value of
+   * {@link CalciteConnectionProperty#APPROXIMATE_DISTINCT_COUNT}. */
   boolean approximateDistinctCount();
-  /** @see CalciteConnectionProperty#APPROXIMATE_TOP_N */
+  /** Returns the value of
+   * {@link CalciteConnectionProperty#APPROXIMATE_TOP_N}. */
   boolean approximateTopN();
-  /** @see CalciteConnectionProperty#APPROXIMATE_DECIMAL */
+  /** Returns the value of
+   * {@link CalciteConnectionProperty#APPROXIMATE_DECIMAL}. */
   boolean approximateDecimal();
-  /** @see CalciteConnectionProperty#NULL_EQUAL_TO_EMPTY */
+  /** Returns the value of
+   * {@link CalciteConnectionProperty#NULL_EQUAL_TO_EMPTY}. */
   boolean nullEqualToEmpty();
-  /** @see CalciteConnectionProperty#AUTO_TEMP */
+  /** Returns the value of
+   * {@link CalciteConnectionProperty#AUTO_TEMP}. */
   boolean autoTemp();
-  /** @see CalciteConnectionProperty#MATERIALIZATIONS_ENABLED */
+  /** Returns the value of
+   * {@link CalciteConnectionProperty#MATERIALIZATIONS_ENABLED}. */
   boolean materializationsEnabled();
-  /** @see CalciteConnectionProperty#CREATE_MATERIALIZATIONS */
+  /** Returns the value of
+   * {@link CalciteConnectionProperty#CREATE_MATERIALIZATIONS}. */
   boolean createMaterializations();
-  /** @see CalciteConnectionProperty#DEFAULT_NULL_COLLATION */
+  /** Returns the value of
+   * {@link CalciteConnectionProperty#DEFAULT_NULL_COLLATION}. */
   NullCollation defaultNullCollation();
-  /** @see CalciteConnectionProperty#FUN */
+  /** Returns the value of {@link CalciteConnectionProperty#FUN}. */
   <T> T fun(Class<T> operatorTableClass, T defaultOperatorTable);
-  /** @see CalciteConnectionProperty#MODEL */
+  /** Returns the value of {@link CalciteConnectionProperty#MODEL}. */
   String model();
-  /** @see CalciteConnectionProperty#LEX */
+  /** Returns the value of {@link CalciteConnectionProperty#LEX}. */
   Lex lex();
-  /** @see CalciteConnectionProperty#QUOTING */
+  /** Returns the value of {@link CalciteConnectionProperty#QUOTING}. */
   Quoting quoting();
-  /** @see CalciteConnectionProperty#UNQUOTED_CASING */
+  /** Returns the value of {@link CalciteConnectionProperty#UNQUOTED_CASING}. */
   Casing unquotedCasing();
-  /** @see CalciteConnectionProperty#QUOTED_CASING */
+  /** Returns the value of {@link CalciteConnectionProperty#QUOTED_CASING}. */
   Casing quotedCasing();
-  /** @see CalciteConnectionProperty#CASE_SENSITIVE */
+  /** Returns the value of {@link CalciteConnectionProperty#CASE_SENSITIVE}. */
   boolean caseSensitive();
-  /** @see CalciteConnectionProperty#PARSER_FACTORY */
+  /** Returns the value of {@link CalciteConnectionProperty#PARSER_FACTORY}. */
   <T> T parserFactory(Class<T> parserFactoryClass, T defaultParserFactory);
-  /** @see CalciteConnectionProperty#SCHEMA_FACTORY */
+  /** Returns the value of {@link CalciteConnectionProperty#SCHEMA_FACTORY}. */
   <T> T schemaFactory(Class<T> schemaFactoryClass, T defaultSchemaFactory);
-  /** @see CalciteConnectionProperty#SCHEMA_TYPE */
+  /** Returns the value of {@link CalciteConnectionProperty#SCHEMA_TYPE}. */
   JsonSchema.Type schemaType();
-  /** @see CalciteConnectionProperty#SPARK */
+  /** Returns the value of {@link CalciteConnectionProperty#SPARK}. */
   boolean spark();
-  /** @see CalciteConnectionProperty#FORCE_DECORRELATE */
+  /** Returns the value of
+   * {@link CalciteConnectionProperty#FORCE_DECORRELATE}. */
   boolean forceDecorrelate();
-  /** @see CalciteConnectionProperty#TYPE_SYSTEM */
+  /** Returns the value of {@link CalciteConnectionProperty#TYPE_SYSTEM}. */
   <T> T typeSystem(Class<T> typeSystemClass, T defaultTypeSystem);
-  /** @see CalciteConnectionProperty#CONFORMANCE */
+  /** Returns the value of {@link CalciteConnectionProperty#CONFORMANCE}. */
   SqlConformance conformance();
-  /** @see CalciteConnectionProperty#TIME_ZONE */
+  /** Returns the value of {@link CalciteConnectionProperty#TIME_ZONE}. */
   @Override String timeZone();
-  /** @see CalciteConnectionProperty#LOCALE */
+  /** Returns the value of {@link CalciteConnectionProperty#LOCALE}. */
   String locale();
-  /** @see CalciteConnectionProperty#TYPE_COERCION */
+  /** Returns the value of {@link CalciteConnectionProperty#TYPE_COERCION}. */
   boolean typeCoercion();
-  /** @see CalciteConnectionProperty#LENIENT_OPERATOR_LOOKUP */
+  /** Returns the value of
+   * {@link CalciteConnectionProperty#LENIENT_OPERATOR_LOOKUP}. */
   boolean lenientOperatorLookup();
-  /** @see CalciteConnectionProperty#TOPDOWN_OPT */
+  /** Returns the value of {@link CalciteConnectionProperty#TOPDOWN_OPT}. */
   boolean topDownOpt();
 }
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 d15952b..48be219 100644
--- a/core/src/main/java/org/apache/calcite/config/CalciteSystemProperty.java
+++ b/core/src/main/java/org/apache/calcite/config/CalciteSystemProperty.java
@@ -357,8 +357,9 @@ public final class CalciteSystemProperty<T> {
   }
 
   /**
-   * Returns the value of the system property with the specified name as int, or
-   * the <code>defaultValue</code> if any of the conditions below hold:
+   * Returns the value of the system property with the specified name as {@code
+   * int}. If any of the conditions below hold, returns the
+   * <code>defaultValue</code>:
    *
    * <ol>
    * <li>the property is not defined;
diff --git a/core/src/main/java/org/apache/calcite/interpreter/Bindables.java b/core/src/main/java/org/apache/calcite/interpreter/Bindables.java
index 7d9adac..d75de31 100644
--- a/core/src/main/java/org/apache/calcite/interpreter/Bindables.java
+++ b/core/src/main/java/org/apache/calcite/interpreter/Bindables.java
@@ -109,6 +109,7 @@ public class Bindables {
   public static final RelOptRule BINDABLE_SET_OP_RULE =
       BindableSetOpRule.DEFAULT_CONFIG.toRule(BindableSetOpRule.class);
 
+  // CHECKSTYLE: IGNORE 1
   /** @deprecated Use {@link #BINDABLE_SET_OP_RULE}. */
   public static final RelOptRule BINDABLE_SETOP_RULE =
       BINDABLE_SET_OP_RULE;
diff --git a/core/src/main/java/org/apache/calcite/interpreter/Row.java b/core/src/main/java/org/apache/calcite/interpreter/Row.java
index 1a16f81..365d5f7 100644
--- a/core/src/main/java/org/apache/calcite/interpreter/Row.java
+++ b/core/src/main/java/org/apache/calcite/interpreter/Row.java
@@ -113,7 +113,8 @@ public class Row {
     }
 
     /**
-     * Set the value of a particular column.
+     * Sets the value of a particular column.
+     *
      * @param index Zero-indexed position of value.
      * @param value Desired column value.
      */
@@ -121,7 +122,7 @@ public class Row {
       values[index] = value;
     }
 
-    /** Return a Row object **/
+    /** Returns a Row. */
     public Row build() {
       return new Row(values);
     }
diff --git a/core/src/main/java/org/apache/calcite/model/ModelHandler.java b/core/src/main/java/org/apache/calcite/model/ModelHandler.java
index 2aba4aa..edc367c 100644
--- a/core/src/main/java/org/apache/calcite/model/ModelHandler.java
+++ b/core/src/main/java/org/apache/calcite/model/ModelHandler.java
@@ -100,6 +100,7 @@ public class ModelHandler {
     visit(root);
   }
 
+  // CHECKSTYLE: IGNORE 1
   /** @deprecated Use {@link #addFunctions}. */
   @Deprecated
   public static void create(SchemaPlus schema, String functionName,
diff --git a/core/src/main/java/org/apache/calcite/plan/AbstractRelOptPlanner.java b/core/src/main/java/org/apache/calcite/plan/AbstractRelOptPlanner.java
index 08731a8..b842842 100644
--- a/core/src/main/java/org/apache/calcite/plan/AbstractRelOptPlanner.java
+++ b/core/src/main/java/org/apache/calcite/plan/AbstractRelOptPlanner.java
@@ -169,7 +169,7 @@ public abstract class AbstractRelOptPlanner implements RelOptPlanner {
   }
 
   /**
-   * Returns the rule with a given description
+   * Returns the rule with a given description.
    *
    * @param description Description
    * @return Rule with given description, or null if not found
@@ -411,12 +411,11 @@ public abstract class AbstractRelOptPlanner implements RelOptPlanner {
   }
 
   /**
-   * Takes care of tracing and listener notification when a rel is discarded
+   * Takes care of tracing and listener notification when a rel is discarded.
    *
-   * @param rel discarded rel
+   * @param rel Discarded rel
    */
-  protected void notifyDiscard(
-      RelNode rel) {
+  protected void notifyDiscard(RelNode rel) {
     if (listener != null) {
       RelOptListener.RelDiscardedEvent event =
           new RelOptListener.RelDiscardedEvent(
diff --git a/core/src/main/java/org/apache/calcite/plan/Convention.java b/core/src/main/java/org/apache/calcite/plan/Convention.java
index ac30507..61cf9f1 100644
--- a/core/src/main/java/org/apache/calcite/plan/Convention.java
+++ b/core/src/main/java/org/apache/calcite/plan/Convention.java
@@ -82,9 +82,8 @@ public interface Convention extends RelTrait {
     return false;
   }
 
-  /**
-   * Return RelFactories struct for the convention which can be used to build RelNode
-   */
+  /** Return RelFactories struct for this convention. It can can be used to
+   * build RelNode. */
   default RelFactories.Struct getRelFactories() {
     return RelFactories.DEFAULT_STRUCT;
   }
diff --git a/core/src/main/java/org/apache/calcite/plan/RelOptCluster.java b/core/src/main/java/org/apache/calcite/plan/RelOptCluster.java
index 43379b3..3ce33f3 100644
--- a/core/src/main/java/org/apache/calcite/plan/RelOptCluster.java
+++ b/core/src/main/java/org/apache/calcite/plan/RelOptCluster.java
@@ -235,6 +235,7 @@ public class RelOptCluster {
     return emptyTraitSet;
   }
 
+  // CHECKSTYLE: IGNORE 2
   /** @deprecated For {@code traitSetOf(t1, t2)},
    * use {@link #traitSet}().replace(t1).replace(t2). */
   @Deprecated // to be removed before 2.0
diff --git a/core/src/main/java/org/apache/calcite/plan/RelOptCost.java b/core/src/main/java/org/apache/calcite/plan/RelOptCost.java
index aae811d..331d2ab 100644
--- a/core/src/main/java/org/apache/calcite/plan/RelOptCost.java
+++ b/core/src/main/java/org/apache/calcite/plan/RelOptCost.java
@@ -29,29 +29,21 @@ package org.apache.calcite.plan;
 public interface RelOptCost {
   //~ Methods ----------------------------------------------------------------
 
-  /**
-   * @return number of rows processed; this should not be confused with the
-   * row count produced by a relational expression
-   * ({@link org.apache.calcite.rel.RelNode#estimateRowCount})
-   */
+  /** Returns the number of rows processed; this should not be
+   * confused with the row count produced by a relational expression
+   * ({@link org.apache.calcite.rel.RelNode#estimateRowCount}). */
   double getRows();
 
-  /**
-   * @return usage of CPU resources
-   */
+  /** Returns usage of CPU resources. */
   double getCpu();
 
-  /**
-   * @return usage of I/O resources
-   */
+  /** Returns usage of I/O resources. */
   double getIo();
 
-  /**
-   * @return true iff this cost represents an expression that hasn't actually
+  /** Returns whether this cost represents an expression that hasn't actually
    * been implemented (e.g. a pure relational algebra expression) or can't
    * actually be implemented, e.g. a transfer of data between two disconnected
-   * sites
-   */
+   * sites. */
   boolean isInfinite();
 
   // REVIEW jvs 3-Apr-2006:  we should standardize this
diff --git a/core/src/main/java/org/apache/calcite/plan/RelOptPlanner.java b/core/src/main/java/org/apache/calcite/plan/RelOptPlanner.java
index e2117b5..2114f73 100644
--- a/core/src/main/java/org/apache/calcite/plan/RelOptPlanner.java
+++ b/core/src/main/java/org/apache/calcite/plan/RelOptPlanner.java
@@ -212,10 +212,9 @@ public interface RelOptPlanner {
    */
   RelOptCost getCost(RelNode rel, RelMetadataQuery mq);
 
-  /**
-   * @deprecated Use {@link #getCost(RelNode, RelMetadataQuery)}
-   * or, better, call {@link RelMetadataQuery#getCumulativeCost(RelNode)}.
-   */
+  // CHECKSTYLE: IGNORE 2
+  /** @deprecated Use {@link #getCost(RelNode, RelMetadataQuery)}
+   * or, better, call {@link RelMetadataQuery#getCumulativeCost(RelNode)}. */
   @Deprecated // to be removed before 2.0
   RelOptCost getCost(RelNode rel);
 
@@ -334,6 +333,7 @@ public interface RelOptPlanner {
   /** Called when a relational expression is copied to a similar expression. */
   void onCopy(RelNode rel, RelNode newRel);
 
+  // CHECKSTYLE: IGNORE 1
   /** @deprecated Use {@link RexExecutor} */
   @Deprecated // to be removed before 2.0
   interface Executor extends RexExecutor {
diff --git a/core/src/main/java/org/apache/calcite/plan/RelOptRule.java b/core/src/main/java/org/apache/calcite/plan/RelOptRule.java
index 6eea4b8..cfd478b 100644
--- a/core/src/main/java/org/apache/calcite/plan/RelOptRule.java
+++ b/core/src/main/java/org/apache/calcite/plan/RelOptRule.java
@@ -178,6 +178,7 @@ public abstract class RelOptRule {
         operandList.operands);
   }
 
+  // CHECKSTYLE: IGNORE 1
   /** @deprecated Use {@link #operandJ} */
   @SuppressWarnings("Guava")
   @Deprecated // to be removed before 2.0
@@ -269,6 +270,7 @@ public abstract class RelOptRule {
     return new ConverterRelOptRuleOperand(clazz, trait, predicate);
   }
 
+  // CHECKSTYLE: IGNORE 1
   /** @deprecated Use {@link #convertOperand(Class, Predicate, RelTrait)}. */
   @SuppressWarnings("Guava")
   @Deprecated // to be removed before 2.0
@@ -435,7 +437,7 @@ public abstract class RelOptRule {
   }
 
   /**
-   * Returns the root operand of this rule
+   * Returns the root operand of this rule.
    *
    * @return the root operand of this rule
    */
diff --git a/core/src/main/java/org/apache/calcite/plan/RelOptRuleCall.java b/core/src/main/java/org/apache/calcite/plan/RelOptRuleCall.java
index fd5f50f..c0dfdfa 100644
--- a/core/src/main/java/org/apache/calcite/plan/RelOptRuleCall.java
+++ b/core/src/main/java/org/apache/calcite/plan/RelOptRuleCall.java
@@ -219,7 +219,7 @@ public abstract class RelOptRuleCall {
   }
 
   /**
-   * @return list of parents of the first relational expression
+   * Returns a list of parents of the first relational expression.
    */
   public List<RelNode> getParents() {
     return parents;
diff --git a/core/src/main/java/org/apache/calcite/plan/RelOptRuleOperand.java b/core/src/main/java/org/apache/calcite/plan/RelOptRuleOperand.java
index 913db6f..75dfc51 100644
--- a/core/src/main/java/org/apache/calcite/plan/RelOptRuleOperand.java
+++ b/core/src/main/java/org/apache/calcite/plan/RelOptRuleOperand.java
@@ -158,7 +158,7 @@ public class RelOptRuleOperand {
   }
 
   /**
-   * Sets the rule this operand belongs to
+   * Sets the rule this operand belongs to.
    *
    * @param rule containing rule
    */
@@ -249,7 +249,7 @@ public class RelOptRuleOperand {
   }
 
   /**
-   * @return relational expression class matched by this operand
+   * Returns relational expression class matched by this operand.
    */
   public Class<? extends RelNode> getMatchedClass() {
     return clazz;
diff --git a/core/src/main/java/org/apache/calcite/plan/RelOptRules.java b/core/src/main/java/org/apache/calcite/plan/RelOptRules.java
index 74b34a6..5e33487 100644
--- a/core/src/main/java/org/apache/calcite/plan/RelOptRules.java
+++ b/core/src/main/java/org/apache/calcite/plan/RelOptRules.java
@@ -44,9 +44,8 @@ public class RelOptRules {
   private RelOptRules() {
   }
 
-  /**
-   * The calc rule set is public for use from {@link org.apache.calcite.tools.Programs}
-   */
+  /** Calc rule set; public so that {@link org.apache.calcite.tools.Programs} can
+   * use it. */
   public static final ImmutableList<RelOptRule> CALC_RULES =
       ImmutableList.of(
           Bindables.FROM_NONE_RULE,
diff --git a/core/src/main/java/org/apache/calcite/plan/RelOptTable.java b/core/src/main/java/org/apache/calcite/plan/RelOptTable.java
index 521765b..e80bf40 100644
--- a/core/src/main/java/org/apache/calcite/plan/RelOptTable.java
+++ b/core/src/main/java/org/apache/calcite/plan/RelOptTable.java
@@ -102,7 +102,7 @@ public interface RelOptTable extends Wrapper {
 
   /**
    * Returns a list of unique keys, empty list if no key exist,
-   * the result should be consistent with {@code isKey}
+   * the result should be consistent with {@code isKey}.
    */
   List<ImmutableBitSet> getKeys();
 
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 13958ef..8e30ee5 100644
--- a/core/src/main/java/org/apache/calcite/plan/RelOptUtil.java
+++ b/core/src/main/java/org/apache/calcite/plan/RelOptUtil.java
@@ -188,14 +188,14 @@ public abstract class RelOptUtil {
   }
 
   /**
-   * Returns a set of tables used by this expression or its children
+   * Returns a set of tables used by this expression or its children.
    */
   public static Set<RelOptTable> findTables(RelNode rel) {
     return new LinkedHashSet<>(findAllTables(rel));
   }
 
   /**
-   * Returns a list of all tables used by this expression or its children
+   * Returns a list of all tables used by this expression or its children.
    */
   public static List<RelOptTable> findAllTables(RelNode rel) {
     final Multimap<Class<? extends RelNode>, RelNode> nodes =
@@ -929,6 +929,7 @@ public abstract class RelOptUtil {
         null, aggCalls);
   }
 
+  // CHECKSTYLE: IGNORE 1
   /** @deprecated Use {@link RelBuilder#distinct()}. */
   @Deprecated // to be removed before 2.0
   public static RelNode createDistinctRel(RelNode rel) {
@@ -2831,7 +2832,7 @@ public abstract class RelOptUtil {
 
   /**
    * Splits a filter into two lists, depending on whether or not the filter
-   * only references its child input
+   * only references its child input.
    *
    * @param childBitmap Fields in the child
    * @param predicate   filters that will be split
@@ -3235,6 +3236,7 @@ public abstract class RelOptUtil {
         .build();
   }
 
+  // CHECKSTYLE: IGNORE 1
   /** @deprecated Use
    * {@link RelBuilder#projectNamed(Iterable, Iterable, boolean)} */
   @Deprecated // to be removed before 2.0
@@ -4074,12 +4076,11 @@ public abstract class RelOptUtil {
    *
    * <ul>
    *   <li>Project: remove the hints that have non-empty inherit path
-   *   (which means the hint was not originally declared from it); </li>
-   *   <li>Aggregate: remove the hints that have non-empty inherit path;</>
-   *   <li>Join: remove all the hints;</li>
-   *   <li>TableScan: remove the hints that have non-empty inherit path.</li>
+   *   (which means the hint was not originally declared from it);
+   *   <li>Aggregate: remove the hints that have non-empty inherit path;
+   *   <li>Join: remove all the hints;
+   *   <li>TableScan: remove the hints that have non-empty inherit path.
    * </ul>
-   *
    */
   private static class ResetHintsShuttle extends RelHomogeneousShuttle {
     public RelNode visit(RelNode node) {
@@ -4310,6 +4311,8 @@ public abstract class RelOptUtil {
     private final int[] adjustments;
 
     /**
+     * Creates a RexInputConverter.
+     *
      * @param rexBuilder      builder for creating new RexInputRefs
      * @param srcFields       fields where the RexInputRefs originated
      *                        from; if null, a new RexInputRef is always
@@ -4455,7 +4458,7 @@ public abstract class RelOptUtil {
   }
 
   /** Result of calling
-   * {@link org.apache.calcite.plan.RelOptUtil#createExistsPlan} */
+   * {@link org.apache.calcite.plan.RelOptUtil#createExistsPlan}. */
   public static class Exists {
     public final RelNode r;
     public final boolean indicator;
diff --git a/core/src/main/java/org/apache/calcite/plan/RelTraitDef.java b/core/src/main/java/org/apache/calcite/plan/RelTraitDef.java
index 0bba404..c9d99e1 100644
--- a/core/src/main/java/org/apache/calcite/plan/RelTraitDef.java
+++ b/core/src/main/java/org/apache/calcite/plan/RelTraitDef.java
@@ -76,15 +76,11 @@ public abstract class RelTraitDef<T extends RelTrait> {
     return false;
   }
 
-  /**
-   * @return the specific RelTrait type associated with this RelTraitDef.
-   */
+  /** Returns the specific RelTrait type associated with this RelTraitDef. */
   public abstract Class<T> getTraitClass();
 
-  /**
-   * @return a simple name for this RelTraitDef (for use in
-   * {@link org.apache.calcite.rel.RelNode#explain}).
-   */
+  /** Returns a simple name for this RelTraitDef (for use in
+   * {@link org.apache.calcite.rel.RelNode#explain}). */
   public abstract String getSimpleName();
 
   /**
diff --git a/core/src/main/java/org/apache/calcite/plan/RelTraitSet.java b/core/src/main/java/org/apache/calcite/plan/RelTraitSet.java
index 3902b58..e6737dc 100644
--- a/core/src/main/java/org/apache/calcite/plan/RelTraitSet.java
+++ b/core/src/main/java/org/apache/calcite/plan/RelTraitSet.java
@@ -42,7 +42,7 @@ public final class RelTraitSet extends AbstractList<RelTrait> {
   private final Cache cache;
   private final RelTrait[] traits;
   private String string;
-  /** Cache the hash code for the traits */
+  /** Caches the hash code for the traits. */
   private int hash; // Default to 0
 
   //~ Constructors -----------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/plan/TableAccessMap.java b/core/src/main/java/org/apache/calcite/plan/TableAccessMap.java
index 30ca6db..b5810a2 100644
--- a/core/src/main/java/org/apache/calcite/plan/TableAccessMap.java
+++ b/core/src/main/java/org/apache/calcite/plan/TableAccessMap.java
@@ -91,7 +91,7 @@ public class TableAccessMap {
   }
 
   /**
-   * Constructs a TableAccessMap for a single table
+   * Constructs a TableAccessMap for a single table.
    *
    * @param table fully qualified name of the table, represented as a list
    * @param mode  access mode for the table
@@ -104,7 +104,7 @@ public class TableAccessMap {
   //~ Methods ----------------------------------------------------------------
 
   /**
-   * @return set of qualified names for all tables accessed
+   * Returns a set of qualified names for all tables accessed.
    */
   public Set<List<String>> getTablesAccessed() {
     return accessMap.keySet();
diff --git a/core/src/main/java/org/apache/calcite/plan/VisitorDataContext.java b/core/src/main/java/org/apache/calcite/plan/VisitorDataContext.java
index 675ce3a..eae7e27 100644
--- a/core/src/main/java/org/apache/calcite/plan/VisitorDataContext.java
+++ b/core/src/main/java/org/apache/calcite/plan/VisitorDataContext.java
@@ -38,7 +38,7 @@ import java.math.BigDecimal;
 import java.util.List;
 
 /**
- * DataContext for evaluating an RexExpression
+ * DataContext for evaluating a RexExpression.
  */
 public class VisitorDataContext implements DataContext {
   private static final CalciteLogger LOGGER =
diff --git a/core/src/main/java/org/apache/calcite/plan/hep/HepRelVertex.java b/core/src/main/java/org/apache/calcite/plan/hep/HepRelVertex.java
index ae613f4..add516a 100644
--- a/core/src/main/java/org/apache/calcite/plan/hep/HepRelVertex.java
+++ b/core/src/main/java/org/apache/calcite/plan/hep/HepRelVertex.java
@@ -85,7 +85,7 @@ public class HepRelVertex extends AbstractRelNode {
   }
 
   /**
-   * @return current implementation chosen for this vertex
+   * Returns current implementation chosen for this vertex.
    */
   public RelNode getCurrentRel() {
     return currentRel;
diff --git a/core/src/main/java/org/apache/calcite/plan/volcano/IterativeRuleQueue.java b/core/src/main/java/org/apache/calcite/plan/volcano/IterativeRuleQueue.java
index fed36ca..62394cc 100644
--- a/core/src/main/java/org/apache/calcite/plan/volcano/IterativeRuleQueue.java
+++ b/core/src/main/java/org/apache/calcite/plan/volcano/IterativeRuleQueue.java
@@ -170,7 +170,7 @@ class IterativeRuleQueue extends RuleQueue {
   private static class MatchList {
 
     /**
-     * Rule match queue for SubstitutionRule
+     * Rule match queue for SubstitutionRule.
      */
     private final Queue<VolcanoRuleMatch> preQueue = new LinkedList<>();
 
diff --git a/core/src/main/java/org/apache/calcite/plan/volcano/RelSet.java b/core/src/main/java/org/apache/calcite/plan/volcano/RelSet.java
index bf4fe5d..09cf0e9 100644
--- a/core/src/main/java/org/apache/calcite/plan/volcano/RelSet.java
+++ b/core/src/main/java/org/apache/calcite/plan/volcano/RelSet.java
@@ -124,7 +124,7 @@ class RelSet {
   }
 
   /**
-   * Returns the child Relset for current set
+   * Returns the child RelSet for the current set.
    */
   public Set<RelSet> getChildSets(VolcanoPlanner planner) {
     Set<RelSet> childSets = new HashSet<>();
@@ -143,8 +143,8 @@ class RelSet {
   }
 
   /**
-   * @return all of the {@link RelNode}s contained by any subset of this set
-   * (does not include the subset objects themselves)
+   * Returns all of the {@link RelNode}s contained by any subset of this set
+   * (does not include the subset objects themselves).
    */
   public List<RelNode> getRelsFromAllSubsets() {
     return rels;
diff --git a/core/src/main/java/org/apache/calcite/plan/volcano/RelSubset.java b/core/src/main/java/org/apache/calcite/plan/volcano/RelSubset.java
index f2b9d7a..dccb950 100644
--- a/core/src/main/java/org/apache/calcite/plan/volcano/RelSubset.java
+++ b/core/src/main/java/org/apache/calcite/plan/volcano/RelSubset.java
@@ -84,37 +84,30 @@ public class RelSubset extends AbstractRelNode {
 
   //~ Instance fields --------------------------------------------------------
 
-  /**
-   * Optimization task state
-   */
+  /** Optimization task state. */
   OptimizeState taskState;
 
-  /**
-   * cost of best known plan (it may have improved since)
-   */
+  /** Cost of best known plan (it may have improved since). */
   RelOptCost bestCost;
 
-  /**
-   * The set this subset belongs to.
-   */
+  /** The set this subset belongs to. */
   final RelSet set;
 
-  /**
-   * best known plan
-   */
+  /** Best known plan. */
   RelNode best;
 
-  /**
-   * Timestamp for metadata validity
-   */
+  /** Timestamp for metadata validity. */
   long timestamp;
 
   /**
-   * Physical property state of current subset
-   * 0: logical operators, NONE convention is neither DELIVERED nor REQUIRED
-   * 1: traitSet DELIVERED from child operators or itself
-   * 2: traitSet REQUIRED from parent operators
-   * 3: both DELIVERED and REQUIRED
+   * Physical property state of current subset. Values:
+   *
+   * <ul>
+   * <li>0: logical operators, NONE convention is neither DELIVERED nor REQUIRED
+   * <li>1: traitSet DELIVERED from child operators or itself
+   * <li>2: traitSet REQUIRED from parent operators
+   * <li>3: both DELIVERED and REQUIRED
+   * </ul>
    */
   private int state = 0;
 
@@ -781,6 +774,7 @@ public class RelSubset extends AbstractRelNode {
     }
   }
 
+  /** State of optimizer. */
   enum OptimizeState {
     OPTIMIZING,
     COMPLETED
diff --git a/core/src/main/java/org/apache/calcite/plan/volcano/TopDownRuleDriver.java b/core/src/main/java/org/apache/calcite/plan/volcano/TopDownRuleDriver.java
index f67f133..d41c624 100644
--- a/core/src/main/java/org/apache/calcite/plan/volcano/TopDownRuleDriver.java
+++ b/core/src/main/java/org/apache/calcite/plan/volcano/TopDownRuleDriver.java
@@ -130,6 +130,7 @@ class TopDownRuleDriver implements RuleDriver {
     applying = null;
   }
 
+  /** Procedure. */
   private interface Procedure {
     void exec();
   }
@@ -280,6 +281,7 @@ class TopDownRuleDriver implements RuleDriver {
     }
   }
 
+  /** Task for generator. */
   private interface GeneratorTask extends Task {
     RelSubset group();
     boolean exploring();
@@ -435,7 +437,7 @@ class TopDownRuleDriver implements RuleDriver {
   }
 
   /**
-   * Explore an input for a RelNode
+   * Explore an input for a RelNode.
    */
   private class ExploreInput implements Task {
     private final RelSubset group;
@@ -496,7 +498,7 @@ class TopDownRuleDriver implements RuleDriver {
   }
 
   /**
-   * Apply a rule match
+   * Apply a rule match.
    */
   private class ApplyRule implements GeneratorTask {
     private final VolcanoRuleMatch match;
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 6014c79..fa26892 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
@@ -945,7 +945,7 @@ public class VolcanoPlanner extends AbstractRelOptPlanner {
   }
 
   /**
-   * Find the new root subset in case the root is merged with another subset
+   * Find the new root subset in case the root is merged with another subset.
    */
   void canonize() {
     root = canonize(root);
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 c9c1d7e..ba3426a 100644
--- a/core/src/main/java/org/apache/calcite/prepare/PlannerImpl.java
+++ b/core/src/main/java/org/apache/calcite/prepare/PlannerImpl.java
@@ -118,7 +118,7 @@ public class PlannerImpl implements Planner, ViewExpander {
     reset();
   }
 
-  /** Gets a user defined config and appends default connection values */
+  /** Gets a user-defined config and appends default connection values. */
   private CalciteConnectionConfig connConfig() {
     CalciteConnectionConfigImpl config = Util.first(
         context.unwrap(CalciteConnectionConfigImpl.class),
@@ -256,9 +256,10 @@ public class PlannerImpl implements Planner, ViewExpander {
     return root;
   }
 
+  // CHECKSTYLE: IGNORE 2
   /** @deprecated Now {@link PlannerImpl} implements {@link ViewExpander}
    * directly. */
-  @Deprecated
+  @Deprecated // to be removed before 2.0
   public class ViewExpanderImpl implements ViewExpander {
     ViewExpanderImpl() {
     }
diff --git a/core/src/main/java/org/apache/calcite/rel/AbstractRelNode.java b/core/src/main/java/org/apache/calcite/rel/AbstractRelNode.java
index 9f104a1..206503d 100644
--- a/core/src/main/java/org/apache/calcite/rel/AbstractRelNode.java
+++ b/core/src/main/java/org/apache/calcite/rel/AbstractRelNode.java
@@ -76,14 +76,10 @@ public abstract class AbstractRelNode implements RelNode {
 
   private final RelOptCluster cluster;
 
-  /**
-   * unique id of this object -- for debugging
-   */
+  /** Unique id of this object, for debugging. */
   protected final int id;
 
-  /**
-   * The RelTraitSet that describes the traits of this RelNode.
-   */
+  /** RelTraitSet that describes the traits of this RelNode. */
   protected RelTraitSet traitSet;
 
   //~ Constructors -----------------------------------------------------------
@@ -296,12 +292,11 @@ public abstract class AbstractRelNode implements RelNode {
     throw new UnsupportedOperationException("replaceInput called on " + this);
   }
 
-  /** Description, consists of id plus digest */
+  /** Description; consists of id plus digest. */
   public String toString() {
     return "rel#" + id + ':' + getDigest();
   }
 
-  /** Description, consists of id plus digest */
   @Deprecated // to be removed before 2.0
   public final String getDescription() {
     return this.toString();
@@ -419,10 +414,9 @@ public abstract class AbstractRelNode implements RelNode {
     return rdw.attrs;
   }
 
+  /** Implementation of {@link RelDigest}. */
   private class InnerRelDigest implements RelDigest {
-    /**
-     * Cache of hash code.
-     */
+    /** Cached hash code. */
     private int hash = 0;
 
     @Override public RelNode getRel() {
diff --git a/core/src/main/java/org/apache/calcite/rel/RelNode.java b/core/src/main/java/org/apache/calcite/rel/RelNode.java
index 3a25bde..54aae1f 100644
--- a/core/src/main/java/org/apache/calcite/rel/RelNode.java
+++ b/core/src/main/java/org/apache/calcite/rel/RelNode.java
@@ -233,19 +233,23 @@ public interface RelNode extends RelOptNode, Cloneable {
   RelNode onRegister(RelOptPlanner planner);
 
   /**
-   * Returns digest string of this {@code RelNode}. It will create new digest
-   * string on each call, so don't forget to cache the result if necessary.
+   * Returns a digest string of this {@code RelNode}.
+   *
+   * <p>Each call creates a new digest string,
+   * so don't forget to cache the result if necessary.
    *
    * @return Digest string of this {@code RelNode}
+   *
+   * @see #getRelDigest()
    */
   default String getDigest() {
     return getRelDigest().toString();
   }
 
   /**
-   * Digest of the {@code RelNode}, for planner internal use only.
+   * Returns a digest of this {@code RelNode}.
    *
-   * <p>INTERNAL USE ONLY.</p>
+   * <p>INTERNAL USE ONLY. For use by the planner.
    *
    * @return Digest of this {@code RelNode}
    * @see #getDigest()
@@ -254,7 +258,9 @@ public interface RelNode extends RelOptNode, Cloneable {
   RelDigest getRelDigest();
 
   /**
-   * Recomputes the digest. For planner internal use only.
+   * Recomputes the digest.
+   *
+   * <p>INTERNAL USE ONLY. For use by the planner.
    *
    * @see #getDigest()
    */
diff --git a/core/src/main/java/org/apache/calcite/rel/RelWriter.java b/core/src/main/java/org/apache/calcite/rel/RelWriter.java
index 92f78b6..f6be869 100644
--- a/core/src/main/java/org/apache/calcite/rel/RelWriter.java
+++ b/core/src/main/java/org/apache/calcite/rel/RelWriter.java
@@ -42,9 +42,7 @@ public interface RelWriter {
    */
   void explain(RelNode rel, List<Pair<String, Object>> valueList);
 
-  /**
-   * @return detail level at which plan should be generated
-   */
+  /** Returns detail level at which plan should be generated. */
   SqlExplainLevel getDetailLevel();
 
   /**
diff --git a/core/src/main/java/org/apache/calcite/rel/convert/Converter.java b/core/src/main/java/org/apache/calcite/rel/convert/Converter.java
index 13cc3c0..2639f21 100644
--- a/core/src/main/java/org/apache/calcite/rel/convert/Converter.java
+++ b/core/src/main/java/org/apache/calcite/rel/convert/Converter.java
@@ -67,7 +67,7 @@ public interface Converter extends RelNode {
   RelTraitDef getTraitDef();
 
   /**
-   * Returns the sole input relational expression
+   * Returns the sole input relational expression.
    *
    * @return child relational expression
    */
diff --git a/core/src/main/java/org/apache/calcite/rel/core/Aggregate.java b/core/src/main/java/org/apache/calcite/rel/core/Aggregate.java
index 82de9fc..e8a4b73 100644
--- a/core/src/main/java/org/apache/calcite/rel/core/Aggregate.java
+++ b/core/src/main/java/org/apache/calcite/rel/core/Aggregate.java
@@ -481,7 +481,7 @@ public abstract class Aggregate extends SingleRel implements Hintable {
     return Group.induce(groupSet, groupSets);
   }
 
-  /** What kind of roll-up is it? */
+  /** Describes the kind of roll-up. */
   public enum Group {
     SIMPLE,
     ROLLUP,
@@ -577,7 +577,7 @@ public abstract class Aggregate extends SingleRel implements Hintable {
     private final boolean filter;
 
     /**
-     * Creates an AggCallBinding
+     * Creates an AggCallBinding.
      *
      * @param typeFactory  Type factory
      * @param aggFunction  Aggregate function
diff --git a/core/src/main/java/org/apache/calcite/rel/core/CorrelationId.java b/core/src/main/java/org/apache/calcite/rel/core/CorrelationId.java
index 0c23729..11be21b 100644
--- a/core/src/main/java/org/apache/calcite/rel/core/CorrelationId.java
+++ b/core/src/main/java/org/apache/calcite/rel/core/CorrelationId.java
@@ -22,7 +22,7 @@ import java.util.Set;
 
 /**
  * Describes the necessary parameters for an implementation in order to
- * identify and set dynamic variables
+ * identify and set dynamic variables.
  */
 public class CorrelationId implements Cloneable, Comparable<CorrelationId> {
   /**
diff --git a/core/src/main/java/org/apache/calcite/rel/core/Join.java b/core/src/main/java/org/apache/calcite/rel/core/Join.java
index 4f38695..be2d47b 100644
--- a/core/src/main/java/org/apache/calcite/rel/core/Join.java
+++ b/core/src/main/java/org/apache/calcite/rel/core/Join.java
@@ -191,6 +191,7 @@ public abstract class Join extends BiRel implements Hintable {
     return planner.getCostFactory().makeCost(rowCount, 0, 0);
   }
 
+  // CHECKSTYLE: IGNORE 1
   /** @deprecated Use {@link RelMdUtil#getJoinRowCount(RelMetadataQuery, Join, RexNode)}. */
   @Deprecated // to be removed before 2.0
   public static double estimateJoinedRows(
diff --git a/core/src/main/java/org/apache/calcite/rel/core/RepeatUnion.java b/core/src/main/java/org/apache/calcite/rel/core/RepeatUnion.java
index 9975429..94d3865 100644
--- a/core/src/main/java/org/apache/calcite/rel/core/RepeatUnion.java
+++ b/core/src/main/java/org/apache/calcite/rel/core/RepeatUnion.java
@@ -59,7 +59,7 @@ public abstract class RepeatUnion extends BiRel {
 
   /**
    * Maximum number of times to repeat the iterative relational expression;
-   * negative value means no limit, 0 means only seed will be evaluated
+   * negative value means no limit, 0 means only seed will be evaluated.
    */
   public final int iterationLimit;
 
diff --git a/core/src/main/java/org/apache/calcite/rel/metadata/RelColumnOrigin.java b/core/src/main/java/org/apache/calcite/rel/metadata/RelColumnOrigin.java
index 4416df7..6a3d7e5 100644
--- a/core/src/main/java/org/apache/calcite/rel/metadata/RelColumnOrigin.java
+++ b/core/src/main/java/org/apache/calcite/rel/metadata/RelColumnOrigin.java
@@ -44,19 +44,15 @@ public class RelColumnOrigin {
 
   //~ Methods ----------------------------------------------------------------
 
-  /**
-   * @return table of origin
-   */
+  /** Returns table of origin. */
   public RelOptTable getOriginTable() {
     return originTable;
   }
 
-  /**
-   * @return 0-based index of column in origin table; whether this ordinal is
-   * flattened or unflattened depends on whether UDT flattening has already
+  /** Returns the 0-based index of column in origin table; whether this ordinal
+   * is flattened or unflattened depends on whether UDT flattening has already
    * been performed on the relational expression which produced this
-   * description
-   */
+   * description. */
   public int getOriginColumnOrdinal() {
     return iOriginColumn;
   }
diff --git a/core/src/main/java/org/apache/calcite/rel/metadata/RelMdPredicates.java b/core/src/main/java/org/apache/calcite/rel/metadata/RelMdPredicates.java
index 0061042..698b059 100644
--- a/core/src/main/java/org/apache/calcite/rel/metadata/RelMdPredicates.java
+++ b/core/src/main/java/org/apache/calcite/rel/metadata/RelMdPredicates.java
@@ -485,6 +485,7 @@ public class RelMdPredicates
     return mq.getPulledUpPredicates(input);
   }
 
+  // CHECKSTYLE: IGNORE 1
   /** @see RelMetadataQuery#getPulledUpPredicates(RelNode) */
   public RelOptPredicateList getPredicates(RelSubset r,
       RelMetadataQuery mq) {
diff --git a/core/src/main/java/org/apache/calcite/rel/metadata/RelMdUtil.java b/core/src/main/java/org/apache/calcite/rel/metadata/RelMdUtil.java
index 2a28588..779e9b4 100644
--- a/core/src/main/java/org/apache/calcite/rel/metadata/RelMdUtil.java
+++ b/core/src/main/java/org/apache/calcite/rel/metadata/RelMdUtil.java
@@ -425,7 +425,7 @@ public class RelMdUtil {
 
   /**
    * Takes the difference between two predicates, removing from the first any
-   * predicates also in the second
+   * predicates also in the second.
    *
    * @param rexBuilder rexBuilder used to construct AND'd RexNode
    * @param pred1      first predicate
@@ -472,7 +472,8 @@ public class RelMdUtil {
 
   /**
    * Forms two bitmaps by splitting the columns in a bitmap according to
-   * whether or not the column references the child input or is an expression
+   * whether or not the column references the child input or is an expression.
+   *
    * @param projExprs Project expressions
    * @param groupKey  Bitmap whose columns will be split
    * @param baseCols  Bitmap representing columns from the child input
@@ -507,22 +508,22 @@ public class RelMdUtil {
   }
 
   /**
-   * Computes the population size for a set of keys returned from a join
+   * Computes the population size for a set of keys returned from a join.
    *
-   * @param joinRel  the join rel
-   * @param groupKey keys to compute the population for
+   * @param join_  Join relational operator
+   * @param groupKey Keys to compute the population for
    * @return computed population size
    */
   public static Double getJoinPopulationSize(RelMetadataQuery mq,
-      RelNode joinRel, ImmutableBitSet groupKey) {
-    Join join = (Join) joinRel;
+      RelNode join_, ImmutableBitSet groupKey) {
+    Join join = (Join) join_;
     if (!join.getJoinType().projectsRight()) {
       return mq.getPopulationSize(join.getLeft(), groupKey);
     }
     ImmutableBitSet.Builder leftMask = ImmutableBitSet.builder();
     ImmutableBitSet.Builder rightMask = ImmutableBitSet.builder();
-    RelNode left = joinRel.getInputs().get(0);
-    RelNode right = joinRel.getInputs().get(1);
+    RelNode left = join.getLeft();
+    RelNode right = join.getRight();
 
     // separate the mask into masks for the left and right
     RelMdUtil.setLeftRightBitmaps(
@@ -533,7 +534,7 @@ public class RelMdUtil {
             mq.getPopulationSize(left, leftMask.build()),
             mq.getPopulationSize(right, rightMask.build()));
 
-    return numDistinctVals(population, mq.getRowCount(joinRel));
+    return numDistinctVals(population, mq.getRowCount(join));
   }
 
   /** Add an epsilon to the value passed in. **/
@@ -560,7 +561,7 @@ public class RelMdUtil {
 
   /**
    * Computes the number of distinct rows for a set of keys returned from a
-   * semi-join
+   * semi-join.
    *
    * @param semiJoinRel RelNode representing the semi-join
    * @param mq          metadata query
diff --git a/core/src/main/java/org/apache/calcite/rel/mutable/MutableCalc.java b/core/src/main/java/org/apache/calcite/rel/mutable/MutableCalc.java
index 79c2b81..5791bb9 100644
--- a/core/src/main/java/org/apache/calcite/rel/mutable/MutableCalc.java
+++ b/core/src/main/java/org/apache/calcite/rel/mutable/MutableCalc.java
@@ -30,7 +30,7 @@ public class MutableCalc extends MutableSingleRel {
   }
 
   /**
-   * Creates a MutableCalc
+   * Creates a MutableCalc.
    *
    * @param input   Input relational expression
    * @param program Calc program
diff --git a/core/src/main/java/org/apache/calcite/rel/mutable/MutableMultiRel.java b/core/src/main/java/org/apache/calcite/rel/mutable/MutableMultiRel.java
index 277c3ef..00af4cf 100644
--- a/core/src/main/java/org/apache/calcite/rel/mutable/MutableMultiRel.java
+++ b/core/src/main/java/org/apache/calcite/rel/mutable/MutableMultiRel.java
@@ -25,7 +25,7 @@ import com.google.common.collect.Lists;
 import java.util.ArrayList;
 import java.util.List;
 
-/** Base Class for relations with three or more inputs */
+/** Base Class for relations with three or more inputs. */
 abstract class MutableMultiRel extends MutableRel {
   protected final List<MutableRel> inputs;
 
diff --git a/core/src/main/java/org/apache/calcite/rel/rel2sql/RelToSqlConverter.java b/core/src/main/java/org/apache/calcite/rel/rel2sql/RelToSqlConverter.java
index c1f87b8..67325b3 100644
--- a/core/src/main/java/org/apache/calcite/rel/rel2sql/RelToSqlConverter.java
+++ b/core/src/main/java/org/apache/calcite/rel/rel2sql/RelToSqlConverter.java
@@ -156,7 +156,7 @@ public class RelToSqlConverter extends SqlImplementor
             frame.parent);
   }
 
-  /** @see #dispatch */
+  /** Visits a RelNode; called by {@link #dispatch} via reflection. */
   public Result visit(RelNode e) {
     throw new AssertionError("Need to implement " + e.getClass().getName());
   }
@@ -192,7 +192,7 @@ public class RelToSqlConverter extends SqlImplementor
     }
   }
 
-  /** @see #dispatch */
+  /** Visits a Join; called by {@link #dispatch} via reflection. */
   public Result visit(Join e) {
     switch (e.getJoinType()) {
     case ANTI:
@@ -284,7 +284,7 @@ public class RelToSqlConverter extends SqlImplementor
     return e.getJoinType() == JoinRelType.INNER && e.getCondition().isAlwaysTrue();
   }
 
-  /** @see #dispatch */
+  /** Visits a Correlate; called by {@link #dispatch} via reflection. */
   public Result visit(Correlate e) {
     final Result leftResult =
         visitInput(e, 0)
@@ -308,7 +308,7 @@ public class RelToSqlConverter extends SqlImplementor
     return result(join, leftResult, rightResult);
   }
 
-  /** @see #dispatch */
+  /** Visits a Filter; called by {@link #dispatch} via reflection. */
   public Result visit(Filter e) {
     final RelNode input = e.getInput();
     if (input instanceof Aggregate) {
@@ -329,7 +329,7 @@ public class RelToSqlConverter extends SqlImplementor
     }
   }
 
-  /** @see #dispatch */
+  /** Visits a Project; called by {@link #dispatch} via reflection. */
   public Result visit(Project e) {
     final Result x = visitInput(e, 0, Clause.SELECT);
     parseCorrelTable(e, x);
@@ -361,7 +361,7 @@ public class RelToSqlConverter extends SqlImplementor
             sqlNodeNull, dialect.getCastSpec(field.getType()));
   }
 
-  /** @see #dispatch */
+  /** Visits a Window; called by {@link #dispatch} via reflection. */
   public Result visit(Window e) {
     final Result x = visitInput(e, 0);
     final Builder builder = x.builder(e);
@@ -385,7 +385,7 @@ public class RelToSqlConverter extends SqlImplementor
     return builder.result();
   }
 
-  /** @see #dispatch */
+  /** Visits an Aggregate; called by {@link #dispatch} via reflection. */
   public Result visit(Aggregate e) {
     final Builder builder =
         visitAggregate(e, e.getGroupSet().toList(), Clause.GROUP_BY);
@@ -510,34 +510,34 @@ public class RelToSqlConverter extends SqlImplementor
     }
   }
 
-  /** @see #dispatch */
+  /** Visits a TableScan; called by {@link #dispatch} via reflection. */
   public Result visit(TableScan e) {
     final SqlIdentifier identifier = getSqlTargetTable(e);
     return result(identifier, ImmutableList.of(Clause.FROM), e, null);
   }
 
-  /** @see #dispatch */
+  /** Visits a Union; called by {@link #dispatch} via reflection. */
   public Result visit(Union e) {
     return setOpToSql(e.all
         ? SqlStdOperatorTable.UNION_ALL
         : SqlStdOperatorTable.UNION, e);
   }
 
-  /** @see #dispatch */
+  /** Visits an Intersect; called by {@link #dispatch} via reflection. */
   public Result visit(Intersect e) {
     return setOpToSql(e.all
         ? SqlStdOperatorTable.INTERSECT_ALL
         : SqlStdOperatorTable.INTERSECT, e);
   }
 
-  /** @see #dispatch */
+  /** Visits a Minus; called by {@link #dispatch} via reflection. */
   public Result visit(Minus e) {
     return setOpToSql(e.all
         ? SqlStdOperatorTable.EXCEPT_ALL
         : SqlStdOperatorTable.EXCEPT, e);
   }
 
-  /** @see #dispatch */
+  /** Visits a Calc; called by {@link #dispatch} via reflection. */
   public Result visit(Calc e) {
     final RexProgram program = e.getProgram();
     final ImmutableSet<Clause> expectedClauses =
@@ -563,7 +563,7 @@ public class RelToSqlConverter extends SqlImplementor
     return builder.result();
   }
 
-  /** @see #dispatch */
+  /** Visits a Values; called by {@link #dispatch} via reflection. */
   public Result visit(Values e) {
     final List<Clause> clauses = ImmutableList.of(Clause.SELECT);
     final Map<String, RelDataType> pairs = ImmutableMap.of();
@@ -680,7 +680,7 @@ public class RelToSqlConverter extends SqlImplementor
                     SqlLiteral.createExactNumeric("0", POS)));
   }
 
-  /** @see #dispatch */
+  /** Visits a Sort; called by {@link #dispatch} via reflection. */
   public Result visit(Sort e) {
     if (e.getInput() instanceof Aggregate) {
       final Aggregate aggregate = (Aggregate) e.getInput();
@@ -784,7 +784,7 @@ public class RelToSqlConverter extends SqlImplementor
     return sqlTargetTable;
   }
 
-  /** @see #dispatch */
+  /** Visits a TableModify; called by {@link #dispatch} via reflection. */
   public Result visit(TableModify modify) {
     final Map<String, RelDataType> pairs = ImmutableMap.of();
     final Context context = aliasContext(pairs, false);
@@ -847,7 +847,7 @@ public class RelToSqlConverter extends SqlImplementor
         Lists.transform(names, name -> new SqlIdentifier(name, POS)), POS);
   }
 
-  /** @see #dispatch */
+  /** Visits a Match; called by {@link #dispatch} via reflection. */
   public Result visit(Match e) {
     final RelNode input = e.getInput();
     final Result x = visitInput(e, 0);
diff --git a/core/src/main/java/org/apache/calcite/rel/rel2sql/SqlImplementor.java b/core/src/main/java/org/apache/calcite/rel/rel2sql/SqlImplementor.java
index 8fe1fad..1ec9ee0 100644
--- a/core/src/main/java/org/apache/calcite/rel/rel2sql/SqlImplementor.java
+++ b/core/src/main/java/org/apache/calcite/rel/rel2sql/SqlImplementor.java
@@ -137,9 +137,10 @@ public abstract class SqlImplementor {
     };
   }
 
+  // CHECKSTYLE: IGNORE 1
   /** @deprecated Use either {@link #visitRoot(RelNode)} or
    * {@link #visitInput(RelNode, int)}. */
-  @Deprecated
+  @Deprecated // to be removed before 2.0
   public final Result visitChild(int i, RelNode e) {
     throw new UnsupportedOperationException();
   }
@@ -1261,9 +1262,7 @@ public abstract class SqlImplementor {
     return new TableFunctionScanContext(dialect, inputSqlNodes);
   }
 
-  /**
-   * Context for translating MATCH_RECOGNIZE clause
-   */
+  /** Context for translating MATCH_RECOGNIZE clause. */
   public class MatchRecognizeContext extends AliasContext {
     protected MatchRecognizeContext(SqlDialect dialect,
         Map<String, RelDataType> aliases) {
@@ -1398,6 +1397,7 @@ public abstract class SqlImplementor {
       return builder(rel, expectedClauses);
     }
 
+    // CHECKSTYLE: IGNORE 3
     /** @deprecated Provide the expected clauses up-front, when you call
      * {@link #visitInput(RelNode, int, Set)}, then create a builder using
      * {@link #builder(RelNode)}. */
diff --git a/core/src/main/java/org/apache/calcite/rel/rules/AggregateReduceFunctionsRule.java b/core/src/main/java/org/apache/calcite/rel/rules/AggregateReduceFunctionsRule.java
index 0ffbbcb..427ebc2 100644
--- a/core/src/main/java/org/apache/calcite/rel/rules/AggregateReduceFunctionsRule.java
+++ b/core/src/main/java/org/apache/calcite/rel/rules/AggregateReduceFunctionsRule.java
@@ -178,7 +178,7 @@ public class AggregateReduceFunctionsRule
   }
 
   /**
-   * Returns whether the aggregate call is a reducible function
+   * Returns whether the aggregate call is a reducible function.
    */
   private boolean isReducible(final SqlKind kind) {
     return functionsToReduce.contains(kind);
diff --git a/core/src/main/java/org/apache/calcite/rel/rules/JoinProjectTransposeRule.java b/core/src/main/java/org/apache/calcite/rel/rules/JoinProjectTransposeRule.java
index c790a37..2edd53e 100644
--- a/core/src/main/java/org/apache/calcite/rel/rules/JoinProjectTransposeRule.java
+++ b/core/src/main/java/org/apache/calcite/rel/rules/JoinProjectTransposeRule.java
@@ -284,26 +284,17 @@ public class JoinProjectTransposeRule
     call.transformTo(relBuilder.build());
   }
 
-  /**
-   * @param call RelOptRuleCall
-   * @return true if the rule was invoked with a left project child
-   */
+  /** Returns whether the rule was invoked with a left project child. */
   protected boolean hasLeftChild(RelOptRuleCall call) {
     return call.rel(1) instanceof Project;
   }
 
-  /**
-   * @param call RelOptRuleCall
-   * @return true if the rule was invoked with 2 children
-   */
+  /** Returns whether the rule was invoked with 2 children. */
   protected boolean hasRightChild(RelOptRuleCall call) {
     return call.rels.length == 3;
   }
 
-  /**
-   * @param call RelOptRuleCall
-   * @return LogicalProject corresponding to the right child
-   */
+  /** Returns the Project corresponding to the right child. */
   protected Project getRightChild(RelOptRuleCall call) {
     return call.rel(2);
   }
diff --git a/core/src/main/java/org/apache/calcite/rel/rules/JoinToMultiJoinRule.java b/core/src/main/java/org/apache/calcite/rel/rules/JoinToMultiJoinRule.java
index 8e0651c..2f1d18b 100644
--- a/core/src/main/java/org/apache/calcite/rel/rules/JoinToMultiJoinRule.java
+++ b/core/src/main/java/org/apache/calcite/rel/rules/JoinToMultiJoinRule.java
@@ -378,25 +378,25 @@ public class JoinToMultiJoinRule
    * Combines the join filters from the left and right inputs (if they are
    * MultiJoinRels) with the join filter in the joinrel into a single AND'd
    * join filter, unless the inputs correspond to null generating inputs in an
-   * outer join
+   * outer join.
    *
-   * @param joinRel join rel
-   * @param left    left child of the join
-   * @param right   right child of the join
+   * @param join    Join
+   * @param left    Left input of the join
+   * @param right   Right input of the join
    * @return combined join filters AND-ed together
    */
   private List<RexNode> combineJoinFilters(
-      Join joinRel,
+      Join join,
       RelNode left,
       RelNode right) {
-    JoinRelType joinType = joinRel.getJoinType();
+    JoinRelType joinType = join.getJoinType();
 
     // AND the join condition if this isn't a left or right outer join;
     // in those cases, the outer join condition is already tracked
     // separately
     final List<RexNode> filters = new ArrayList<>();
     if ((joinType != JoinRelType.LEFT) && (joinType != JoinRelType.RIGHT)) {
-      filters.add(joinRel.getCondition());
+      filters.add(join.getCondition());
     }
     if (canCombine(left, joinType.generatesNullsOnLeft())) {
       filters.add(((MultiJoin) left).getJoinFilter());
@@ -406,7 +406,7 @@ public class JoinToMultiJoinRule
     if (canCombine(right, joinType.generatesNullsOnRight())) {
       MultiJoin multiJoin = (MultiJoin) right;
       filters.add(
-          shiftRightFilter(joinRel, left, multiJoin,
+          shiftRightFilter(join, left, multiJoin,
               multiJoin.getJoinFilter()));
     }
 
diff --git a/core/src/main/java/org/apache/calcite/rel/rules/LoptJoinTree.java b/core/src/main/java/org/apache/calcite/rel/rules/LoptJoinTree.java
index dcec3bf..de8de4b 100644
--- a/core/src/main/java/org/apache/calcite/rel/rules/LoptJoinTree.java
+++ b/core/src/main/java/org/apache/calcite/rel/rules/LoptJoinTree.java
@@ -175,9 +175,7 @@ public class LoptJoinTree {
       this.id = rootId;
     }
 
-    /**
-     * @return the id associated with a leaf node in a binary tree
-     */
+    /** Returns the id associated with a leaf node in a binary tree. */
     public int getId() {
       return id;
     }
diff --git a/core/src/main/java/org/apache/calcite/rel/rules/LoptMultiJoin.java b/core/src/main/java/org/apache/calcite/rel/rules/LoptMultiJoin.java
index f49644f..4614b2f 100644
--- a/core/src/main/java/org/apache/calcite/rel/rules/LoptMultiJoin.java
+++ b/core/src/main/java/org/apache/calcite/rel/rules/LoptMultiJoin.java
@@ -51,9 +51,7 @@ import java.util.Set;
 public class LoptMultiJoin {
   //~ Instance fields --------------------------------------------------------
 
-  /**
-   * The MultiJoin being optimized
-   */
+  /** The MultiJoin being optimized. */
   MultiJoin multiJoin;
 
   /**
@@ -68,32 +66,26 @@ public class LoptMultiJoin {
    */
   private List<RexNode> allJoinFilters;
 
-  /**
-   * Number of factors into the MultiJoin
-   */
+  /** Number of factors into the MultiJoin. */
   private final int nJoinFactors;
 
-  /**
-   * Total number of fields in the MultiJoin
-   */
+  /** Total number of fields in the MultiJoin. */
   private int nTotalFields;
 
-  /**
-   * Original inputs into the MultiJoin
-   */
+  /** Original inputs into the MultiJoin. */
   private final ImmutableList<RelNode> joinFactors;
 
   /**
-   * If a join factor is null generating in a left or right outer join,
+   * If a join factor is null-generating in a left or right outer join,
    * joinTypes indicates the join type corresponding to the factor. Otherwise,
    * it is set to INNER.
    */
   private final ImmutableList<JoinRelType> joinTypes;
 
   /**
-   * If a join factor is null generating in a left or right outer join, the
-   * bitmap contains the non-null generating factors that the null generating
-   * factor is dependent upon
+   * If a join factor is null-generating in a left or right outer join, the
+   * bitmap contains the non-null generating factors that the null-generating
+   * factor is dependent upon.
    */
   private final ImmutableBitSet [] outerJoinFactors;
 
@@ -115,39 +107,39 @@ public class LoptMultiJoin {
 
   /**
    * For each join filter, associates a bitmap indicating all factors
-   * referenced by the filter
+   * referenced by the filter.
    */
   private Map<RexNode, ImmutableBitSet> factorsRefByJoinFilter;
 
   /**
    * For each join filter, associates a bitmap indicating all fields
-   * referenced by the filter
+   * referenced by the filter.
    */
   private Map<RexNode, ImmutableBitSet> fieldsRefByJoinFilter;
 
   /**
-   * Starting RexInputRef index corresponding to each join factor
+   * Starting RexInputRef index corresponding to each join factor.
    */
   int [] joinStart;
 
   /**
-   * Number of fields in each join factor
+   * Number of fields in each join factor.
    */
   int [] nFieldsInJoinFactor;
 
   /**
    * Bitmap indicating which factors each factor references in join filters
-   * that correspond to comparisons
+   * that correspond to comparisons.
    */
   ImmutableBitSet [] factorsRefByFactor;
 
   /**
-   * Weights of each factor combination
+   * Weights of each factor combination.
    */
   int [][] factorWeights;
 
   /**
-   * Type factory
+   * Type factory.
    */
   final RelDataTypeFactory factory;
 
@@ -161,13 +153,13 @@ public class LoptMultiJoin {
   Integer [] joinRemovalFactors;
 
   /**
-   * The semijoins that allow the join of a dimension table to be removed
+   * The semijoins that allow the join of a dimension table to be removed.
    */
   LogicalJoin[] joinRemovalSemiJoins;
 
   /**
    * Set of null-generating factors whose corresponding outer join can be
-   * removed from the query plan
+   * removed from the query plan.
    */
   Set<Integer> removableOuterJoinFactors;
 
@@ -241,168 +233,168 @@ public class LoptMultiJoin {
   //~ Methods ----------------------------------------------------------------
 
   /**
-   * @return the MultiJoin corresponding to this multijoin
+   * Returns the MultiJoin corresponding to this multi-join.
    */
   public MultiJoin getMultiJoinRel() {
     return multiJoin;
   }
 
   /**
-   * @return number of factors in this multijoin
+   * Returns the number of factors in this multi-join.
    */
   public int getNumJoinFactors() {
     return nJoinFactors;
   }
 
   /**
-   * @param factIdx factor to be returned
+   * Returns the factor corresponding to the given factor index.
    *
-   * @return factor corresponding to the factor index passed in
+   * @param factIdx Factor to be returned
    */
   public RelNode getJoinFactor(int factIdx) {
     return joinFactors.get(factIdx);
   }
 
   /**
-   * @return total number of fields in the multijoin
+   * Returns the total number of fields in the multi-join.
    */
   public int getNumTotalFields() {
     return nTotalFields;
   }
 
   /**
-   * @param factIdx desired factor
+   * Returns the number of fields in a given factor.
    *
-   * @return number of fields in the specified factor
+   * @param factIdx Desired factor
    */
   public int getNumFieldsInJoinFactor(int factIdx) {
     return nFieldsInJoinFactor[factIdx];
   }
 
   /**
-   * @return all non-outer join filters in this multijoin
+   * Returns all non-outer join filters in this multi-join.
    */
   public List<RexNode> getJoinFilters() {
     return joinFilters;
   }
 
   /**
-   * @param joinFilter filter for which information will be returned
+   * Returns a bitmap corresponding to the factors referenced within
+   * the specified join filter.
    *
-   * @return bitmap corresponding to the factors referenced within the
-   * specified join filter
+   * @param joinFilter Filter for which information will be returned
    */
   public ImmutableBitSet getFactorsRefByJoinFilter(RexNode joinFilter) {
     return factorsRefByJoinFilter.get(joinFilter);
   }
 
   /**
-   * Returns array of fields contained within the multi-join
+   * Returns an array of fields contained within the multi-join.
    */
   public List<RelDataTypeField> getMultiJoinFields() {
     return multiJoin.getRowType().getFieldList();
   }
 
   /**
-   * @param joinFilter the filter for which information will be returned
+   * Returns a bitmap corresponding to the fields referenced by a join filter.
    *
-   * @return bitmap corresponding to the fields referenced by a join filter
+   * @param joinFilter the filter for which information will be returned
    */
   public ImmutableBitSet getFieldsRefByJoinFilter(RexNode joinFilter) {
     return fieldsRefByJoinFilter.get(joinFilter);
   }
 
   /**
-   * @return weights of the different factors relative to one another
+   * Returns weights of the different factors relative to one another.
    */
   public int [][] getFactorWeights() {
     return factorWeights;
   }
 
   /**
-   * @param factIdx factor for which information will be returned
+   * Returns a bitmap corresponding to the factors referenced by the specified
+   * factor in the various join filters that correspond to comparisons.
    *
-   * @return bitmap corresponding to the factors referenced by the specified
-   * factor in the various join filters that correspond to comparisons
+   * @param factIdx Factor for which information will be returned
    */
   public ImmutableBitSet getFactorsRefByFactor(int factIdx) {
     return factorsRefByFactor[factIdx];
   }
 
   /**
-   * @param factIdx factor for which information will be returned
+   * Returns the starting offset within the multi-join for the specified factor.
    *
-   * @return starting offset within the multijoin for the specified factor
+   * @param factIdx Factor for which information will be returned
    */
   public int getJoinStart(int factIdx) {
     return joinStart[factIdx];
   }
 
   /**
-   * @param factIdx factor for which information will be returned
+   * Returns whether the factor corresponds to a null-generating factor
+   * in a left or right outer join.
    *
-   * @return whether or not the factor corresponds to a null-generating factor
-   * in a left or right outer join
+   * @param factIdx Factor for which information will be returned
    */
   public boolean isNullGenerating(int factIdx) {
     return joinTypes.get(factIdx).isOuterJoin();
   }
 
   /**
-   * @param factIdx factor for which information will be returned
+   * Returns a bitmap containing the factors that a null-generating factor is
+   * dependent upon, if the factor is null-generating in a left or right outer
+   * join; otherwise null is returned.
    *
-   * @return bitmap containing the factors that a null generating factor is
-   * dependent upon, if the factor is null generating in a left or right outer
-   * join; otherwise null is returned
+   * @param factIdx Factor for which information will be returned
    */
   public ImmutableBitSet getOuterJoinFactors(int factIdx) {
     return outerJoinFactors[factIdx];
   }
 
   /**
-   * @param factIdx factor for which information will be returned
+   * Returns outer join conditions associated with the specified null-generating
+   * factor.
    *
-   * @return outer join conditions associated with the specified null
-   * generating factor
+   * @param factIdx Factor for which information will be returned
    */
   public RexNode getOuterJoinCond(int factIdx) {
     return multiJoin.getOuterJoinConditions().get(factIdx);
   }
 
   /**
-   * @param factIdx factor for which information will be returned
+   * Returns a bitmap containing the fields that are projected from a factor.
    *
-   * @return bitmap containing the fields that are projected from a factor
+   * @param factIdx Factor for which information will be returned
    */
   public ImmutableBitSet getProjFields(int factIdx) {
     return projFields.get(factIdx);
   }
 
   /**
-   * @param factIdx factor for which information will be returned
+   * Returns the join field reference counts for a factor.
    *
-   * @return the join field reference counts for a factor
+   * @param factIdx Factor for which information will be returned
    */
   public int [] getJoinFieldRefCounts(int factIdx) {
     return joinFieldRefCountsMap.get(factIdx);
   }
 
   /**
-   * @param dimIdx the dimension factor for which information will be returned
+   * Returns the factor id of the fact table corresponding to a dimension
+   * table in a semi-join, in the case where the join with the dimension table
+   * can be removed.
    *
-   * @return the factor id of the fact table corresponding to a dimension
-   * table in a semijoin, in the case where the join with the dimension table
-   * can be removed
+   * @param dimIdx Dimension factor for which information will be returned
    */
   public Integer getJoinRemovalFactor(int dimIdx) {
     return joinRemovalFactors[dimIdx];
   }
 
   /**
-   * @param dimIdx the dimension factor for which information will be returned
+   * Returns the semi-join that allows the join of a dimension table to be
+   * removed.
    *
-   * @return the semijoin that allows the join of a dimension table to be
-   * removed
+   * @param dimIdx Dimension factor for which information will be returned
    */
   public LogicalJoin getJoinRemovalSemiJoin(int dimIdx) {
     return joinRemovalSemiJoins[dimIdx];
@@ -412,18 +404,18 @@ public class LoptMultiJoin {
    * Indicates that a dimension factor's join can be removed because of a
    * semijoin with a fact table.
    *
-   * @param dimIdx id of the dimension factor
-   * @param factIdx id of the fact factor
+   * @param dimIdx Dimension factor
+   * @param factIdx Fact factor
    */
   public void setJoinRemovalFactor(int dimIdx, int factIdx) {
     joinRemovalFactors[dimIdx] = factIdx;
   }
 
   /**
-   * Indicates the semijoin that allows the join of a dimension table to be
-   * removed
+   * Indicates the semi-join that allows the join of a dimension table to be
+   * removed.
    *
-   * @param dimIdx id of the dimension factor
+   * @param dimIdx Dimension factor
    * @param semiJoin the semijoin
    */
   public void setJoinRemovalSemiJoin(int dimIdx, LogicalJoin semiJoin) {
@@ -431,7 +423,7 @@ public class LoptMultiJoin {
   }
 
   /**
-   * Returns a bitmap representing the factors referenced in a join filter
+   * Returns a bitmap representing the factors referenced in a join filter.
    *
    * @param joinFilter the join filter
    * @param setFields if true, add the fields referenced by the join filter
@@ -458,7 +450,7 @@ public class LoptMultiJoin {
 
   /**
    * Sets bitmaps indicating which factors and fields each join filter
-   * references
+   * references.
    */
   private void setJoinFilterRefs() {
     fieldsRefByJoinFilter = new HashMap<>();
@@ -480,7 +472,7 @@ public class LoptMultiJoin {
 
   /**
    * Sets the bitmap indicating which factors a filter references based on
-   * which fields it references
+   * which fields it references.
    *
    * @param fieldRefBitmap bitmap representing fields referenced
    * @return bitmap representing factors referenced that will
@@ -496,7 +488,7 @@ public class LoptMultiJoin {
   }
 
   /**
-   * Determines the join factor corresponding to a RexInputRef
+   * Determines the join factor corresponding to a RexInputRef.
    *
    * @param rexInputRef rexInputRef index
    *
@@ -590,7 +582,7 @@ public class LoptMultiJoin {
 
   /**
    * Sets an individual weight if the new weight is better than the current
-   * one
+   * one.
    *
    * @param weight weight to be set
    * @param leftFactor index of left factor
@@ -604,10 +596,10 @@ public class LoptMultiJoin {
   }
 
   /**
-   * Returns true if a join tree contains all factors required
+   * Returns whether if a join tree contains all factors required.
    *
-   * @param joinTree join tree to be examined
-   * @param factorsNeeded bitmap of factors required
+   * @param joinTree Join tree to be examined
+   * @param factorsNeeded Bitmap of factors required
    *
    * @return true if join tree contains all required factors
    */
@@ -618,7 +610,7 @@ public class LoptMultiJoin {
   }
 
   /**
-   * Sets a bitmap indicating all child RelNodes in a join tree
+   * Sets a bitmap indicating all child RelNodes in a join tree.
    *
    * @param joinTree join tree to be examined
    * @param childFactors bitmap to be set
@@ -633,7 +625,7 @@ public class LoptMultiJoin {
 
   /**
    * Retrieves the fields corresponding to a join between a left and right
-   * tree
+   * tree.
    *
    * @param left left hand side of the join
    * @param right right hand side of the join
@@ -652,20 +644,20 @@ public class LoptMultiJoin {
 
   /**
    * Adds a join factor to the set of factors that can be removed because the
-   * factor is the null generating factor in an outer join, its join keys are
-   * unique, and the factor is not projected in the query
+   * factor is the null-generating factor in an outer join, its join keys are
+   * unique, and the factor is not projected in the query.
    *
-   * @param factIdx join factor
+   * @param factIdx Join factor
    */
   public void addRemovableOuterJoinFactor(int factIdx) {
     removableOuterJoinFactors.add(factIdx);
   }
 
   /**
-   * @param factIdx factor in question
+   * Return whether the factor corresponds to the null-generating factor in
+   * an outer join that can be removed.
    *
-   * @return true if the factor corresponds to the null generating factor in
-   * an outer join that can be removed
+   * @param factIdx Factor in question
    */
   public boolean isRemovableOuterJoinFactor(int factIdx) {
     return removableOuterJoinFactors.contains(factIdx);
@@ -746,37 +738,37 @@ public class LoptMultiJoin {
     RemovableSelfJoin selfJoin = removableSelfJoinPairs.get(factIdx);
     if (selfJoin == null) {
       return null;
-    } else if (selfJoin.getRightFactor() == factIdx) {
-      return selfJoin.getLeftFactor();
+    } else if (selfJoin.rightFactor == factIdx) {
+      return selfJoin.leftFactor;
     } else {
-      return selfJoin.getRightFactor();
+      return selfJoin.rightFactor;
     }
   }
 
   /**
-   * @param factIdx factor in a self-join
+   * Returns whether the factor is the left factor in a self-join.
    *
-   * @return true if the factor is the left factor in a self-join
+   * @param factIdx Factor in a self-join
    */
   public boolean isLeftFactorInRemovableSelfJoin(int factIdx) {
     RemovableSelfJoin selfJoin = removableSelfJoinPairs.get(factIdx);
     if (selfJoin == null) {
       return false;
     }
-    return selfJoin.getLeftFactor() == factIdx;
+    return selfJoin.leftFactor == factIdx;
   }
 
   /**
-   * @param factIdx factor in a self-join
+   * Returns whether the factor is the right factor in a self-join.
    *
-   * @return true if the factor is the right factor in a self-join
+   * @param factIdx Factor in a self-join
    */
   public boolean isRightFactorInRemovableSelfJoin(int factIdx) {
     RemovableSelfJoin selfJoin = removableSelfJoinPairs.get(factIdx);
     if (selfJoin == null) {
       return false;
     }
-    return selfJoin.getRightFactor() == factIdx;
+    return selfJoin.rightFactor == factIdx;
   }
 
   /**
@@ -792,8 +784,8 @@ public class LoptMultiJoin {
    */
   public Integer getRightColumnMapping(int rightFactor, int rightOffset) {
     RemovableSelfJoin selfJoin = removableSelfJoinPairs.get(rightFactor);
-    assert selfJoin.getRightFactor() == rightFactor;
-    return selfJoin.getColumnMapping().get(rightOffset);
+    assert selfJoin.rightFactor == rightFactor;
+    return selfJoin.columnMapping.get(rightOffset);
   }
 
   public Edge createEdge(RexNode condition) {
@@ -827,24 +819,18 @@ public class LoptMultiJoin {
    * Utility class used to keep track of the factors in a removable self-join.
    * The right factor in the self-join is the one that will be removed.
    */
-  private class RemovableSelfJoin {
-    /**
-     * The left factor in a removable self-join
-     */
-    private int leftFactor;
+  private static class RemovableSelfJoin {
+    /** The left factor in a removable self-join. */
+    private final int leftFactor;
 
-    /**
-     * The right factor in a removable self-join, namely the factor that
-     * will be removed
-     */
-    private int rightFactor;
+    /** The right factor in a removable self-join, namely the factor that will
+     * be removed. */
+    private final int rightFactor;
 
-    /**
-     * A mapping that maps references to columns from the right factor to
+    /** A mapping that maps references to columns from the right factor to
      * columns in the left factor, if the column is referenced in both
-     * factors
-     */
-    private Map<Integer, Integer> columnMapping;
+     * factors. */
+    private final Map<Integer, Integer> columnMapping;
 
     RemovableSelfJoin(
         int leftFactor,
@@ -854,17 +840,5 @@ public class LoptMultiJoin {
       this.rightFactor = rightFactor;
       this.columnMapping = columnMapping;
     }
-
-    public int getLeftFactor() {
-      return leftFactor;
-    }
-
-    public int getRightFactor() {
-      return rightFactor;
-    }
-
-    public Map<Integer, Integer> getColumnMapping() {
-      return columnMapping;
-    }
   }
 }
diff --git a/core/src/main/java/org/apache/calcite/rel/rules/LoptOptimizeJoinRule.java b/core/src/main/java/org/apache/calcite/rel/rules/LoptOptimizeJoinRule.java
index c6671e6..996b4cf 100644
--- a/core/src/main/java/org/apache/calcite/rel/rules/LoptOptimizeJoinRule.java
+++ b/core/src/main/java/org/apache/calcite/rel/rules/LoptOptimizeJoinRule.java
@@ -663,7 +663,7 @@ public class LoptOptimizeJoinRule
 
   /**
    * Generates a join tree with a specific factor as the first factor in the
-   * join tree
+   * join tree.
    *
    * @param multiJoin join factors being optimized
    * @param semiJoinOpt optimal semijoins for each factor
@@ -1009,7 +1009,7 @@ public class LoptOptimizeJoinRule
 
   /**
    * Creates a join tree where the new factor is pushed down one of the
-   * operands of the current join tree
+   * operands of the current join tree.
    *
    * @param multiJoin join factors being optimized
    * @param semiJoinOpt optimal semijoins for each factor
@@ -1164,7 +1164,7 @@ public class LoptOptimizeJoinRule
   }
 
   /**
-   * Creates a join tree with the new factor added to the top of the tree
+   * Creates a join tree with the new factor added to the top of the tree.
    *
    * @param multiJoin join factors being optimized
    * @param semiJoinOpt optimal semijoins for each factor
@@ -1337,7 +1337,7 @@ public class LoptOptimizeJoinRule
 
   /**
    * Adjusts a filter to reflect a newly added factor in the middle of an
-   * existing join tree
+   * existing join tree.
    *
    * @param multiJoin join factors being optimized
    * @param left left subtree of the join
@@ -1896,7 +1896,7 @@ public class LoptOptimizeJoinRule
   }
 
   /**
-   * Adjusts a filter to reflect swapping of join inputs
+   * Adjusts a filter to reflect swapping of join inputs.
    *
    * @param rexBuilder rexBuilder
    * @param multiJoin join factors being optimized
@@ -1938,7 +1938,7 @@ public class LoptOptimizeJoinRule
 
   /**
    * Sets an array indicating how much each factor in a join tree needs to be
-   * adjusted to reflect the tree's join ordering
+   * adjusted to reflect the tree's join ordering.
    *
    * @param multiJoin join factors being optimized
    * @param adjustments array to be filled out
diff --git a/core/src/main/java/org/apache/calcite/rel/rules/LoptSemiJoinOptimizer.java b/core/src/main/java/org/apache/calcite/rel/rules/LoptSemiJoinOptimizer.java
index 1ae11a5..77e2109 100644
--- a/core/src/main/java/org/apache/calcite/rel/rules/LoptSemiJoinOptimizer.java
+++ b/core/src/main/java/org/apache/calcite/rel/rules/LoptSemiJoinOptimizer.java
@@ -327,7 +327,7 @@ public class LoptSemiJoinOptimizer {
 
   /**
    * Modifies the semijoin condition to reflect the fact that the RHS is now
-   * the second factor into a join and the LHS is the first
+   * the second factor into a join and the LHS is the first.
    *
    * @param multiJoin join factors being optimized
    * @param leftAdjustment amount the left RexInputRefs need to be adjusted by
@@ -764,7 +764,7 @@ public class LoptSemiJoinOptimizer {
   }
 
   /**
-   * Removes a dimension table from a fact table's list of possible semijoins
+   * Removes a dimension table from a fact table's list of possible semi-joins.
    *
    * @param possibleDimensions possible dimension tables associated with the
    * fact table
@@ -789,10 +789,10 @@ public class LoptSemiJoinOptimizer {
   }
 
   /**
-   * @param factIdx index corresponding to the desired factor
+   * Returns the optimal semijoin for the specified factor; may be the factor
+   * itself if semijoins are not chosen for the factor.
    *
-   * @return optimal semijoin for the specified factor; may be the factor
-   * itself if semijoins are not chosen for the factor
+   * @param factIdx Index corresponding to the desired factor
    */
   public RelNode getChosenSemiJoin(int factIdx) {
     return chosenSemiJoins[factIdx];
diff --git a/core/src/main/java/org/apache/calcite/rel/rules/MultiJoin.java b/core/src/main/java/org/apache/calcite/rel/rules/MultiJoin.java
index 9fb75d9..90d3119 100644
--- a/core/src/main/java/org/apache/calcite/rel/rules/MultiJoin.java
+++ b/core/src/main/java/org/apache/calcite/rel/rules/MultiJoin.java
@@ -205,59 +205,59 @@ public final class MultiJoin extends AbstractRelNode {
   }
 
   /**
-   * @return join filters associated with this MultiJoin
+   * Returns join filters associated with this MultiJoin.
    */
   public RexNode getJoinFilter() {
     return joinFilter;
   }
 
   /**
-   * @return true if the MultiJoin corresponds to a full outer join.
+   * Returns true if the MultiJoin corresponds to a full outer join.
    */
   public boolean isFullOuterJoin() {
     return isFullOuterJoin;
   }
 
   /**
-   * @return outer join conditions for null-generating inputs
+   * Returns outer join conditions for null-generating inputs.
    */
   public List<RexNode> getOuterJoinConditions() {
     return outerJoinConditions;
   }
 
   /**
-   * @return join types of each input
+   * Returns join types of each input.
    */
   public List<JoinRelType> getJoinTypes() {
     return joinTypes;
   }
 
   /**
-   * @return bitmaps representing the fields projected from each input; if an
-   * entry is null, all fields are projected
+   * Returns bitmaps representing the fields projected from each input; if an
+   * entry is null, all fields are projected.
    */
   public List<ImmutableBitSet> getProjFields() {
     return projFields;
   }
 
   /**
-   * @return the map of reference counts for each input, representing the
-   * fields accessed in join conditions
+   * Returns the map of reference counts for each input, representing the fields
+   * accessed in join conditions.
    */
   public ImmutableMap<Integer, ImmutableIntList> getJoinFieldRefCountsMap() {
     return joinFieldRefCountsMap;
   }
 
   /**
-   * @return a copy of the map of reference counts for each input,
-   * representing the fields accessed in join conditions
+   * Returns a copy of the map of reference counts for each input, representing
+   * the fields accessed in join conditions.
    */
   public Map<Integer, int[]> getCopyJoinFieldRefCountsMap() {
     return cloneJoinFieldRefCountsMap();
   }
 
   /**
-   * @return post-join filter associated with this MultiJoin
+   * Returns post-join filter associated with this MultiJoin.
    */
   public RexNode getPostJoinFilter() {
     return postJoinFilter;
diff --git a/core/src/main/java/org/apache/calcite/rel/rules/PushProjector.java b/core/src/main/java/org/apache/calcite/rel/rules/PushProjector.java
index fcbd60e..5807540 100644
--- a/core/src/main/java/org/apache/calcite/rel/rules/PushProjector.java
+++ b/core/src/main/java/org/apache/calcite/rel/rules/PushProjector.java
@@ -76,22 +76,22 @@ public class PushProjector {
   private final RelBuilder relBuilder;
 
   /**
-   * Original projection expressions
+   * Original projection expressions.
    */
   final List<RexNode> origProjExprs;
 
   /**
-   * Fields from the RelNode that the projection is being pushed past
+   * Fields from the RelNode that the projection is being pushed past.
    */
   final List<RelDataTypeField> childFields;
 
   /**
-   * Number of fields in the RelNode that the projection is being pushed past
+   * Number of fields in the RelNode that the projection is being pushed past.
    */
   final int nChildFields;
 
   /**
-   * Bitmap containing the references in the original projection
+   * Bitmap containing the references in the original projection.
    */
   final BitSet projRefs;
 
@@ -571,7 +571,7 @@ public class PushProjector {
 
   /**
    * Determines how much each input reference needs to be adjusted as a result
-   * of projection
+   * of projection.
    *
    * @return array indicating how much each input needs to be adjusted by
    */
diff --git a/core/src/main/java/org/apache/calcite/rel/rules/ReduceDecimalsRule.java b/core/src/main/java/org/apache/calcite/rel/rules/ReduceDecimalsRule.java
index 2a79689..bbf1c84 100644
--- a/core/src/main/java/org/apache/calcite/rel/rules/ReduceDecimalsRule.java
+++ b/core/src/main/java/org/apache/calcite/rel/rules/ReduceDecimalsRule.java
@@ -134,7 +134,7 @@ public class ReduceDecimalsRule
     }
 
     /**
-     * Rewrites a call in place, from bottom up, as follows:
+     * Rewrites a call in place, from bottom up. Algorithm is as follows:
      *
      * <ol>
      * <li>visit operands
@@ -163,7 +163,7 @@ public class ReduceDecimalsRule
     }
 
     /**
-     * Registers node so it will not be computed again
+     * Registers node so it will not be computed again.
      */
     private void register(RexNode node, RexNode reducedNode) {
       Pair<RexNode, String> key = RexUtil.makeKey(node);
@@ -175,7 +175,7 @@ public class ReduceDecimalsRule
     }
 
     /**
-     * Lookup registered node
+     * Looks up a registered node.
      */
     private RexNode lookup(RexNode node) {
       Pair<RexNode, String> key = RexUtil.makeKey(node);
@@ -186,7 +186,7 @@ public class ReduceDecimalsRule
     }
 
     /**
-     * Rewrites a call, if required, or returns the original call
+     * Rewrites a call, if required, or returns the original call.
      */
     private RexNode rewriteCall(RexCall call) {
       SqlOperator operator = call.getOperator();
@@ -202,7 +202,7 @@ public class ReduceDecimalsRule
     }
 
     /**
-     * Returns a {@link RexExpander} for a call
+     * Returns a {@link RexExpander} for a call.
      */
     private RexExpander getExpander(RexCall call) {
       return expanderMap.getExpander(call);
@@ -285,7 +285,7 @@ public class ReduceDecimalsRule
    */
   public abstract static class RexExpander {
     /**
-     * Factory for constructing new relational expressions
+     * Factory for creating relational expressions.
      */
     final RexBuilder builder;
 
@@ -302,7 +302,7 @@ public class ReduceDecimalsRule
     final RelDataType real8;
 
     /**
-     * Constructs a RexExpander
+     * Creates a RexExpander.
      */
     RexExpander(RexBuilder builder) {
       this.builder = builder;
@@ -333,7 +333,7 @@ public class ReduceDecimalsRule
     public abstract RexNode expand(RexCall call);
 
     /**
-     * Makes an exact numeric literal to be used for scaling
+     * Makes an exact numeric literal to be used for scaling.
      *
      * @param scale a scale from one to max precision - 1
      * @return 10^scale as an exact numeric value
@@ -346,7 +346,7 @@ public class ReduceDecimalsRule
     }
 
     /**
-     * Makes an approximate literal to be used for scaling
+     * Makes an approximate literal to be used for scaling.
      *
      * @param scale a scale from -99 to 99
      * @return 10^scale as an approximate value
@@ -376,7 +376,7 @@ public class ReduceDecimalsRule
     }
 
     /**
-     * Calculates a power of ten, as a long value
+     * Calculates a power of ten, as a long value.
      */
     protected long powerOfTen(int scale) {
       assert scale >= 0;
@@ -386,7 +386,7 @@ public class ReduceDecimalsRule
     }
 
     /**
-     * Makes an exact, non-nullable literal of Bigint type
+     * Makes an exact, non-nullable literal of Bigint type.
      */
     protected RexNode makeExactLiteral(long l) {
       BigDecimal bd = BigDecimal.valueOf(l);
@@ -394,7 +394,7 @@ public class ReduceDecimalsRule
     }
 
     /**
-     * Makes an approximate literal of double precision
+     * Makes an approximate literal of double precision.
      */
     protected RexNode makeApproxLiteral(BigDecimal bd) {
       return builder.makeApproxLiteral(bd);
@@ -535,7 +535,7 @@ public class ReduceDecimalsRule
     }
 
     /**
-     * Retrieves a decimal node's integer representation
+     * Retrieves a decimal node's integer representation.
      *
      * @param decimalNode the decimal value as an opaque type
      * @return an integer representation of the decimal value
@@ -995,7 +995,8 @@ public class ReduceDecimalsRule
   }
 
   /**
-   * Expander that rewrites floor(decimal) expressions:
+   * Expander that rewrites {@code FLOOR(DECIMAL)} expressions.
+   * Rewrite is as follows:
    *
    * <blockquote><pre>
    * if (value &lt; 0)
@@ -1044,7 +1045,8 @@ public class ReduceDecimalsRule
   }
 
   /**
-   * Expander that rewrites ceiling(decimal) expressions:
+   * Expander that rewrites {@code CEILING(DECIMAL)} expressions.
+   * Rewrite is as follows:
    *
    * <blockquote><pre>
    * if (value &gt; 0)
@@ -1174,7 +1176,7 @@ public class ReduceDecimalsRule
   }
 
   /**
-   * An expander that casts decimal arguments as doubles
+   * Expander that casts DECIMAL arguments as DOUBLE.
    */
   private static class CastArgAsDoubleExpander extends CastArgAsTypeExpander {
     private CastArgAsDoubleExpander(RexBuilder builder) {
@@ -1194,7 +1196,7 @@ public class ReduceDecimalsRule
   }
 
   /**
-   * An expander that casts decimal arguments as another type
+   * Expander that casts DECIMAL arguments as another type.
    */
   private abstract static class CastArgAsTypeExpander extends RexExpander {
     private CastArgAsTypeExpander(RexBuilder builder) {
diff --git a/core/src/main/java/org/apache/calcite/rel/rules/materialize/MaterializedViewRule.java b/core/src/main/java/org/apache/calcite/rel/rules/materialize/MaterializedViewRule.java
index 9c2e0dc..8ff7e43 100644
--- a/core/src/main/java/org/apache/calcite/rel/rules/materialize/MaterializedViewRule.java
+++ b/core/src/main/java/org/apache/calcite/rel/rules/materialize/MaterializedViewRule.java
@@ -605,10 +605,11 @@ public abstract class MaterializedViewRule<C extends MaterializedViewRule.Config
     return result;
   }
 
-  /** Currently we only support TableScan - Project - Filter - Inner Join */
+  /** Returns whether a RelNode is a valid tree. Currently we only support
+   * TableScan - Project - Filter - Inner Join. */
   protected boolean isValidRelNodePlan(RelNode node, RelMetadataQuery mq) {
     final Multimap<Class<? extends RelNode>, RelNode> m =
-            mq.getNodeTypes(node);
+        mq.getNodeTypes(node);
     if (m == null) {
       return false;
     }
@@ -616,9 +617,9 @@ public abstract class MaterializedViewRule<C extends MaterializedViewRule.Config
     for (Entry<Class<? extends RelNode>, Collection<RelNode>> e : m.asMap().entrySet()) {
       Class<? extends RelNode> c = e.getKey();
       if (!TableScan.class.isAssignableFrom(c)
-              && !Project.class.isAssignableFrom(c)
-              && !Filter.class.isAssignableFrom(c)
-              && (!Join.class.isAssignableFrom(c))) {
+          && !Project.class.isAssignableFrom(c)
+          && !Filter.class.isAssignableFrom(c)
+          && (!Join.class.isAssignableFrom(c))) {
         // Skip it
         return false;
       }
@@ -1329,7 +1330,7 @@ public abstract class MaterializedViewRule<C extends MaterializedViewRule.Config
     }
   }
 
-  /** Edge for graph */
+  /** Edge for graph. */
   protected static class Edge extends DefaultEdge {
     final Multimap<RexTableInputRef, RexTableInputRef> equiColumns =
         ArrayListMultimap.create();
@@ -1343,7 +1344,7 @@ public abstract class MaterializedViewRule<C extends MaterializedViewRule.Config
     }
   }
 
-  /** View partitioning result */
+  /** View partitioning result. */
   protected static class ViewPartialRewriting {
     private final RelNode newView;
     private final Project newTopViewProject;
diff --git a/core/src/main/java/org/apache/calcite/rel/type/RelDataType.java b/core/src/main/java/org/apache/calcite/rel/type/RelDataType.java
index 0e41c0d..8655c04 100644
--- a/core/src/main/java/org/apache/calcite/rel/type/RelDataType.java
+++ b/core/src/main/java/org/apache/calcite/rel/type/RelDataType.java
@@ -227,27 +227,20 @@ public interface RelDataType {
    */
   RelDataTypeFamily getFamily();
 
-  /**
-   * @return precedence list for this type
-   */
+  /** Returns the precedence list for this type. */
   RelDataTypePrecedenceList getPrecedenceList();
 
-  /**
-   * @return the category of comparison operators which make sense when
-   * applied to values of this type
-   */
+  /** Returns the category of comparison operators that make sense when applied
+   * to values of this type. */
   RelDataTypeComparability getComparability();
 
-  /**
-   * @return whether it has dynamic structure (for "schema-on-read" table)
-   */
+  /** Returns whether this type has dynamic structure (for "schema-on-read"
+   * table). */
   boolean isDynamicStruct();
 
-  /**
-   * @return whether the field types are equal with each other by ignoring
-   * the field names. If it is not a struct, just return the result of
-   * {@code #equals(Object)}.
-   */
+  /** Returns whether the field types are equal with each other by ignoring the
+   * field names. If it is not a struct, just return the result of {@code
+   * #equals(Object)}. */
   @API(since = "1.24", status = API.Status.INTERNAL)
   default boolean equalsSansFieldNames(RelDataType that) {
     if (this == that) {
diff --git a/core/src/main/java/org/apache/calcite/rel/type/RelDataTypeFactory.java b/core/src/main/java/org/apache/calcite/rel/type/RelDataTypeFactory.java
index 4663d50..e82d3b4 100644
--- a/core/src/main/java/org/apache/calcite/rel/type/RelDataTypeFactory.java
+++ b/core/src/main/java/org/apache/calcite/rel/type/RelDataTypeFactory.java
@@ -183,9 +183,7 @@ public interface RelDataTypeFactory {
       Charset charset,
       SqlCollation collation);
 
-  /**
-   * @return the default {@link Charset} for string types
-   */
+  /** Returns the default {@link Charset} (valid if this is a string type). */
   Charset getDefaultCharset();
 
   /**
diff --git a/core/src/main/java/org/apache/calcite/rex/RexBuilder.java b/core/src/main/java/org/apache/calcite/rex/RexBuilder.java
index e58fe32..092bb5b 100644
--- a/core/src/main/java/org/apache/calcite/rex/RexBuilder.java
+++ b/core/src/main/java/org/apache/calcite/rex/RexBuilder.java
@@ -138,7 +138,7 @@ public class RexBuilder {
   //~ Methods ----------------------------------------------------------------
 
   /**
-   * Returns this RexBuilder's type factory
+   * Returns this RexBuilder's type factory.
    *
    * @return type factory
    */
@@ -147,7 +147,7 @@ public class RexBuilder {
   }
 
   /**
-   * Returns this RexBuilder's operator table
+   * Returns this RexBuilder's operator table.
    *
    * @return operator table
    */
@@ -730,7 +730,7 @@ public class RexBuilder {
   }
 
   /**
-   * Retrieves an interval or decimal node's integer representation
+   * Retrieves an INTERVAL or DECIMAL node's integer representation.
    *
    * @param node the interval or decimal value as an opaque type
    * @return an integer representation of the decimal value
@@ -1138,6 +1138,7 @@ public class RexBuilder {
     return makeLiteral(str, type, SqlTypeName.CHAR);
   }
 
+  // CHECKSTYLE: IGNORE 1
   /** @deprecated Use {@link #makeDateLiteral(DateString)}. */
   @Deprecated // to be removed before 2.0
   public RexLiteral makeDateLiteral(Calendar calendar) {
@@ -1152,6 +1153,7 @@ public class RexBuilder {
         typeFactory.createSqlType(SqlTypeName.DATE), SqlTypeName.DATE);
   }
 
+  // CHECKSTYLE: IGNORE 1
   /** @deprecated Use {@link #makeTimeLiteral(TimeString, int)}. */
   @Deprecated // to be removed before 2.0
   public RexLiteral makeTimeLiteral(Calendar calendar, int precision) {
@@ -1178,6 +1180,7 @@ public class RexBuilder {
         SqlTypeName.TIME_WITH_LOCAL_TIME_ZONE);
   }
 
+  // CHECKSTYLE: IGNORE 1
   /** @deprecated Use {@link #makeTimestampLiteral(TimestampString, int)}. */
   @Deprecated // to be removed before 2.0
   public RexLiteral makeTimestampLiteral(Calendar calendar, int precision) {
@@ -1230,7 +1233,7 @@ public class RexBuilder {
   }
 
   /**
-   * Creates a reference to a dynamic parameter
+   * Creates a reference to a dynamic parameter.
    *
    * @param type  Type of dynamic parameter
    * @param index Index of dynamic parameter
@@ -1260,12 +1263,14 @@ public class RexBuilder {
     return (RexLiteral) makeCast(type, constantNull);
   }
 
+  // CHECKSTYLE: IGNORE 1
   /** @deprecated Use {@link #makeNullLiteral(RelDataType)} */
   @Deprecated // to be removed before 2.0
   public RexNode makeNullLiteral(SqlTypeName typeName, int precision) {
     return makeNullLiteral(typeFactory.createSqlType(typeName, precision));
   }
 
+  // CHECKSTYLE: IGNORE 1
   /** @deprecated Use {@link #makeNullLiteral(RelDataType)} */
   @Deprecated // to be removed before 2.0
   public RexNode makeNullLiteral(SqlTypeName typeName) {
@@ -1629,7 +1634,7 @@ public class RexBuilder {
   }
 
   /** Returns a byte-string padded with zero bytes to make it at least a given
-   * length, */
+   * length. */
   private static ByteString padRight(ByteString s, int length) {
     if (s.length() >= length) {
       return s;
diff --git a/core/src/main/java/org/apache/calcite/rex/RexLiteral.java b/core/src/main/java/org/apache/calcite/rex/RexLiteral.java
index f40d126..4195966 100644
--- a/core/src/main/java/org/apache/calcite/rex/RexLiteral.java
+++ b/core/src/main/java/org/apache/calcite/rex/RexLiteral.java
@@ -282,10 +282,8 @@ public class RexLiteral extends RexNode {
     return shouldIncludeType(value, type);
   }
 
-  /**
-   * @return whether value is appropriate for its type (we have rules about
-   * these things)
-   */
+  /** Returns whether a value is appropriate for its type. (We have rules about
+   * these things!) */
   public static boolean valueMatchesType(
       Comparable value,
       SqlTypeName typeName,
diff --git a/core/src/main/java/org/apache/calcite/rex/RexMultisetUtil.java b/core/src/main/java/org/apache/calcite/rex/RexMultisetUtil.java
index 971cf47..f0606e2 100644
--- a/core/src/main/java/org/apache/calcite/rex/RexMultisetUtil.java
+++ b/core/src/main/java/org/apache/calcite/rex/RexMultisetUtil.java
@@ -32,7 +32,7 @@ public class RexMultisetUtil {
   //~ Static fields/initializers ---------------------------------------------
 
   /**
-   * A set defining all implementable multiset calls
+   * A set defining all implementable multiset calls.
    */
   private static final Set<SqlOperator> MULTISET_OPERATORS =
       ImmutableSet.of(
@@ -134,7 +134,7 @@ public class RexMultisetUtil {
 
   /**
    * Returns a reference to the first found multiset call or null if none was
-   * found
+   * found.
    */
   public static RexCall findFirstMultiset(final RexNode node, boolean deep) {
     if (node instanceof RexFieldAccess) {
diff --git a/core/src/main/java/org/apache/calcite/rex/RexPatternFieldRef.java b/core/src/main/java/org/apache/calcite/rex/RexPatternFieldRef.java
index a0d039e..560f88f 100644
--- a/core/src/main/java/org/apache/calcite/rex/RexPatternFieldRef.java
+++ b/core/src/main/java/org/apache/calcite/rex/RexPatternFieldRef.java
@@ -20,7 +20,7 @@ import org.apache.calcite.rel.type.RelDataType;
 import org.apache.calcite.sql.SqlKind;
 
 /**
- * Variable which references a field of an input relational expression
+ * Variable that references a field of an input relational expression.
  */
 public class RexPatternFieldRef extends RexInputRef {
   private final String alpha;
diff --git a/core/src/main/java/org/apache/calcite/rex/RexProgram.java b/core/src/main/java/org/apache/calcite/rex/RexProgram.java
index 7369217..24e4194 100644
--- a/core/src/main/java/org/apache/calcite/rex/RexProgram.java
+++ b/core/src/main/java/org/apache/calcite/rex/RexProgram.java
@@ -401,7 +401,7 @@ public class RexProgram {
   }
 
   /**
-   * Returns whether this program contains windowed aggregate functions
+   * Returns whether this program contains windowed aggregate functions.
    *
    * @return whether this program contains windowed aggregate functions
    */
diff --git a/core/src/main/java/org/apache/calcite/rex/RexProgramBuilder.java b/core/src/main/java/org/apache/calcite/rex/RexProgramBuilder.java
index 71ac25c..488b2a0 100644
--- a/core/src/main/java/org/apache/calcite/rex/RexProgramBuilder.java
+++ b/core/src/main/java/org/apache/calcite/rex/RexProgramBuilder.java
@@ -867,7 +867,7 @@ public class RexProgramBuilder {
   }
 
   /**
-   * Returns the rowtype of the input to the program
+   * Returns the row type of the input to the program.
    */
   public RelDataType getInputRowType() {
     return inputRowType;
@@ -1045,8 +1045,8 @@ public class RexProgramBuilder {
   }
 
   /**
-   * Shuttle which rewires {@link RexLocalRef} using a list of updated
-   * references
+   * Shuttle that rewires {@link RexLocalRef} using a list of updated
+   * references.
    */
   private class UpdateRefShuttle extends RexShuttle {
     private List<RexLocalRef> newRefs;
diff --git a/core/src/main/java/org/apache/calcite/rex/RexSimplify.java b/core/src/main/java/org/apache/calcite/rex/RexSimplify.java
index 67af8f1..35ded74 100644
--- a/core/src/main/java/org/apache/calcite/rex/RexSimplify.java
+++ b/core/src/main/java/org/apache/calcite/rex/RexSimplify.java
@@ -1032,7 +1032,7 @@ public class RexSimplify {
             && oldType.isNullable());
   }
 
-  /** Object to describe a Case branch */
+  /** Object to describe a CASE branch. */
   static final class CaseBranch {
 
     private final RexNode cond;
@@ -1074,7 +1074,7 @@ public class RexSimplify {
   }
 
   /**
-   * Decides whether it is safe to flatten the given case part into AND/ORs
+   * Decides whether it is safe to flatten the given CASE part into ANDs/ORs.
    */
   enum SafeRexVisitor implements RexVisitor<Boolean> {
     INSTANCE;
diff --git a/core/src/main/java/org/apache/calcite/rex/RexSqlReflectiveConvertletTable.java b/core/src/main/java/org/apache/calcite/rex/RexSqlReflectiveConvertletTable.java
index c19d540..65d4fc6 100644
--- a/core/src/main/java/org/apache/calcite/rex/RexSqlReflectiveConvertletTable.java
+++ b/core/src/main/java/org/apache/calcite/rex/RexSqlReflectiveConvertletTable.java
@@ -72,7 +72,7 @@ public class RexSqlReflectiveConvertletTable implements RexSqlConvertletTable {
   }
 
   /**
-   * Registers a convertlet for a given operator instance
+   * Registers a convertlet for a given operator instance.
    *
    * @param op         Operator instance, say
    * {@link org.apache.calcite.sql.fun.SqlStdOperatorTable#MINUS}
diff --git a/core/src/main/java/org/apache/calcite/rex/RexTableInputRef.java b/core/src/main/java/org/apache/calcite/rex/RexTableInputRef.java
index fd14ede..6ebf65b 100644
--- a/core/src/main/java/org/apache/calcite/rex/RexTableInputRef.java
+++ b/core/src/main/java/org/apache/calcite/rex/RexTableInputRef.java
@@ -94,7 +94,8 @@ public class RexTableInputRef extends RexInputRef {
     return SqlKind.TABLE_INPUT_REF;
   }
 
-  /** Identifies uniquely a table by its qualified name and its entity number (occurrence) */
+  /** Identifies uniquely a table by its qualified name and its entity number
+   * (occurrence). */
   public static class RelTableRef implements Comparable<RelTableRef> {
 
     private final RelOptTable table;
diff --git a/core/src/main/java/org/apache/calcite/rex/RexUtil.java b/core/src/main/java/org/apache/calcite/rex/RexUtil.java
index 67d60a5..e1aff76 100644
--- a/core/src/main/java/org/apache/calcite/rex/RexUtil.java
+++ b/core/src/main/java/org/apache/calcite/rex/RexUtil.java
@@ -94,7 +94,7 @@ public class RexUtil {
   }
 
   /**
-   * Generates a cast from one row type to another
+   * Generates a cast from one row type to another.
    *
    * @param rexBuilder RexBuilder to use for constructing casts
    * @param lhsRowType target row type
@@ -629,7 +629,7 @@ public class RexUtil {
   }
 
    /**
-   * Returns whether a given node contains a RexCall with a specified operator
+   * Returns whether a given node contains a RexCall with a specified operator.
    *
    * @param operator Operator to look for
    * @param node     a RexNode tree
@@ -766,7 +766,7 @@ public class RexUtil {
   }
 
   /**
-   * Determines whether any operand of a set requires decimal expansion
+   * Determines whether any operand of a set requires decimal expansion.
    */
   public static boolean requiresDecimalExpansion(
       List<RexNode> operands,
diff --git a/core/src/main/java/org/apache/calcite/runtime/CalciteContextException.java b/core/src/main/java/org/apache/calcite/runtime/CalciteContextException.java
index fdedfe6..0b269d1 100644
--- a/core/src/main/java/org/apache/calcite/runtime/CalciteContextException.java
+++ b/core/src/main/java/org/apache/calcite/runtime/CalciteContextException.java
@@ -132,43 +132,44 @@ public class CalciteContextException extends CalciteException {
   }
 
   /**
-   * @return 1-based line number, or 0 for missing position information
+   * Returns the 1-based line number, or 0 for missing position information.
    */
   public int getPosLine() {
     return posLine;
   }
 
   /**
-   * @return 1-based column number, or 0 for missing position information
+   * Returns the 1-based column number, or 0 for missing position information.
    */
   public int getPosColumn() {
     return posColumn;
   }
 
   /**
-   * @return 1-based ending line number, or 0 for missing position information
+   * Returns the 1-based ending line number, or 0 for missing position
+   * information.
    */
   public int getEndPosLine() {
     return endPosLine;
   }
 
   /**
-   * @return 1-based ending column number, or 0 for missing position
-   * information
+   * Returns the 1-based ending column number, or 0 for missing position
+   * information.
    */
   public int getEndPosColumn() {
     return endPosColumn;
   }
 
   /**
-   * @return the input string that is associated with the context
+   * Returns the input string that is associated with the context.
    */
   public String getOriginalStatement() {
     return originalStatement;
   }
 
   /**
-   * @param originalStatement - String to associate with the current context
+   * Sets the input string to associate with the current context.
    */
   public void setOriginalStatement(String originalStatement) {
     this.originalStatement = originalStatement;
diff --git a/core/src/main/java/org/apache/calcite/runtime/Enumerables.java b/core/src/main/java/org/apache/calcite/runtime/Enumerables.java
index a9c3f82..1114391 100644
--- a/core/src/main/java/org/apache/calcite/runtime/Enumerables.java
+++ b/core/src/main/java/org/apache/calcite/runtime/Enumerables.java
@@ -91,7 +91,7 @@ public class Enumerables {
           /** Current result row. Null if no row is ready. */
           TResult resultRow;
 
-          /** Match counter is 1 based in Oracle */
+          /** Match counter is 1-based in Oracle. */
           final AtomicInteger matchCounter = new AtomicInteger(1);
 
           public TResult current() {
diff --git a/core/src/main/java/org/apache/calcite/runtime/EnumeratorCursor.java b/core/src/main/java/org/apache/calcite/runtime/EnumeratorCursor.java
index c1a12b6..042f88c 100644
--- a/core/src/main/java/org/apache/calcite/runtime/EnumeratorCursor.java
+++ b/core/src/main/java/org/apache/calcite/runtime/EnumeratorCursor.java
@@ -35,7 +35,8 @@ public abstract class EnumeratorCursor<T> extends PositionedCursor<T> {
   private final Enumerator<T> enumerator;
 
   /**
-   * Creates a {@code EnumeratorCursor}
+   * Creates an {@code EnumeratorCursor}.
+   *
    * @param enumerator input enumerator
    */
   protected EnumeratorCursor(Enumerator<T> enumerator) {
diff --git a/core/src/main/java/org/apache/calcite/runtime/Hook.java b/core/src/main/java/org/apache/calcite/runtime/Hook.java
index 372572f..ce681f3 100644
--- a/core/src/main/java/org/apache/calcite/runtime/Hook.java
+++ b/core/src/main/java/org/apache/calcite/runtime/Hook.java
@@ -130,6 +130,7 @@ public enum Hook {
     return () -> remove(handler);
   }
 
+  // CHECKSTYLE: IGNORE 1
   /** @deprecated Use {@link #add(Consumer)}. */
   @SuppressWarnings("Guava")
   @Deprecated // to be removed before 2.0
@@ -149,6 +150,7 @@ public enum Hook {
     return () -> removeThread(handler);
   }
 
+  // CHECKSTYLE: IGNORE 1
   /** @deprecated Use {@link #addThread(Consumer)}. */
   @SuppressWarnings("Guava")
   @Deprecated // to be removed before 2.0
@@ -162,6 +164,7 @@ public enum Hook {
     return threadHandlers.get().remove(handler);
   }
 
+  // CHECKSTYLE: IGNORE 1
   /** @deprecated Use {@link #propertyJ}. */
   @SuppressWarnings("Guava")
   @Deprecated // return type will change in 2.0
diff --git a/core/src/main/java/org/apache/calcite/runtime/Matcher.java b/core/src/main/java/org/apache/calcite/runtime/Matcher.java
index 3d835cb..627c6f5 100644
--- a/core/src/main/java/org/apache/calcite/runtime/Matcher.java
+++ b/core/src/main/java/org/apache/calcite/runtime/Matcher.java
@@ -320,7 +320,7 @@ public class Matcher<E> {
   }
 
   /**
-   * Represents a Tuple of a symbol and a row
+   * A 2-tuple consisting of a symbol and a row.
    *
    * @param <E> Type of Row
    */
diff --git a/core/src/main/java/org/apache/calcite/runtime/Pattern.java b/core/src/main/java/org/apache/calcite/runtime/Pattern.java
index d0ae8a2..1818fb6 100644
--- a/core/src/main/java/org/apache/calcite/runtime/Pattern.java
+++ b/core/src/main/java/org/apache/calcite/runtime/Pattern.java
@@ -38,9 +38,9 @@ public interface Pattern {
   enum Op {
     /** A leaf pattern, consisting of a single symbol. */
     SYMBOL(0, 0),
-    /** Anchor for start "^" */
+    /** Anchor for start "^". */
     ANCHOR_START(0, 0),
-    /** Anchor for end "$" */
+    /** Anchor for end "$". */
     ANCHOR_END(0, 0),
     /** Pattern that matches one pattern followed by another. */
     SEQ(2, -1),
@@ -53,7 +53,7 @@ public interface Pattern {
     /** Pattern that matches a pattern repeated between {@code minRepeat}
      * and {@code maxRepeat} times. */
     REPEAT(1, 1),
-    /** Pattern that machtes a pattern one time or zero times */
+    /** Pattern that matches a pattern one time or zero times. */
     OPTIONAL(1, 1);
 
     private final int minArity;
diff --git a/core/src/main/java/org/apache/calcite/runtime/SocketFactoryImpl.java b/core/src/main/java/org/apache/calcite/runtime/SocketFactoryImpl.java
index 6bd857d..857fd98 100644
--- a/core/src/main/java/org/apache/calcite/runtime/SocketFactoryImpl.java
+++ b/core/src/main/java/org/apache/calcite/runtime/SocketFactoryImpl.java
@@ -43,53 +43,53 @@ import javax.net.SocketFactory;
  */
 public class SocketFactoryImpl extends SocketFactory {
   /**
-   * should keep alives be sent
+   * Whether keep-alives should be sent.
    */
   public static final boolean SO_KEEPALIVE = false;
 
   /**
-   * is out of band in-line enabled
+   * Whether out-of-band in-line is enabled.
    */
   public static final boolean OOBINLINE = false;
 
   /**
-   * should the address be reused
+   * Whether the address should be reused.
    */
   public static final boolean SO_REUSEADDR = false;
 
   /**
-   * do not buffer send(s) iff true
+   * Whether to not buffer send(s).
    */
   public static final boolean TCP_NODELAY = true;
 
   /**
-   * size of receiving buffer
+   * Size of receiving buffer.
    */
   public static final int SO_RCVBUF = 8192;
 
   /**
-   * size of sending buffer iff needed
+   * Size of sending buffer iff needed.
    */
   public static final int SO_SNDBUF = 1024;
 
   /**
-   * read timeout in milliseconds
+   * Read timeout in milliseconds.
    */
   public static final int SO_TIMEOUT = 12000;
 
   /**
-   * connect timeout in milliseconds
+   * Connect timeout in milliseconds.
    */
   public static final int SO_CONNECT_TIMEOUT = 5000;
 
   /**
-   * enabling lingering with 0-timeout will cause the socket to be
-   * closed forcefully upon execution of close()
+   * Enabling lingering with 0-timeout will cause the socket to be
+   * closed forcefully upon execution of {@code close()}.
    */
   public static final boolean SO_LINGER = true;
 
   /**
-   * amount of time to linger
+   * Amount of time to linger.
    */
   public static final int LINGER = 0;
 
@@ -153,6 +153,8 @@ public class SocketFactoryImpl extends SocketFactory {
   }
 
   /**
+   * Returns a copy of the environment's default socket factory.
+   *
    * @see javax.net.SocketFactory#getDefault()
    */
   public static SocketFactory getDefault() {
diff --git a/core/src/main/java/org/apache/calcite/runtime/SqlFunctions.java b/core/src/main/java/org/apache/calcite/runtime/SqlFunctions.java
index 248990e..608af6f 100644
--- a/core/src/main/java/org/apache/calcite/runtime/SqlFunctions.java
+++ b/core/src/main/java/org/apache/calcite/runtime/SqlFunctions.java
@@ -1110,19 +1110,19 @@ public class SqlFunctions {
   }
 
   // &
-  /** Helper function for implementing <code>BIT_AND</code> */
+  /** Helper function for implementing <code>BIT_AND</code>. */
   public static long bitAnd(long b0, long b1) {
     return b0 & b1;
   }
 
   // |
-  /** Helper function for implementing <code>BIT_OR</code> */
+  /** Helper function for implementing <code>BIT_OR</code>. */
   public static long bitOr(long b0, long b1) {
     return b0 | b1;
   }
 
   // ^
-  /** Helper function for implementing <code>BIT_XOR</code> */
+  /** Helper function for implementing <code>BIT_XOR</code>. */
   public static long bitXor(long b0, long b1) {
     return b0 ^ b1;
   }
diff --git a/core/src/main/java/org/apache/calcite/runtime/TrustAllSslSocketFactory.java b/core/src/main/java/org/apache/calcite/runtime/TrustAllSslSocketFactory.java
index 3e212a4..9218bdb 100644
--- a/core/src/main/java/org/apache/calcite/runtime/TrustAllSslSocketFactory.java
+++ b/core/src/main/java/org/apache/calcite/runtime/TrustAllSslSocketFactory.java
@@ -76,6 +76,8 @@ public class TrustAllSslSocketFactory extends SocketFactoryImpl {
   }
 
   /**
+   * Returns a copy of the environment's default socket factory.
+   *
    * @see javax.net.SocketFactory#getDefault()
    */
   public static TrustAllSslSocketFactory getDefault() {
diff --git a/core/src/main/java/org/apache/calcite/runtime/Utilities.java b/core/src/main/java/org/apache/calcite/runtime/Utilities.java
index 68bfd3b..b09c91a 100644
--- a/core/src/main/java/org/apache/calcite/runtime/Utilities.java
+++ b/core/src/main/java/org/apache/calcite/runtime/Utilities.java
@@ -33,6 +33,7 @@ public class Utilities {
   protected Utilities() {
   }
 
+  // CHECKSTYLE: IGNORE 1
   /** @deprecated Use {@link java.util.Objects#equals}. */
   @Deprecated // to be removed before 2.0
   public static boolean equal(Object o0, Object o1) {
diff --git a/core/src/main/java/org/apache/calcite/schema/impl/ReflectiveFunctionBase.java b/core/src/main/java/org/apache/calcite/schema/impl/ReflectiveFunctionBase.java
index 82a966e..2ce8160 100644
--- a/core/src/main/java/org/apache/calcite/schema/impl/ReflectiveFunctionBase.java
+++ b/core/src/main/java/org/apache/calcite/schema/impl/ReflectiveFunctionBase.java
@@ -42,8 +42,9 @@ public abstract class ReflectiveFunctionBase implements Function {
   public final List<FunctionParameter> parameters;
 
   /**
-   * {@code ReflectiveFunctionBase} constructor
-   * @param method method that is used to get type information from
+   * Creates a ReflectiveFunctionBase.
+   *
+   * @param method Method that is used to get type information from
    */
   public ReflectiveFunctionBase(Method method) {
     this.method = method;
diff --git a/core/src/main/java/org/apache/calcite/sql/JoinConditionType.java b/core/src/main/java/org/apache/calcite/sql/JoinConditionType.java
index 421af6f..faf4ee8 100644
--- a/core/src/main/java/org/apache/calcite/sql/JoinConditionType.java
+++ b/core/src/main/java/org/apache/calcite/sql/JoinConditionType.java
@@ -21,19 +21,19 @@ package org.apache.calcite.sql;
  */
 public enum JoinConditionType implements Symbolizable {
   /**
-   * Join clause has no condition, for example "FROM EMP, DEPT"
+   * Join clause has no condition, for example "{@code FROM EMP, DEPT}".
    */
   NONE,
 
   /**
-   * Join clause has an ON condition, for example "FROM EMP JOIN DEPT ON
-   * EMP.DEPTNO = DEPT.DEPTNO"
+   * Join clause has an {@code ON} condition,
+   * for example "{@code FROM EMP JOIN DEPT ON EMP.DEPTNO = DEPT.DEPTNO}".
    */
   ON,
 
   /**
-   * Join clause has a USING condition, for example "FROM EMP JOIN DEPT
-   * USING (DEPTNO)"
+   * Join clause has a {@code USING} condition,
+   * for example "{@code FROM EMP JOIN DEPT USING (DEPTNO)}".
    */
   USING
 }
diff --git a/core/src/main/java/org/apache/calcite/sql/SqlAccessEnum.java b/core/src/main/java/org/apache/calcite/sql/SqlAccessEnum.java
index 7abf1f7..80d9411 100644
--- a/core/src/main/java/org/apache/calcite/sql/SqlAccessEnum.java
+++ b/core/src/main/java/org/apache/calcite/sql/SqlAccessEnum.java
@@ -17,7 +17,7 @@
 package org.apache.calcite.sql;
 
 /**
- * Enumeration representing different access types
+ * Access type.
  */
 public enum SqlAccessEnum {
   SELECT, UPDATE, INSERT, DELETE;
diff --git a/core/src/main/java/org/apache/calcite/sql/SqlAccessType.java b/core/src/main/java/org/apache/calcite/sql/SqlAccessType.java
index b5e0466..7f9ee24 100644
--- a/core/src/main/java/org/apache/calcite/sql/SqlAccessType.java
+++ b/core/src/main/java/org/apache/calcite/sql/SqlAccessType.java
@@ -20,7 +20,9 @@ import java.util.EnumSet;
 import java.util.Locale;
 
 /**
- * SqlAccessType is represented by a set of allowed access types
+ * SqlAccessType is represented by a set of allowed access types.
+ *
+ * @see SqlAccessEnum
  */
 public class SqlAccessType {
   //~ Static fields/initializers ---------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/sql/SqlBasicTypeNameSpec.java b/core/src/main/java/org/apache/calcite/sql/SqlBasicTypeNameSpec.java
index 01358fb..4e3d7db 100644
--- a/core/src/main/java/org/apache/calcite/sql/SqlBasicTypeNameSpec.java
+++ b/core/src/main/java/org/apache/calcite/sql/SqlBasicTypeNameSpec.java
@@ -235,7 +235,7 @@ public class SqlBasicTypeNameSpec extends SqlTypeNameSpec {
   //~ Tools ------------------------------------------------------------------
 
   /**
-   * @return true if this type name has "local time zone" definition.
+   * Returns whether this type name has "local time zone" definition.
    */
   private static boolean isWithLocalTimeZoneDef(SqlTypeName typeName) {
     switch (typeName) {
diff --git a/core/src/main/java/org/apache/calcite/sql/SqlBinaryStringLiteral.java b/core/src/main/java/org/apache/calcite/sql/SqlBinaryStringLiteral.java
index a2f2433..9c225da 100644
--- a/core/src/main/java/org/apache/calcite/sql/SqlBinaryStringLiteral.java
+++ b/core/src/main/java/org/apache/calcite/sql/SqlBinaryStringLiteral.java
@@ -26,8 +26,8 @@ import java.util.List;
 /**
  * A binary (or hexadecimal) string literal.
  *
- * <p>The {@link #value} field is a {@link BitString} and {@link #typeName} is
- * {@link SqlTypeName#BINARY}.
+ * <p>The {@link #value} field is a {@link BitString} and {@link #getTypeName()}
+ * is {@link SqlTypeName#BINARY}.
  */
 public class SqlBinaryStringLiteral extends SqlAbstractStringLiteral {
 
@@ -41,9 +41,11 @@ public class SqlBinaryStringLiteral extends SqlAbstractStringLiteral {
 
   //~ Methods ----------------------------------------------------------------
 
-  /**
-   * @return the underlying BitString
+  /** Returns the underlying {@link BitString}.
+   *
+   * @deprecated Use {@link SqlLiteral#getValueAs getValueAs(BitString.class)}
    */
+  @Deprecated // to be removed before 2.0
   public BitString getBitString() {
     return (BitString) value;
   }
@@ -64,7 +66,7 @@ public class SqlBinaryStringLiteral extends SqlAbstractStringLiteral {
     return new SqlBinaryStringLiteral(
         BitString.concat(
             Util.transform(literals,
-                literal -> ((SqlBinaryStringLiteral) literal).getBitString())),
+                literal -> literal.getValueAs(BitString.class))),
         literals.get(0).getParserPosition());
   }
 }
diff --git a/core/src/main/java/org/apache/calcite/sql/SqlCall.java b/core/src/main/java/org/apache/calcite/sql/SqlCall.java
index e425e16..8678c7e 100755
--- a/core/src/main/java/org/apache/calcite/sql/SqlCall.java
+++ b/core/src/main/java/org/apache/calcite/sql/SqlCall.java
@@ -183,9 +183,9 @@ public abstract class SqlCall extends SqlNode {
   }
 
   /**
-   * Test to see if it is the function COUNT(*)
+   * Returns whether it is the function {@code COUNT(*)}.
    *
-   * @return boolean true if function call to COUNT(*)
+   * @return true if function call to COUNT(*)
    */
   public boolean isCountStar() {
     SqlOperator sqlOperator = getOperator();
diff --git a/core/src/main/java/org/apache/calcite/sql/SqlCharStringLiteral.java b/core/src/main/java/org/apache/calcite/sql/SqlCharStringLiteral.java
index bf3e43b..15c7c6d 100644
--- a/core/src/main/java/org/apache/calcite/sql/SqlCharStringLiteral.java
+++ b/core/src/main/java/org/apache/calcite/sql/SqlCharStringLiteral.java
@@ -41,17 +41,20 @@ public class SqlCharStringLiteral extends SqlAbstractStringLiteral {
   //~ Methods ----------------------------------------------------------------
 
   /**
-   * @return the underlying NlsString
+   * Returns the underlying NlsString.
+   *
+   * @deprecated Use {@link #getValueAs getValueAs(NlsString.class)}
    */
+  @Deprecated // to be removed before 2.0
   public NlsString getNlsString() {
     return (NlsString) value;
   }
 
   /**
-   * @return the collation
+   * Returns the collation.
    */
   public SqlCollation getCollation() {
-    return getNlsString().getCollation();
+    return ((NlsString) value).getCollation();
   }
 
   @Override public SqlCharStringLiteral clone(SqlParserPos pos) {
@@ -77,7 +80,7 @@ public class SqlCharStringLiteral extends SqlAbstractStringLiteral {
     return new SqlCharStringLiteral(
         NlsString.concat(
             Util.transform(literals,
-                literal -> ((SqlCharStringLiteral) literal).getNlsString())),
+                literal -> literal.getValueAs(NlsString.class))),
         literals.get(0).getParserPosition());
   }
 }
diff --git a/core/src/main/java/org/apache/calcite/sql/SqlCollation.java b/core/src/main/java/org/apache/calcite/sql/SqlCollation.java
index f1e3366..cadc89e 100644
--- a/core/src/main/java/org/apache/calcite/sql/SqlCollation.java
+++ b/core/src/main/java/org/apache/calcite/sql/SqlCollation.java
@@ -301,9 +301,9 @@ public class SqlCollation implements Serializable {
   }
 
   /**
-   * @return the {@link Collator} to compare values having the current collation,
-   * or {@code null} if no specific {@link Collator} is needed, in which case
-   * {@link String#compareTo} will be used.
+   * Returns the {@link Collator} to compare values having the current
+   * collation, or {@code null} if no specific {@link Collator} is needed, in
+   * which case {@link String#compareTo} will be used.
    */
   public Collator getCollator() {
     return null;
diff --git a/core/src/main/java/org/apache/calcite/sql/SqlDelete.java b/core/src/main/java/org/apache/calcite/sql/SqlDelete.java
index d4aa732..e9772f1 100644
--- a/core/src/main/java/org/apache/calcite/sql/SqlDelete.java
+++ b/core/src/main/java/org/apache/calcite/sql/SqlDelete.java
@@ -86,14 +86,14 @@ public class SqlDelete extends SqlCall {
   }
 
   /**
-   * @return the identifier for the target table of the deletion
+   * Returns the identifier for the target table of the deletion.
    */
   public SqlNode getTargetTable() {
     return targetTable;
   }
 
   /**
-   * @return the alias for the target table of the deletion
+   * Returns the alias for the target table of the deletion.
    */
   public SqlIdentifier getAlias() {
     return alias;
diff --git a/core/src/main/java/org/apache/calcite/sql/SqlExplain.java b/core/src/main/java/org/apache/calcite/sql/SqlExplain.java
index caa434f..79ccc5f 100644
--- a/core/src/main/java/org/apache/calcite/sql/SqlExplain.java
+++ b/core/src/main/java/org/apache/calcite/sql/SqlExplain.java
@@ -102,14 +102,14 @@ public class SqlExplain extends SqlCall {
   }
 
   /**
-   * @return the underlying SQL statement to be explained
+   * Returns the underlying SQL statement to be explained.
    */
   public SqlNode getExplicandum() {
     return explicandum;
   }
 
   /**
-   * @return detail level to be generated
+   * Return the detail level to be generated.
    */
   public SqlExplainLevel getDetailLevel() {
     return detailLevel.symbolValue(SqlExplainLevel.class);
@@ -123,21 +123,21 @@ public class SqlExplain extends SqlCall {
   }
 
   /**
-   * @return the number of dynamic parameters in the statement
+   * Returns the number of dynamic parameters in the statement.
    */
   public int getDynamicParamCount() {
     return dynamicParameterCount;
   }
 
   /**
-   * @return whether physical plan implementation should be returned
+   * Returns whether physical plan implementation should be returned.
    */
   public boolean withImplementation() {
     return getDepth() == Depth.PHYSICAL;
   }
 
   /**
-   * @return whether type should be returned
+   * Returns whether type should be returned.
    */
   public boolean withType() {
     return getDepth() == Depth.TYPE;
diff --git a/core/src/main/java/org/apache/calcite/sql/SqlFunction.java b/core/src/main/java/org/apache/calcite/sql/SqlFunction.java
index 55706d4..fa6b180 100644
--- a/core/src/main/java/org/apache/calcite/sql/SqlFunction.java
+++ b/core/src/main/java/org/apache/calcite/sql/SqlFunction.java
@@ -53,9 +53,9 @@ public class SqlFunction extends SqlOperator {
   //~ Constructors -----------------------------------------------------------
 
   /**
-   * Creates a new SqlFunction for a call to a builtin function.
+   * Creates a new SqlFunction for a call to a built-in function.
    *
-   * @param name                 Name of builtin function
+   * @param name                 Name of built-in function
    * @param kind                 kind of operator implemented by function
    * @param returnTypeInference  strategy to use for return type inference
    * @param operandTypeInference strategy to use for parameter type inference
@@ -70,7 +70,7 @@ public class SqlFunction extends SqlOperator {
       SqlOperandTypeChecker operandTypeChecker,
       SqlFunctionCategory category) {
     // We leave sqlIdentifier as null to indicate
-    // that this is a builtin.  Same for paramTypes.
+    // that this is a built-in.  Same for paramTypes.
     this(name, null, kind, returnTypeInference, operandTypeInference,
         operandTypeChecker, null, category);
 
@@ -80,7 +80,7 @@ public class SqlFunction extends SqlOperator {
 
   /**
    * Creates a placeholder SqlFunction for an invocation of a function with a
-   * possibly qualified name. This name must be resolved into either a builtin
+   * possibly qualified name. This name must be resolved into either a built-in
    * function or a user-defined function.
    *
    * @param sqlIdentifier        possibly qualified identifier for function
@@ -130,7 +130,8 @@ public class SqlFunction extends SqlOperator {
   }
 
   /**
-   * @return fully qualified name of function, or null for a builtin function
+   * Returns the fully-qualified name of function, or null for a built-in
+   * function.
    */
   public SqlIdentifier getSqlIdentifier() {
     return sqlIdentifier;
@@ -144,7 +145,7 @@ public class SqlFunction extends SqlOperator {
   }
 
   /**
-   * @return array of parameter types, or null for builtin function
+   * Return array of parameter types, or null for built-in function.
    */
   public List<RelDataType> getParamTypes() {
     return paramTypes;
@@ -168,7 +169,7 @@ public class SqlFunction extends SqlOperator {
   }
 
   /**
-   * @return function category
+   * Return function category.
    */
   @Nonnull public SqlFunctionCategory getFunctionType() {
     return this.category;
diff --git a/core/src/main/java/org/apache/calcite/sql/SqlInsert.java b/core/src/main/java/org/apache/calcite/sql/SqlInsert.java
index 722d0ce..b3dccd7 100644
--- a/core/src/main/java/org/apache/calcite/sql/SqlInsert.java
+++ b/core/src/main/java/org/apache/calcite/sql/SqlInsert.java
@@ -105,14 +105,14 @@ public class SqlInsert extends SqlCall {
   }
 
   /**
-   * @return the identifier for the target table of the insertion
+   * Return the identifier for the target table of the insertion.
    */
   public SqlNode getTargetTable() {
     return targetTable;
   }
 
   /**
-   * @return the source expression for the data to be inserted
+   * Returns the source expression for the data to be inserted.
    */
   public SqlNode getSource() {
     return source;
@@ -123,8 +123,8 @@ public class SqlInsert extends SqlCall {
   }
 
   /**
-   * @return the list of target column names, or null for all columns in the
-   * target table
+   * Returns the list of target column names, or null for all columns in the
+   * target table.
    */
   public SqlNodeList getTargetColumnList() {
     return columnList;
diff --git a/core/src/main/java/org/apache/calcite/sql/SqlIntervalQualifier.java b/core/src/main/java/org/apache/calcite/sql/SqlIntervalQualifier.java
index 20baf1e..032b5db 100644
--- a/core/src/main/java/org/apache/calcite/sql/SqlIntervalQualifier.java
+++ b/core/src/main/java/org/apache/calcite/sql/SqlIntervalQualifier.java
@@ -328,7 +328,7 @@ public class SqlIntervalQualifier extends SqlNode {
   }
 
   /**
-   * @return 1 or -1
+   * Returns 1 or -1.
    */
   public int getIntervalSign(String value) {
     int sign = 1; // positive until proven otherwise
diff --git a/core/src/main/java/org/apache/calcite/sql/SqlJdbcFunctionCall.java b/core/src/main/java/org/apache/calcite/sql/SqlJdbcFunctionCall.java
index fb727b8..c04babc 100644
--- a/core/src/main/java/org/apache/calcite/sql/SqlJdbcFunctionCall.java
+++ b/core/src/main/java/org/apache/calcite/sql/SqlJdbcFunctionCall.java
@@ -528,28 +528,28 @@ public class SqlJdbcFunctionCall extends SqlFunction {
   }
 
   /**
-   * @see java.sql.DatabaseMetaData#getNumericFunctions
+   * As {@link java.sql.DatabaseMetaData#getNumericFunctions}.
    */
   public static String getNumericFunctions() {
     return NUMERIC_FUNCTIONS;
   }
 
   /**
-   * @see java.sql.DatabaseMetaData#getStringFunctions
+   * As {@link java.sql.DatabaseMetaData#getStringFunctions}.
    */
   public static String getStringFunctions() {
     return STRING_FUNCTIONS;
   }
 
   /**
-   * @see java.sql.DatabaseMetaData#getTimeDateFunctions
+   * As {@link java.sql.DatabaseMetaData#getTimeDateFunctions}.
    */
   public static String getTimeDateFunctions() {
     return TIME_DATE_FUNCTIONS;
   }
 
   /**
-   * @see java.sql.DatabaseMetaData#getSystemFunctions
+   * As {@link java.sql.DatabaseMetaData#getSystemFunctions}.
    */
   public static String getSystemFunctions() {
     return SYSTEM_FUNCTIONS;
@@ -657,7 +657,7 @@ public class SqlJdbcFunctionCall extends SqlFunction {
   }
 
   /**
-   * Lookup table between JDBC functions and internal representation
+   * Lookup table between JDBC functions and internal representation.
    */
   private static class JdbcToInternalLookupTable {
     /**
diff --git a/core/src/main/java/org/apache/calcite/sql/SqlJsonConstructorNullClause.java b/core/src/main/java/org/apache/calcite/sql/SqlJsonConstructorNullClause.java
index 6415fd7..08f9e4c 100644
--- a/core/src/main/java/org/apache/calcite/sql/SqlJsonConstructorNullClause.java
+++ b/core/src/main/java/org/apache/calcite/sql/SqlJsonConstructorNullClause.java
@@ -17,7 +17,7 @@
 package org.apache.calcite.sql;
 
 /**
- * Indicating that how do Json constructors handle null
+ * Indicating how JSON constructors handle null.
  */
 public enum SqlJsonConstructorNullClause {
   NULL_ON_NULL("NULL ON NULL"),
diff --git a/core/src/main/java/org/apache/calcite/sql/SqlKind.java b/core/src/main/java/org/apache/calcite/sql/SqlKind.java
index 247c985..adb3223 100644
--- a/core/src/main/java/org/apache/calcite/sql/SqlKind.java
+++ b/core/src/main/java/org/apache/calcite/sql/SqlKind.java
@@ -114,14 +114,10 @@ public enum SqlKind {
    */
   JOIN,
 
-  /**
-   * Identifier
-   */
+  /** An identifier. */
   IDENTIFIER,
 
-  /**
-   * A literal.
-   */
+  /** A literal. */
   LITERAL,
 
   /**
@@ -131,49 +127,31 @@ public enum SqlKind {
    */
   OTHER_FUNCTION,
 
-  /**
-   * POSITION Function
-   */
+  /** POSITION function. */
   POSITION,
 
-  /**
-   * EXPLAIN statement
-   */
+  /** EXPLAIN statement. */
   EXPLAIN,
 
-  /**
-   * DESCRIBE SCHEMA statement
-   */
+  /** DESCRIBE SCHEMA statement. */
   DESCRIBE_SCHEMA,
 
-  /**
-   * DESCRIBE TABLE statement
-   */
+  /** DESCRIBE TABLE statement. */
   DESCRIBE_TABLE,
 
-  /**
-   * INSERT statement
-   */
+  /** INSERT statement. */
   INSERT,
 
-  /**
-   * DELETE statement
-   */
+  /** DELETE statement. */
   DELETE,
 
-  /**
-   * UPDATE statement
-   */
+  /** UPDATE statement. */
   UPDATE,
 
-  /**
-   * "ALTER scope SET option = value" statement.
-   */
+  /** "{@code ALTER scope SET option = value}" statement. */
   SET_OPTION,
 
-  /**
-   * A dynamic parameter.
-   */
+  /** A dynamic parameter. */
   DYNAMIC_PARAM,
 
   /**
@@ -191,347 +169,239 @@ public enum SqlKind {
   /** Item in WITH clause. */
   WITH_ITEM,
 
-  /** Item expression */
+  /** Item expression. */
   ITEM,
 
-  /**
-   * Union
-   */
+  /** {@code UNION} relational operator. */
   UNION,
 
-  /**
-   * Except
-   */
+  /** {@code EXCEPT} relational operator (known as {@code MINUS} in some SQL
+   * dialects). */
   EXCEPT,
 
-  /**
-   * Intersect
-   */
+  /** {@code INTERSECT} relational operator. */
   INTERSECT,
 
-  /**
-   * AS operator
-   */
+  /** {@code AS} operator. */
   AS,
 
-  /**
-   * ARGUMENT_ASSIGNMENT operator, {@code =>}
-   */
+  /** Argument assignment operator, {@code =>}. */
   ARGUMENT_ASSIGNMENT,
 
-  /**
-   * DEFAULT operator
-   */
+  /** {@code DEFAULT} operator. */
   DEFAULT,
 
-  /**
-   * OVER operator
-   */
+  /** {@code OVER} operator. */
   OVER,
 
-  /**
-   * RESPECT NULLS operator
-   */
+  /** {@code RESPECT NULLS} operator. */
   RESPECT_NULLS("RESPECT NULLS"),
 
-  /**
-   * IGNORE NULLS operator
-   */
+  /** {@code IGNORE NULLS} operator. */
   IGNORE_NULLS("IGNORE NULLS"),
 
-  /**
-   * FILTER operator
-   */
+  /** {@code FILTER} operator. */
   FILTER,
 
-  /**
-   * WITHIN_GROUP operator
-   */
+  /** {@code WITHIN GROUP} operator. */
   WITHIN_GROUP,
 
-  /**
-   * Window specification
-   */
+  /** Window specification. */
   WINDOW,
 
-  /**
-   * MERGE statement
-   */
+  /** MERGE statement. */
   MERGE,
 
-  /**
-   * TABLESAMPLE operator
-   */
+  /** TABLESAMPLE relational operator. */
   TABLESAMPLE,
 
-  /**
-   * MATCH_RECOGNIZE clause
-   */
+  /** MATCH_RECOGNIZE clause. */
   MATCH_RECOGNIZE,
 
-  /**
-   * SNAPSHOT operator
-   */
+  /** SNAPSHOT operator. */
   SNAPSHOT,
 
   // binary operators
 
-  /**
-   * The arithmetic multiplication operator, "*".
-   */
+  /** Arithmetic multiplication operator, "*". */
   TIMES,
 
-  /**
-   * The arithmetic division operator, "/".
-   */
+  /** Arithmetic division operator, "/". */
   DIVIDE,
 
-  /**
-   * The arithmetic remainder operator, "MOD" (and "%" in some dialects).
-   */
+  /** Arithmetic remainder operator, "MOD" (and "%" in some dialects). */
   MOD,
 
   /**
-   * The arithmetic plus operator, "+".
+   * Arithmetic plus operator, "+".
    *
    * @see #PLUS_PREFIX
    */
   PLUS,
 
   /**
-   * The arithmetic minus operator, "-".
+   * Arithmetic minus operator, "-".
    *
    * @see #MINUS_PREFIX
    */
   MINUS,
 
   /**
-   * the alternation operator in a pattern expression within a match_recognize clause
+   * Alternation operator in a pattern expression within a
+   * {@code MATCH_RECOGNIZE} clause.
    */
   PATTERN_ALTER,
 
   /**
-   * the concatenation operator in a pattern expression within a match_recognize clause
+   * Concatenation operator in a pattern expression within a
+   * {@code MATCH_RECOGNIZE} clause.
    */
   PATTERN_CONCAT,
 
   // comparison operators
 
-  /**
-   * The "IN" operator.
-   */
+  /** {@code IN} operator. */
   IN,
 
   /**
-   * The "NOT IN" operator.
+   * {@code NOT IN} operator.
    *
    * <p>Only occurs in SqlNode trees. Is expanded to NOT(IN ...) before
    * entering RelNode land.
    */
   NOT_IN("NOT IN"),
 
-  /**
-   * The less-than operator, "&lt;".
-   */
+  /** Less-than operator, "&lt;". */
   LESS_THAN("<"),
 
-  /**
-   * The greater-than operator, "&gt;".
-   */
+  /** Greater-than operator, "&gt;". */
   GREATER_THAN(">"),
 
-  /**
-   * The less-than-or-equal operator, "&lt;=".
-   */
+  /** Less-than-or-equal operator, "&lt;=". */
   LESS_THAN_OR_EQUAL("<="),
 
-  /**
-   * The greater-than-or-equal operator, "&gt;=".
-   */
+  /** Greater-than-or-equal operator, "&gt;=". */
   GREATER_THAN_OR_EQUAL(">="),
 
-  /**
-   * The equals operator, "=".
-   */
+  /** Equals operator, "=". */
   EQUALS("="),
 
   /**
-   * The not-equals operator, "&#33;=" or "&lt;&gt;".
+   * Not-equals operator, "&#33;=" or "&lt;&gt;".
    * The latter is standard, and preferred.
    */
   NOT_EQUALS("<>"),
 
-  /**
-   * The is-distinct-from operator.
-   */
+  /** {@code IS DISTINCT FROM} operator. */
   IS_DISTINCT_FROM,
 
-  /**
-   * The is-not-distinct-from operator.
-   */
+  /** {@code IS NOT DISTINCT FROM} operator. */
   IS_NOT_DISTINCT_FROM,
 
-  /**
-   * The logical "OR" operator.
-   */
+  /** Logical "OR" operator. */
   OR,
 
-  /**
-   * The logical "AND" operator.
-   */
+  /** Logical "AND" operator. */
   AND,
 
   // other infix
 
-  /**
-   * Dot
-   */
+  /** Dot. */
   DOT,
 
-  /**
-   * The "OVERLAPS" operator for periods.
-   */
+  /** {@code OVERLAPS} operator for periods. */
   OVERLAPS,
 
-  /**
-   * The "CONTAINS" operator for periods.
-   */
+  /** {@code CONTAINS} operator for periods. */
   CONTAINS,
 
-  /**
-   * The "PRECEDES" operator for periods.
-   */
+  /** {@code PRECEDES} operator for periods. */
   PRECEDES,
 
-  /**
-   * The "IMMEDIATELY PRECEDES" operator for periods.
-   */
+  /** {@code IMMEDIATELY PRECEDES} operator for periods. */
   IMMEDIATELY_PRECEDES("IMMEDIATELY PRECEDES"),
 
-  /**
-   * The "SUCCEEDS" operator for periods.
-   */
+  /** {@code SUCCEEDS} operator for periods. */
   SUCCEEDS,
 
-  /**
-   * The "IMMEDIATELY SUCCEEDS" operator for periods.
-   */
+  /** {@code IMMEDIATELY SUCCEEDS} operator for periods. */
   IMMEDIATELY_SUCCEEDS("IMMEDIATELY SUCCEEDS"),
 
-  /**
-   * The "EQUALS" operator for periods.
-   */
+  /** {@code EQUALS} operator for periods. */
   PERIOD_EQUALS("EQUALS"),
 
-  /**
-   * The "LIKE" operator.
-   */
+  /** {@code LIKE} operator. */
   LIKE,
 
-  /**
-   * The "SIMILAR" operator.
-   */
+  /** {@code SIMILAR} operator. */
   SIMILAR,
 
-  /**
-   * The "~" operator.
-   */
+  /** {@code ~} operator (for POSIX-style regular expressions). */
   POSIX_REGEX_CASE_SENSITIVE,
 
-  /**
-   * The "~*" operator.
-   */
+  /** {@code ~*} operator (for case-insensitive POSIX-style regular
+   * expressions). */
   POSIX_REGEX_CASE_INSENSITIVE,
 
-  /**
-   * The "BETWEEN" operator.
-   */
+  /** {@code BETWEEN} operator. */
   BETWEEN,
 
-  /**
-   * A "CASE" expression.
-   */
+  /** {@code CASE} expression. */
   CASE,
 
-  /**
-   * The "NULLIF" operator.
-   */
+  /** {@code NULLIF} operator. */
   NULLIF,
 
-  /**
-   * The "COALESCE" operator.
-   */
+  /** {@code COALESCE} operator. */
   COALESCE,
 
-  /**
-   * The "DECODE" function (Oracle).
-   */
+  /** {@code DECODE} function (Oracle). */
   DECODE,
 
-  /**
-   * The "NVL" function (Oracle).
-   */
+  /** {@code NVL} function (Oracle). */
   NVL,
 
-  /**
-   * The "GREATEST" function (Oracle).
-   */
+  /** {@code GREATEST} function (Oracle). */
   GREATEST,
 
-  /**
-   * The "LEAST" function (Oracle).
-   */
+  /** {@code LEAST} function (Oracle). */
   LEAST,
 
-  /**
-   * The "TIMESTAMP_ADD" function (ODBC, SQL Server, MySQL).
-   */
+  /** {@code TIMESTAMP_ADD} function (ODBC, SQL Server, MySQL). */
   TIMESTAMP_ADD,
 
-  /**
-   * The "TIMESTAMP_DIFF" function (ODBC, SQL Server, MySQL).
-   */
+  /** {@code TIMESTAMP_DIFF} function (ODBC, SQL Server, MySQL). */
   TIMESTAMP_DIFF,
 
   // prefix operators
 
-  /**
-   * The logical "NOT" operator.
-   */
+  /** Logical {@code NOT} operator. */
   NOT,
 
   /**
-   * The unary plus operator, as in "+1".
+   * Unary plus operator, as in "+1".
    *
    * @see #PLUS
    */
   PLUS_PREFIX,
 
   /**
-   * The unary minus operator, as in "-1".
+   * Unary minus operator, as in "-1".
    *
    * @see #MINUS
    */
   MINUS_PREFIX,
 
-  /**
-   * The "EXISTS" operator.
-   */
+  /** {@code EXISTS} operator. */
   EXISTS,
 
-  /**
-   * The "SOME" quantification operator (also called "ANY").
-   */
+  /** {@code SOME} quantification operator (also called {@code ANY}). */
   SOME,
 
-  /**
-   * The "ALL" quantification operator.
-   */
+  /** {@code ALL} quantification operator. */
   ALL,
 
-  /**
-   * The "VALUES" operator.
-   */
+  /** {@code VALUES} relational operator. */
   VALUES,
 
   /**
@@ -546,110 +416,87 @@ public enum SqlKind {
    */
   SCALAR_QUERY,
 
-  /**
-   * ProcedureCall
-   */
+  /** Procedure call. */
   PROCEDURE_CALL,
 
-  /**
-   * NewSpecification
-   */
+  /** New specification. */
   NEW_SPECIFICATION,
 
+  // special functions in MATCH_RECOGNIZE
 
-  /**
-   * Special functions in MATCH_RECOGNIZE.
-   */
+  /** {@code FINAL} operator in {@code MATCH_RECOGNIZE}. */
   FINAL,
 
+  /** {@code FINAL} operator in {@code MATCH_RECOGNIZE}. */
   RUNNING,
 
+  /** {@code PREV} operator in {@code MATCH_RECOGNIZE}. */
   PREV,
 
+  /** {@code NEXT} operator in {@code MATCH_RECOGNIZE}. */
   NEXT,
 
+  /** {@code FIRST} operator in {@code MATCH_RECOGNIZE}. */
   FIRST,
 
+  /** {@code LAST} operator in {@code MATCH_RECOGNIZE}. */
   LAST,
 
+  /** {@code CLASSIFIER} operator in {@code MATCH_RECOGNIZE}. */
   CLASSIFIER,
 
+  /** {@code MATCH_NUMBER} operator in {@code MATCH_RECOGNIZE}. */
   MATCH_NUMBER,
 
-  /**
-   * The "SKIP TO FIRST" qualifier of restarting point in a MATCH_RECOGNIZE
-   * clause.
-   */
+  /** {@code SKIP TO FIRST} qualifier of restarting point in a
+   * {@code MATCH_RECOGNIZE} clause. */
   SKIP_TO_FIRST,
 
-  /**
-   * The "SKIP TO LAST" qualifier of restarting point in a MATCH_RECOGNIZE
-   * clause.
-   */
+  /** {@code SKIP TO LAST} qualifier of restarting point in a
+   * {@code MATCH_RECOGNIZE} clause. */
   SKIP_TO_LAST,
 
   // postfix operators
 
-  /**
-   * DESC in ORDER BY. A parse tree, not a true expression.
-   */
+  /** {@code DESC} operator in {@code ORDER BY}. A parse tree, not a true
+   * expression. */
   DESCENDING,
 
-  /**
-   * NULLS FIRST clause in ORDER BY. A parse tree, not a true expression.
-   */
+  /** {@code NULLS FIRST} clause in {@code ORDER BY}. A parse tree, not a true
+   * expression. */
   NULLS_FIRST,
 
-  /**
-   * NULLS LAST clause in ORDER BY. A parse tree, not a true expression.
-   */
+  /** {@code NULLS LAST} clause in {@code ORDER BY}. A parse tree, not a true
+   * expression. */
   NULLS_LAST,
 
-  /**
-   * The "IS TRUE" operator.
-   */
+  /** {@code IS TRUE} operator. */
   IS_TRUE,
 
-  /**
-   * The "IS FALSE" operator.
-   */
+  /** {@code IS FALSE} operator. */
   IS_FALSE,
 
-  /**
-   * The "IS NOT TRUE" operator.
-   */
+  /** {@code IS NOT TRUE} operator. */
   IS_NOT_TRUE,
 
-  /**
-   * The "IS NOT FALSE" operator.
-   */
+  /** {@code IS NOT FALSE} operator. */
   IS_NOT_FALSE,
 
-  /**
-   * The "IS UNKNOWN" operator.
-   */
+  /** {@code IS UNKNOWN} operator. */
   IS_UNKNOWN,
 
-  /**
-   * The "IS NULL" operator.
-   */
+  /** {@code IS NULL} operator. */
   IS_NULL,
 
-  /**
-   * The "IS NOT NULL" operator.
-   */
+  /** {@code IS NOT NULL} operator. */
   IS_NOT_NULL,
 
-  /**
-   * The "PRECEDING" qualifier of an interval end-point in a window
-   * specification.
-   */
+  /** {@code PRECEDING} qualifier of an interval end-point in a window
+   * specification. */
   PRECEDING,
 
-  /**
-   * The "FOLLOWING" qualifier of an interval end-point in a window
-   * specification.
-   */
+  /** {@code FOLLOWING} qualifier of an interval end-point in a window
+   * specification. */
   FOLLOWING,
 
   /**
@@ -668,14 +515,14 @@ public enum SqlKind {
   INPUT_REF,
 
   /**
-   * Reference to an input field, with a qualified name and an identifier
+   * Reference to an input field, with a qualified name and an identifier.
    *
    * <p>(Only used at the RexNode level.)</p>
    */
   TABLE_INPUT_REF,
 
   /**
-   * Reference to an input field, with pattern var as modifier
+   * Reference to an input field, with pattern var as modifier.
    *
    * <p>(Only used at the RexNode level.)</p>
    */
@@ -730,74 +577,46 @@ public enum SqlKind {
    */
   CURRENT_VALUE,
 
-  /**
-   * The "FLOOR" function
-   */
+  /** {@code FLOOR} function. */
   FLOOR,
 
-  /**
-   * The "CEIL" function
-   */
+  /** {@code CEIL} function. */
   CEIL,
 
-  /**
-   * The "TRIM" function.
-   */
+  /** {@code TRIM} function. */
   TRIM,
 
-  /**
-   * The "LTRIM" function (Oracle).
-   */
+  /** {@code LTRIM} function (Oracle). */
   LTRIM,
 
-  /**
-   * The "RTRIM" function (Oracle).
-   */
+  /** {@code RTRIM} function (Oracle). */
   RTRIM,
 
-  /**
-   * The "EXTRACT" function.
-   */
+  /** {@code EXTRACT} function. */
   EXTRACT,
 
-  /**
-   * The "REVERSE" function (SQL Server, MySQL).
-   */
+  /** {@code REVERSE} function (SQL Server, MySQL). */
   REVERSE,
 
-  /**
-   * Call to a function using JDBC function syntax.
-   */
+  /** Call to a function using JDBC function syntax. */
   JDBC_FN,
 
-  /**
-   * The MULTISET value constructor.
-   */
+  /** {@code MULTISET} value constructor. */
   MULTISET_VALUE_CONSTRUCTOR,
 
-  /**
-   * The MULTISET query constructor.
-   */
+  /** {@code MULTISET} query constructor. */
   MULTISET_QUERY_CONSTRUCTOR,
 
-  /**
-   * The JSON value expression.
-   */
+  /** {@code JSON} value expression. */
   JSON_VALUE_EXPRESSION,
 
-  /**
-   * The {@code JSON_ARRAYAGG} aggregate function.
-   */
+  /** {@code JSON_ARRAYAGG} aggregate function. */
   JSON_ARRAYAGG,
 
-  /**
-   * The {@code JSON_OBJECTAGG} aggregate function.
-   */
+  /** {@code JSON_OBJECTAGG} aggregate function. */
   JSON_OBJECTAGG,
 
-  /**
-   * The "UNNEST" operator.
-   */
+  /** {@code UNNEST} operator. */
   UNNEST,
 
   /**
@@ -822,20 +641,15 @@ public enum SqlKind {
    */
   ARRAY_QUERY_CONSTRUCTOR,
 
-  /**
-   * Map Value Constructor, e.g. {@code Map['washington', 1, 'obama', 44]}.
-   */
+  /** MAP value constructor, e.g. {@code MAP ['washington', 1, 'obama', 44]}. */
   MAP_VALUE_CONSTRUCTOR,
 
-  /**
-   * Map Query Constructor, e.g. {@code MAP (SELECT empno, deptno FROM emp)}.
-   */
+  /** MAP query constructor,
+   * e.g. {@code MAP (SELECT empno, deptno FROM emp)}. */
   MAP_QUERY_CONSTRUCTOR,
 
-  /**
-   * CURSOR constructor, for example, <code>select * from
-   * TABLE(udx(CURSOR(select ...), x, y, z))</code>
-   */
+  /** {@code CURSOR} constructor, for example, <code>SELECT * FROM
+   * TABLE(udx(CURSOR(SELECT ...), x, y, z))</code>. */
   CURSOR,
 
   // internal operators (evaluated in validator) 200-299
@@ -877,6 +691,7 @@ public enum SqlKind {
   /** The {@code GROUPING(e, ...)} function. */
   GROUPING,
 
+  // CHECKSTYLE: IGNORE 1
   /** @deprecated Use {@link #GROUPING}. */
   @Deprecated // to be removed before 2.0
   GROUPING_ID,
diff --git a/core/src/main/java/org/apache/calcite/sql/SqlLiteral.java b/core/src/main/java/org/apache/calcite/sql/SqlLiteral.java
index 3c87f43..bd80488 100644
--- a/core/src/main/java/org/apache/calcite/sql/SqlLiteral.java
+++ b/core/src/main/java/org/apache/calcite/sql/SqlLiteral.java
@@ -175,17 +175,13 @@ public class SqlLiteral extends SqlNode {
 
   //~ Methods ----------------------------------------------------------------
 
-  /**
-   * @return value of {@link #typeName}
-   */
+  /** Returns the value of {@link #typeName}. */
   public SqlTypeName getTypeName() {
     return typeName;
   }
 
-  /**
-   * @return whether value is appropriate for its type (we have rules about
-   * these things)
-   */
+  /** Returns whether value is appropriate for its type. (We have rules about
+   * these things!) */
   public static boolean valueMatchesType(
       Object value,
       SqlTypeName typeName) {
@@ -500,7 +496,7 @@ public class SqlLiteral extends SqlNode {
   }
 
   /**
-   * For calc program builder - value may be different than {@link #unparse}
+   * For calc program builder - value may be different than {@link #unparse}.
    * Typical values:
    *
    * <ul>
diff --git a/core/src/main/java/org/apache/calcite/sql/SqlMerge.java b/core/src/main/java/org/apache/calcite/sql/SqlMerge.java
index ac529cf..665897e 100644
--- a/core/src/main/java/org/apache/calcite/sql/SqlMerge.java
+++ b/core/src/main/java/org/apache/calcite/sql/SqlMerge.java
@@ -105,23 +105,17 @@ public class SqlMerge extends SqlCall {
     }
   }
 
-  /**
-   * @return the identifier for the target table of the merge
-   */
+  /** Return the identifier for the target table of this MERGE. */
   public SqlNode getTargetTable() {
     return targetTable;
   }
 
-  /**
-   * @return the alias for the target table of the merge
-   */
+  /** Returns the alias for the target table of this MERGE. */
   public SqlIdentifier getAlias() {
     return alias;
   }
 
-  /**
-   * @return the source for the merge
-   */
+  /** Returns the source query of this MERGE. */
   public SqlNode getSourceTableRef() {
     return source;
   }
@@ -130,23 +124,18 @@ public class SqlMerge extends SqlCall {
     this.source = tableRef;
   }
 
-  /**
-   * @return the update statement for the merge
-   */
+  /** Returns the UPDATE statement for this MERGE. */
   public SqlUpdate getUpdateCall() {
     return updateCall;
   }
 
-  /**
-   * @return the insert statement for the merge
-   */
+  /** Returns the INSERT statement for this MERGE. */
   public SqlInsert getInsertCall() {
     return insertCall;
   }
 
-  /**
-   * @return the condition expression to determine whether to update or insert
-   */
+  /** Returns the condition expression to determine whether to UPDATE or
+   * INSERT. */
   public SqlNode getCondition() {
     return condition;
   }
diff --git a/core/src/main/java/org/apache/calcite/sql/SqlNode.java b/core/src/main/java/org/apache/calcite/sql/SqlNode.java
index 9947f78..ec69cae 100644
--- a/core/src/main/java/org/apache/calcite/sql/SqlNode.java
+++ b/core/src/main/java/org/apache/calcite/sql/SqlNode.java
@@ -64,6 +64,7 @@ public abstract class SqlNode implements Cloneable {
 
   //~ Methods ----------------------------------------------------------------
 
+  // CHECKSTYLE: IGNORE 1
   /** @deprecated Please use {@link #clone(SqlNode)}; this method brings
    * along too much baggage from early versions of Java */
   @Deprecated
diff --git a/core/src/main/java/org/apache/calcite/sql/SqlOperator.java b/core/src/main/java/org/apache/calcite/sql/SqlOperator.java
index 14d9edd..e1c2c2c 100644
--- a/core/src/main/java/org/apache/calcite/sql/SqlOperator.java
+++ b/core/src/main/java/org/apache/calcite/sql/SqlOperator.java
@@ -105,19 +105,13 @@ public abstract class SqlOperator {
    */
   private final int rightPrec;
 
-  /**
-   * used to infer the return type of a call to this operator
-   */
+  /** Used to infer the return type of a call to this operator. */
   private final SqlReturnTypeInference returnTypeInference;
 
-  /**
-   * used to infer types of unknown operands
-   */
+  /** Used to infer types of unknown operands. */
   private final SqlOperandTypeInference operandTypeInference;
 
-  /**
-   * used to validate operand types
-   */
+  /** Used to validate operand types. */
   private final SqlOperandTypeChecker operandTypeChecker;
 
   //~ Constructors -----------------------------------------------------------
@@ -885,17 +879,16 @@ public abstract class SqlOperator {
     }
   }
 
-  /**
-   * @return the return type inference strategy for this operator, or null if
-   * return type inference is implemented by a subclass override
-   */
+  /** Returns the return type inference strategy for this operator, or null if
+   * return type inference is implemented by a subclass override. */
   public SqlReturnTypeInference getReturnTypeInference() {
     return returnTypeInference;
   }
 
   /**
-   * Returns the {@link Strong.Policy} strategy for this operator, or null if there is no particular
-   * strategy, in which case this policy will be deducted from the operator's {@link SqlKind}.
+   * Returns the {@link Strong.Policy} strategy for this operator, or null if
+   * there is no particular strategy, in which case this policy will be deducted
+   * from the operator's {@link SqlKind}.
    *
    * @see Strong
    */
@@ -935,7 +928,7 @@ public abstract class SqlOperator {
 
   /**
    * Returns whether a call to this operator is guaranteed to always return
-   * the same result given the same operands; true is assumed by default
+   * the same result given the same operands; true is assumed by default.
    */
   public boolean isDeterministic() {
     return true;
@@ -954,7 +947,7 @@ public abstract class SqlOperator {
 
   /**
    * Returns whether it is unsafe to cache query plans referencing this
-   * operator; false is assumed by default
+   * operator; false is assumed by default.
    */
   public boolean isDynamicFunction() {
     return false;
diff --git a/core/src/main/java/org/apache/calcite/sql/SqlOperatorBinding.java b/core/src/main/java/org/apache/calcite/sql/SqlOperatorBinding.java
index 72973ee..21d1fd9 100644
--- a/core/src/main/java/org/apache/calcite/sql/SqlOperatorBinding.java
+++ b/core/src/main/java/org/apache/calcite/sql/SqlOperatorBinding.java
@@ -75,16 +75,12 @@ public abstract class SqlOperatorBinding {
     return false;
   }
 
-  /**
-   * @return bound operator
-   */
+  /** Returns the bound operator. */
   public SqlOperator getOperator() {
     return sqlOperator;
   }
 
-  /**
-   * @return factory for type creation
-   */
+  /** Returns the factory for type creation. */
   public RelDataTypeFactory getTypeFactory() {
     return typeFactory;
   }
@@ -169,9 +165,7 @@ public abstract class SqlOperatorBinding {
     throw new UnsupportedOperationException();
   }
 
-  /**
-   * @return the number of bound operands
-   */
+  /** Returns the number of bound operands. */
   public abstract int getOperandCount();
 
   /**
diff --git a/core/src/main/java/org/apache/calcite/sql/SqlUpdate.java b/core/src/main/java/org/apache/calcite/sql/SqlUpdate.java
index a6e9b3e..611e181 100644
--- a/core/src/main/java/org/apache/calcite/sql/SqlUpdate.java
+++ b/core/src/main/java/org/apache/calcite/sql/SqlUpdate.java
@@ -100,16 +100,12 @@ public class SqlUpdate extends SqlCall {
     }
   }
 
-  /**
-   * @return the identifier for the target table of the update
-   */
+  /** Returns the identifier for the target table of this UPDATE. */
   public SqlNode getTargetTable() {
     return targetTable;
   }
 
-  /**
-   * @return the alias for the target table of the update
-   */
+  /** Returns the alias for the target table of this UPDATE. */
   public SqlIdentifier getAlias() {
     return alias;
   }
@@ -118,16 +114,12 @@ public class SqlUpdate extends SqlCall {
     this.alias = alias;
   }
 
-  /**
-   * @return the list of target column names
-   */
+  /** Returns the list of target column names. */
   public SqlNodeList getTargetColumnList() {
     return targetColumnList;
   }
 
-  /**
-   * @return the list of source expressions
-   */
+  /** Returns the list of source expressions. */
   public SqlNodeList getSourceExpressionList() {
     return sourceExpressionList;
   }
diff --git a/core/src/main/java/org/apache/calcite/sql/SqlUtil.java b/core/src/main/java/org/apache/calcite/sql/SqlUtil.java
index 16d463b..18dc6bb 100644
--- a/core/src/main/java/org/apache/calcite/sql/SqlUtil.java
+++ b/core/src/main/java/org/apache/calcite/sql/SqlUtil.java
@@ -135,9 +135,7 @@ public abstract class SqlUtil {
     }
   }
 
-  /**
-   * Converts an SqlNode array to a SqlNodeList
-   */
+  /** Converts a SqlNode array to a SqlNodeList. */
   public static SqlNodeList toNodeList(SqlNode[] operands) {
     SqlNodeList ret = new SqlNodeList(SqlParserPos.ZERO);
     for (SqlNode node : operands) {
@@ -581,6 +579,9 @@ public abstract class SqlUtil {
   }
 
   /**
+   * Filters an iterator of routines, keeping only those that have the required
+   * argument types and names.
+   *
    * @see Glossary#SQL99 SQL:1999 Part 2 Section 10.4 Syntax Rule 6.b.iii.2.B
    */
   private static Iterator<SqlOperator> filterRoutinesByParameterTypeAndName(
@@ -653,6 +654,9 @@ public abstract class SqlUtil {
   }
 
   /**
+   * Filters an iterator of routines, keeping only those with the best match for
+   * the actual argument types.
+   *
    * @see Glossary#SQL99 SQL:1999 Part 2 Section 9.4
    */
   private static Iterator<SqlOperator> filterRoutinesByTypePrecedence(
diff --git a/core/src/main/java/org/apache/calcite/sql/SqlWindow.java b/core/src/main/java/org/apache/calcite/sql/SqlWindow.java
index 68c429a..ceba7e0 100644
--- a/core/src/main/java/org/apache/calcite/sql/SqlWindow.java
+++ b/core/src/main/java/org/apache/calcite/sql/SqlWindow.java
@@ -294,6 +294,7 @@ public class SqlWindow extends SqlCall {
     return windowCall;
   }
 
+  // CHECKSTYLE: IGNORE 1
   /** @see Util#deprecated(Object, boolean) */
   static void checkSpecialLiterals(SqlWindow window, SqlValidator validator) {
     final SqlNode lowerBound = window.getLowerBound();
diff --git a/core/src/main/java/org/apache/calcite/sql/SqlWindowTableFunction.java b/core/src/main/java/org/apache/calcite/sql/SqlWindowTableFunction.java
index 00c715a..309efd5 100644
--- a/core/src/main/java/org/apache/calcite/sql/SqlWindowTableFunction.java
+++ b/core/src/main/java/org/apache/calcite/sql/SqlWindowTableFunction.java
@@ -79,9 +79,9 @@ public class SqlWindowTableFunction extends SqlFunction {
   }
 
   /**
-   * Type-inference strategy whereby the result type of a table function call is a ROW,
-   * which is combined from the operand #0(TABLE parameter)'s schema and two
-   * additional fields:
+   * Type-inference strategy whereby the result type of a table function call is
+   * a ROW, which is combined from the operand #0(TABLE parameter)'s schema and
+   * two additional fields. The fields are as follows:
    *
    * <ol>
    *  <li>window_start: TIMESTAMP type to indicate a window's start.</li>
diff --git a/core/src/main/java/org/apache/calcite/sql/advise/SqlAdvisor.java b/core/src/main/java/org/apache/calcite/sql/advise/SqlAdvisor.java
index b15d19a..a642b28 100644
--- a/core/src/main/java/org/apache/calcite/sql/advise/SqlAdvisor.java
+++ b/core/src/main/java/org/apache/calcite/sql/advise/SqlAdvisor.java
@@ -79,7 +79,7 @@ public class SqlAdvisor {
   //~ Constructors -----------------------------------------------------------
 
   /**
-   * Creates a SqlAdvisor with a validator instance
+   * Creates a SqlAdvisor with a validator instance.
    *
    * @param validator Validator
    * @deprecated use {@link #SqlAdvisor(SqlValidatorWithHints, SqlParser.Config)}
@@ -91,7 +91,8 @@ public class SqlAdvisor {
   }
 
   /**
-   * Creates a SqlAdvisor with a validator instance and given parser configuration
+   * Creates a SqlAdvisor with a validator instance and given parser
+   * configuration.
    *
    * @param validator Validator
    * @param parserConfig parser config
@@ -306,8 +307,8 @@ public class SqlAdvisor {
   }
 
   /**
-   * Gets completion hints for a syntactically correct sql statement with dummy
-   * SqlIdentifier
+   * Gets completion hints for a syntactically correct SQL statement with dummy
+   * {@link SqlIdentifier}.
    *
    * @param sql A syntactically correct sql statement for which to retrieve
    *            completion hints
@@ -505,11 +506,12 @@ public class SqlAdvisor {
 
   /**
    * Turns a partially completed or syntactically incorrect sql statement into
-   * a simplified, valid one that can be passed into getCompletionHints()
+   * a simplified, valid one that can be passed into
+   * {@link #getCompletionHints(String, SqlParserPos)}.
    *
-   * @param sql    A partial or syntactically incorrect sql statement
-   * @param cursor to indicate column position in the query at which
-   *               completion hints need to be retrieved.
+   * @param sql    A partial or syntactically incorrect SQL statement
+   * @param cursor Indicates the position in the query at which
+   *               completion hints need to be retrieved
    * @return a completed, valid (and possibly simplified SQL statement
    */
   public String simplifySql(String sql, int cursor) {
@@ -518,7 +520,7 @@ public class SqlAdvisor {
   }
 
   /**
-   * Return an array of SQL reserved and keywords
+   * Returns an array of SQL reserved and keywords.
    *
    * @return an of SQL reserved and keywords
    */
@@ -604,10 +606,7 @@ public class SqlAdvisor {
 
   //~ Inner Classes ----------------------------------------------------------
 
-  /**
-   * An inner class that represents error message text and position info of a
-   * validator or parser exception
-   */
+  /** Text and position info of a validator or parser exception. */
   public class ValidateErrorInfo {
     private int startLineNum;
     private int startColumnNum;
@@ -669,37 +668,27 @@ public class SqlAdvisor {
       this.errorMsg = errorMsg;
     }
 
-    /**
-     * @return 1-based starting line number
-     */
+    /** Returns 1-based starting line number. */
     public int getStartLineNum() {
       return startLineNum;
     }
 
-    /**
-     * @return 1-based starting column number
-     */
+    /** Returns 1-based starting column number. */
     public int getStartColumnNum() {
       return startColumnNum;
     }
 
-    /**
-     * @return 1-based end line number
-     */
+    /** Returns 1-based end line number. */
     public int getEndLineNum() {
       return endLineNum;
     }
 
-    /**
-     * @return 1-based end column number
-     */
+    /** Returns 1-based end column number. */
     public int getEndColumnNum() {
       return endColumnNum;
     }
 
-    /**
-     * @return error message
-     */
+    /** Returns the error message. */
     public String getMessage() {
       return errorMsg;
     }
diff --git a/core/src/main/java/org/apache/calcite/sql/advise/SqlAdvisorHint2.java b/core/src/main/java/org/apache/calcite/sql/advise/SqlAdvisorHint2.java
index a829f9f..bc073a4 100644
--- a/core/src/main/java/org/apache/calcite/sql/advise/SqlAdvisorHint2.java
+++ b/core/src/main/java/org/apache/calcite/sql/advise/SqlAdvisorHint2.java
@@ -23,7 +23,7 @@ import org.apache.calcite.sql.validate.SqlMoniker;
  * {@link SqlAdvisor#getCompletionHints (String, int, String[])}.
  */
 public class SqlAdvisorHint2 extends SqlAdvisorHint {
-  /** Replacement string */
+  /** Replacement string. */
   public final String replacement;
 
   public SqlAdvisorHint2(String id, String[] names, String type, String replacement) {
diff --git a/core/src/main/java/org/apache/calcite/sql/advise/SqlSimpleParser.java b/core/src/main/java/org/apache/calcite/sql/advise/SqlSimpleParser.java
index 2b61b6d..7e98225 100644
--- a/core/src/main/java/org/apache/calcite/sql/advise/SqlSimpleParser.java
+++ b/core/src/main/java/org/apache/calcite/sql/advise/SqlSimpleParser.java
@@ -34,33 +34,28 @@ import java.util.Map;
 public class SqlSimpleParser {
   //~ Enums ------------------------------------------------------------------
 
+  /** Token. */
   enum TokenType {
     // keywords
     SELECT, FROM, JOIN, ON, USING, WHERE, GROUP, HAVING, ORDER, BY,
 
     UNION, INTERSECT, EXCEPT, MINUS,
 
-    /**
-     * left parenthesis
-     */
+    /** Left parenthesis. */
     LPAREN {
       public String sql() {
         return "(";
       }
     },
 
-    /**
-     * right parenthesis
-     */
+    /** Right parenthesis. */
     RPAREN {
       public String sql() {
         return ")";
       }
     },
 
-    /**
-     * identifier, or indeed any miscellaneous sequence of characters
-     */
+    /** Identifier, or indeed any miscellaneous sequence of characters. */
     ID,
 
     /**
@@ -96,10 +91,10 @@ public class SqlSimpleParser {
   //~ Constructors -----------------------------------------------------------
 
   /**
-   * Creates a SqlSimpleParser
+   * Creates a SqlSimpleParser.
    *
    * @param hintToken Hint token
-   * @deprecated
+   * @deprecated Use {@link #SqlSimpleParser(String, SqlParser.Config)}
    */
   @Deprecated // to be removed before 2.0
   public SqlSimpleParser(String hintToken) {
@@ -107,7 +102,7 @@ public class SqlSimpleParser {
   }
 
   /**
-   * Creates a SqlSimpleParser
+   * Creates a SqlSimpleParser.
    *
    * @param hintToken Hint token
    * @param parserConfig parser configuration
@@ -142,8 +137,8 @@ public class SqlSimpleParser {
   }
 
   /**
-   * Turns a partially completed or syntactically incorrect sql statement into
-   * a simplified, valid one that can be validated
+   * Turns a partially completed or syntactically incorrect SQL statement into a
+   * simplified, valid one that can be validated.
    *
    * @param sql A partial or syntactically incorrect sql statement
    * @return a completed, valid (and possibly simplified) SQL statement
@@ -260,6 +255,7 @@ public class SqlSimpleParser {
 
   //~ Inner Classes ----------------------------------------------------------
 
+  /** Tokenizer. */
   public static class Tokenizer {
     private static final Map<String, TokenType> TOKEN_TYPES = new HashMap<>();
 
@@ -444,6 +440,7 @@ public class SqlSimpleParser {
     }
   }
 
+  /** Token. */
   public static class Token {
     private final TokenType type;
     private final String s;
@@ -470,6 +467,7 @@ public class SqlSimpleParser {
     }
   }
 
+  /** Token representing an identifier. */
   public static class IdToken extends Token {
     public IdToken(TokenType type, String s) {
       super(type, s);
@@ -477,6 +475,7 @@ public class SqlSimpleParser {
     }
   }
 
+  /** Token representing a query. */
   static class Query extends Token {
     private final List<Token> tokenList;
 
diff --git a/core/src/main/java/org/apache/calcite/sql/dialect/BigQuerySqlDialect.java b/core/src/main/java/org/apache/calcite/sql/dialect/BigQuerySqlDialect.java
index 7451215..af2e855 100644
--- a/core/src/main/java/org/apache/calcite/sql/dialect/BigQuerySqlDialect.java
+++ b/core/src/main/java/org/apache/calcite/sql/dialect/BigQuerySqlDialect.java
@@ -246,9 +246,11 @@ public class BigQuerySqlDialect extends SqlDialect {
     }
   }
 
-  /** BigQuery data type reference:
+  /** {@inheritDoc}
+   *
+   * <p>BigQuery data type reference:
    * <a href="https://cloud.google.com/bigquery/docs/reference/standard-sql/data-types">
-   * BigQuery Standard SQL Data Types</a>
+   * BigQuery Standard SQL Data Types</a>.
    */
   @Override public SqlNode getCastSpec(final RelDataType type) {
     if (type instanceof BasicSqlType) {
diff --git a/core/src/main/java/org/apache/calcite/sql/fun/SqlCase.java b/core/src/main/java/org/apache/calcite/sql/fun/SqlCase.java
index b0b004e..b2d737a 100644
--- a/core/src/main/java/org/apache/calcite/sql/fun/SqlCase.java
+++ b/core/src/main/java/org/apache/calcite/sql/fun/SqlCase.java
@@ -59,7 +59,7 @@ public class SqlCase extends SqlCall {
   }
 
   /**
-   * Creates a call to the switched form of the case operator, viz:
+   * Creates a call to the switched form of the CASE operator. For example:
    *
    * <blockquote><code>CASE value<br>
    * WHEN whenList[0] THEN thenList[0]<br>
diff --git a/core/src/main/java/org/apache/calcite/sql/fun/SqlLiteralChainOperator.java b/core/src/main/java/org/apache/calcite/sql/fun/SqlLiteralChainOperator.java
index 0aba5c0..61a9978 100644
--- a/core/src/main/java/org/apache/calcite/sql/fun/SqlLiteralChainOperator.java
+++ b/core/src/main/java/org/apache/calcite/sql/fun/SqlLiteralChainOperator.java
@@ -166,7 +166,7 @@ public class SqlLiteralChainOperator extends SqlSpecialOperator {
         writer.newlineAndIndent();
       }
       if (rand instanceof SqlCharStringLiteral) {
-        NlsString nls = ((SqlCharStringLiteral) rand).getNlsString();
+        final NlsString nls = rand.getValueAs(NlsString.class);
         if (operand.i == 0) {
           collation = nls.getCollation();
 
diff --git a/core/src/main/java/org/apache/calcite/sql/fun/SqlStdOperatorTable.java b/core/src/main/java/org/apache/calcite/sql/fun/SqlStdOperatorTable.java
index 3a6027d..46f2208 100644
--- a/core/src/main/java/org/apache/calcite/sql/fun/SqlStdOperatorTable.java
+++ b/core/src/main/java/org/apache/calcite/sql/fun/SqlStdOperatorTable.java
@@ -2240,16 +2240,13 @@ public class SqlStdOperatorTable extends ReflectiveSqlOperatorTable {
           SqlFunctionCategory.SYSTEM, false, false,
           Optionality.FORBIDDEN) {
       };
-  /**
-   * The sequence next value function: <code>NEXT VALUE FOR sequence</code>
-   */
+
+  /** The sequence next value function: <code>NEXT VALUE FOR sequence</code>. */
   public static final SqlOperator NEXT_VALUE =
       new SqlSequenceValueOperator(SqlKind.NEXT_VALUE);
 
-  /**
-   * The sequence current value function: <code>CURRENT VALUE FOR
-   * sequence</code>
-   */
+  /** The sequence current value function: <code>CURRENT VALUE FOR
+   * sequence</code>. */
   public static final SqlOperator CURRENT_VALUE =
       new SqlSequenceValueOperator(SqlKind.CURRENT_VALUE);
 
diff --git a/core/src/main/java/org/apache/calcite/sql/parser/SqlParserPos.java b/core/src/main/java/org/apache/calcite/sql/parser/SqlParserPos.java
index e544422..a1d9d29 100644
--- a/core/src/main/java/org/apache/calcite/sql/parser/SqlParserPos.java
+++ b/core/src/main/java/org/apache/calcite/sql/parser/SqlParserPos.java
@@ -98,32 +98,24 @@ public class SqlParserPos implements Serializable {
         && this.endColumnNumber == ((SqlParserPos) o).endColumnNumber;
   }
 
-  /**
-   * @return 1-based starting line number
-   */
+  /** Returns 1-based starting line number. */
   public int getLineNum() {
     return lineNumber;
   }
 
-  /**
-   * @return 1-based starting column number
-   */
+  /** Returns 1-based starting column number. */
   public int getColumnNum() {
     return columnNumber;
   }
 
-  /**
-   * @return 1-based end line number (same as starting line number if the
-   * ParserPos is a point, not a range)
-   */
+  /** Returns 1-based end line number (same as starting line number if the
+   * ParserPos is a point, not a range). */
   public int getEndLineNum() {
     return endLineNumber;
   }
 
-  /**
-   * @return 1-based end column number (same as starting column number if the
-   * ParserPos is a point, not a range)
-   */
+  /** Returns 1-based end column number (same as starting column number if the
+   * ParserPos is a point, not a range). */
   public int getEndColumnNum() {
     return endColumnNumber;
   }
@@ -141,7 +133,7 @@ public class SqlParserPos implements Serializable {
     }
   }
 
-  /** @return true if this SqlParserPos is quoted. **/
+  /** Returns whether this SqlParserPos is quoted. */
   public boolean isQuoted() {
     return false;
   }
diff --git a/core/src/main/java/org/apache/calcite/sql/parser/SqlParserUtil.java b/core/src/main/java/org/apache/calcite/sql/parser/SqlParserUtil.java
index 84a2559..55325d1 100644
--- a/core/src/main/java/org/apache/calcite/sql/parser/SqlParserUtil.java
+++ b/core/src/main/java/org/apache/calcite/sql/parser/SqlParserUtil.java
@@ -80,10 +80,8 @@ public final class SqlParserUtil {
 
   //~ Methods ----------------------------------------------------------------
 
-  /**
-   * @return the character-set prefix of an sql string literal; returns null
-   * if there is none
-   */
+  /** Returns the character-set prefix of a SQL string literal; returns null if
+   * there is none. */
   public static String getCharacterSet(String s) {
     if (s.charAt(0) == '\'') {
       return null;
@@ -116,25 +114,22 @@ public final class SqlParserUtil {
     return new BigDecimal(s);
   }
 
-  /**
-   * @deprecated this method is not localized for Farrago standards
-   */
+  // CHECKSTYLE: IGNORE 1
+  /** @deprecated this method is not localized for Farrago standards */
   @Deprecated // to be removed before 2.0
   public static java.sql.Date parseDate(String s) {
     return java.sql.Date.valueOf(s);
   }
 
-  /**
-   * @deprecated Does not parse SQL:99 milliseconds
-   */
+  // CHECKSTYLE: IGNORE 1
+  /** @deprecated Does not parse SQL:99 milliseconds */
   @Deprecated // to be removed before 2.0
   public static java.sql.Time parseTime(String s) {
     return java.sql.Time.valueOf(s);
   }
 
-  /**
-   * @deprecated this method is not localized for Farrago standards
-   */
+  // CHECKSTYLE: IGNORE 1
+  /** @deprecated this method is not localized for Farrago standards */
   @Deprecated // to be removed before 2.0
   public static java.sql.Timestamp parseTimestamp(String s) {
     return java.sql.Timestamp.valueOf(s);
@@ -207,7 +202,7 @@ public final class SqlParserUtil {
   }
 
   /**
-   * Checks if the date/time format is valid
+   * Checks if the date/time format is valid, throws if not.
    *
    * @param pattern {@link SimpleDateFormat}  pattern
    */
diff --git a/core/src/main/java/org/apache/calcite/sql/type/MultisetOperandTypeChecker.java b/core/src/main/java/org/apache/calcite/sql/type/MultisetOperandTypeChecker.java
index 621d090..4f97633 100644
--- a/core/src/main/java/org/apache/calcite/sql/type/MultisetOperandTypeChecker.java
+++ b/core/src/main/java/org/apache/calcite/sql/type/MultisetOperandTypeChecker.java
@@ -27,8 +27,8 @@ import com.google.common.collect.ImmutableList;
 import static org.apache.calcite.util.Static.RESOURCE;
 
 /**
- * Parameter type-checking strategy types must be [nullable] Multiset,
- * [nullable] Multiset and the two types must have the same element type
+ * Parameter type-checking strategy where types must be ([nullable] Multiset,
+ * [nullable] Multiset), and the two types must have the same element type.
  *
  * @see MultisetSqlType#getComponentType
  */
diff --git a/core/src/main/java/org/apache/calcite/sql/type/OperandTypes.java b/core/src/main/java/org/apache/calcite/sql/type/OperandTypes.java
index b601aa3..f23cd2a 100644
--- a/core/src/main/java/org/apache/calcite/sql/type/OperandTypes.java
+++ b/core/src/main/java/org/apache/calcite/sql/type/OperandTypes.java
@@ -37,7 +37,7 @@ import static org.apache.calcite.util.Static.RESOURCE;
  * Strategies for checking operand types.
  *
  * <p>This class defines singleton instances of strategy objects for operand
- * type checking. {@link org.apache.calcite.sql.type.ReturnTypes}
+ * type-checking. {@link org.apache.calcite.sql.type.ReturnTypes}
  * and {@link org.apache.calcite.sql.type.InferTypes} provide similar strategies
  * for operand type inference and operator return type inference.
  *
@@ -445,8 +445,8 @@ public abstract class OperandTypes {
       family(SqlTypeFamily.ANY, SqlTypeFamily.NUMERIC);
 
   /**
-   * Parameter type-checking strategy type must a nullable time interval,
-   * nullable time interval
+   * Parameter type-checking strategy where type must a nullable time interval,
+   * nullable time interval.
    */
   public static final SqlSingleOperandTypeChecker INTERVAL_SAME_SAME =
       OperandTypes.and(INTERVAL_INTERVAL, SAME_SAME);
@@ -485,13 +485,13 @@ public abstract class OperandTypes {
           INTERVAL_DATETIME);
 
   /**
-   * Type checking strategy for the "*" operator
+   * Type-checking strategy for the "*" operator.
    */
   public static final SqlSingleOperandTypeChecker MULTIPLY_OPERATOR =
       OperandTypes.or(NUMERIC_NUMERIC, INTERVAL_NUMERIC, NUMERIC_INTERVAL);
 
   /**
-   * Type checking strategy for the "/" operator
+   * Type-checking strategy for the "/" operator.
    */
   public static final SqlSingleOperandTypeChecker DIVISION_OPERATOR =
       OperandTypes.or(NUMERIC_NUMERIC, INTERVAL_NUMERIC);
@@ -681,11 +681,14 @@ public abstract class OperandTypes {
         }
       };
 
-  /** Operand type checker that accepts period types:
-   * PERIOD (DATETIME, DATETIME)
-   * PERIOD (DATETIME, INTERVAL)
-   * [ROW] (DATETIME, DATETIME)
-   * [ROW] (DATETIME, INTERVAL) */
+  /** Operand type-checker that accepts period types. Examples:
+   *
+   * <ul>
+   * <li>PERIOD (DATETIME, DATETIME)
+   * <li>PERIOD (DATETIME, INTERVAL)
+   * <li>[ROW] (DATETIME, DATETIME)
+   * <li>[ROW] (DATETIME, INTERVAL)
+   * </ul> */
   private static class PeriodOperandTypeChecker
       implements SqlSingleOperandTypeChecker {
     public boolean checkSingleOperandType(SqlCallBinding callBinding,
diff --git a/core/src/main/java/org/apache/calcite/sql/type/ReturnTypes.java b/core/src/main/java/org/apache/calcite/sql/type/ReturnTypes.java
index 1a31917..7c76f65 100644
--- a/core/src/main/java/org/apache/calcite/sql/type/ReturnTypes.java
+++ b/core/src/main/java/org/apache/calcite/sql/type/ReturnTypes.java
@@ -287,13 +287,14 @@ public abstract class ReturnTypes {
       cascade(INTEGER, SqlTypeTransforms.TO_NULLABLE);
 
   /**
-   * Type-inference strategy whereby the result type of a call is a Bigint
+   * Type-inference strategy whereby the result type of a call is a BIGINT.
    */
   public static final SqlReturnTypeInference BIGINT =
       explicit(SqlTypeName.BIGINT);
+
   /**
    * Type-inference strategy whereby the result type of a call is a nullable
-   * Bigint
+   * BIGINT.
    */
   public static final SqlReturnTypeInference BIGINT_FORCE_NULLABLE =
       cascade(BIGINT, SqlTypeTransforms.FORCE_NULLABLE);
@@ -331,7 +332,7 @@ public abstract class ReturnTypes {
       cascade(VARCHAR_2000, SqlTypeTransforms.TO_NULLABLE);
 
   /**
-   * Type-inference strategy for Histogram agg support
+   * Type-inference strategy for Histogram agg support.
    */
   public static final SqlReturnTypeInference HISTOGRAM =
       explicit(SqlTypeName.VARBINARY, 8);
@@ -388,7 +389,7 @@ public abstract class ReturnTypes {
   };
 
   /**
-   * Returns a multiset type.
+   * Returns a MULTISET type.
    *
    * <p>For example, given <code>INTEGER</code>, returns
    * <code>INTEGER MULTISET</code>.
@@ -397,7 +398,7 @@ public abstract class ReturnTypes {
       cascade(ARG0, SqlTypeTransforms.TO_MULTISET);
 
   /**
-   * Returns the element type of a multiset
+   * Returns the element type of a MULTISET.
    */
   public static final SqlReturnTypeInference MULTISET_ELEMENT_NULLABLE =
       cascade(MULTISET, SqlTypeTransforms.TO_MULTISET_ELEMENT_TYPE);
@@ -474,10 +475,11 @@ public abstract class ReturnTypes {
     RelDataType type2 = opBinding.getOperandType(1);
     return typeFactory.getTypeSystem().deriveDecimalMultiplyType(typeFactory, type1, type2);
   };
+
   /**
    * Same as {@link #DECIMAL_PRODUCT} but returns with nullability if any of
    * the operands is nullable by using
-   * {@link org.apache.calcite.sql.type.SqlTypeTransforms#TO_NULLABLE}
+   * {@link org.apache.calcite.sql.type.SqlTypeTransforms#TO_NULLABLE}.
    */
   public static final SqlReturnTypeInference DECIMAL_PRODUCT_NULLABLE =
       cascade(DECIMAL_PRODUCT, SqlTypeTransforms.TO_NULLABLE);
@@ -508,7 +510,7 @@ public abstract class ReturnTypes {
   /**
    * Same as {@link #DECIMAL_QUOTIENT} but returns with nullability if any of
    * the operands is nullable by using
-   * {@link org.apache.calcite.sql.type.SqlTypeTransforms#TO_NULLABLE}
+   * {@link org.apache.calcite.sql.type.SqlTypeTransforms#TO_NULLABLE}.
    */
   public static final SqlReturnTypeInference DECIMAL_QUOTIENT_NULLABLE =
       cascade(DECIMAL_QUOTIENT, SqlTypeTransforms.TO_NULLABLE);
@@ -574,7 +576,8 @@ public abstract class ReturnTypes {
           chain(DECIMAL_MOD_NULLABLE, ARG1_NULLABLE);
 
   /**
-   * Type-inference strategy whereby the result type of a call is
+   * Type-inference strategy for concatenating two string arguments. The result
+   * type of a call is:
    *
    * <ul>
    * <li>the same type as the input types but with the combined length of the
@@ -735,7 +738,7 @@ public abstract class ReturnTypes {
 
   /**
    * Same as {@link #DYADIC_STRING_SUM_PRECISION} and using
-   * {@link org.apache.calcite.sql.type.SqlTypeTransforms#TO_NULLABLE}
+   * {@link org.apache.calcite.sql.type.SqlTypeTransforms#TO_NULLABLE}.
    */
   public static final SqlReturnTypeInference DYADIC_STRING_SUM_PRECISION_NULLABLE =
       cascade(DYADIC_STRING_SUM_PRECISION, SqlTypeTransforms.TO_NULLABLE);
diff --git a/core/src/main/java/org/apache/calcite/sql/type/SqlOperandTypeChecker.java b/core/src/main/java/org/apache/calcite/sql/type/SqlOperandTypeChecker.java
index ff2e78c..6e3452c 100644
--- a/core/src/main/java/org/apache/calcite/sql/type/SqlOperandTypeChecker.java
+++ b/core/src/main/java/org/apache/calcite/sql/type/SqlOperandTypeChecker.java
@@ -41,9 +41,7 @@ public interface SqlOperandTypeChecker {
       SqlCallBinding callBinding,
       boolean throwOnFailure);
 
-  /**
-   * @return range of operand counts allowed in a call
-   */
+  /** Returns the range of operand counts allowed in a call. */
   SqlOperandCountRange getOperandCountRange();
 
   /**
diff --git a/core/src/main/java/org/apache/calcite/sql/type/SqlTypeFamily.java b/core/src/main/java/org/apache/calcite/sql/type/SqlTypeFamily.java
index 77973c9..3a689c5 100644
--- a/core/src/main/java/org/apache/calcite/sql/type/SqlTypeFamily.java
+++ b/core/src/main/java/org/apache/calcite/sql/type/SqlTypeFamily.java
@@ -123,9 +123,7 @@ public enum SqlTypeFamily implements RelDataTypeFamily {
     return JDBC_TYPE_TO_FAMILY.get(jdbcType);
   }
 
-  /**
-   * @return collection of {@link SqlTypeName}s included in this family
-   */
+  /** Returns the collection of {@link SqlTypeName}s included in this family. */
   public Collection<SqlTypeName> getTypeNames() {
     switch (this) {
     case CHARACTER:
@@ -181,9 +179,7 @@ public enum SqlTypeFamily implements RelDataTypeFamily {
     }
   }
 
-  /**
-   * @return Default {@link RelDataType} belongs to this family.
-   */
+  /** Return the default {@link RelDataType} that belongs to this family. */
   public RelDataType getDefaultConcreteType(RelDataTypeFactory factory) {
     switch (this) {
     case CHARACTER:
diff --git a/core/src/main/java/org/apache/calcite/sql/type/SqlTypeName.java b/core/src/main/java/org/apache/calcite/sql/type/SqlTypeName.java
index 7752c7f..f1df362 100644
--- a/core/src/main/java/org/apache/calcite/sql/type/SqlTypeName.java
+++ b/core/src/main/java/org/apache/calcite/sql/type/SqlTypeName.java
@@ -338,10 +338,8 @@ public enum SqlTypeName {
     return special;
   }
 
-  /**
-   * @return the ordinal from {@link java.sql.Types} corresponding to this
-   * SqlTypeName
-   */
+  /** Returns the ordinal from {@link java.sql.Types} corresponding to this
+   * SqlTypeName. */
   public int getJdbcOrdinal() {
     return jdbcOrdinal;
   }
@@ -355,10 +353,8 @@ public enum SqlTypeName {
         .build();
   }
 
-  /**
-   * @return default scale for this type if supported, otherwise -1 if scale
-   * is either unsupported or must be specified explicitly
-   */
+  /** Returns the default scale for this type if supported, otherwise -1 if
+   * scale is either unsupported or must be specified explicitly. */
   public int getDefaultScale() {
     switch (this) {
     case DECIMAL:
@@ -938,9 +934,7 @@ public enum SqlTypeName {
     }
   }
 
-  /**
-   * @return name of this type
-   */
+  /** Returns the name of this type. */
   public String getName() {
     return toString();
   }
diff --git a/core/src/main/java/org/apache/calcite/sql/type/SqlTypeTransformCascade.java b/core/src/main/java/org/apache/calcite/sql/type/SqlTypeTransformCascade.java
index 13baf9f..8b60f48 100644
--- a/core/src/main/java/org/apache/calcite/sql/type/SqlTypeTransformCascade.java
+++ b/core/src/main/java/org/apache/calcite/sql/type/SqlTypeTransformCascade.java
@@ -24,7 +24,7 @@ import com.google.common.collect.ImmutableList;
 /**
  * Strategy to infer the type of an operator call from the type of the operands
  * by using one {@link SqlReturnTypeInference} rule and a combination of
- * {@link SqlTypeTransform}s
+ * {@link SqlTypeTransform}s.
  */
 public class SqlTypeTransformCascade implements SqlReturnTypeInference {
   //~ Instance fields --------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/sql/type/SqlTypeTransforms.java b/core/src/main/java/org/apache/calcite/sql/type/SqlTypeTransforms.java
index d5fed0b..5912514 100644
--- a/core/src/main/java/org/apache/calcite/sql/type/SqlTypeTransforms.java
+++ b/core/src/main/java/org/apache/calcite/sql/type/SqlTypeTransforms.java
@@ -40,7 +40,7 @@ public abstract class SqlTypeTransforms {
   /**
    * Parameter type-inference transform strategy where a derived type is
    * transformed into the same type but nullable if any of a calls operands is
-   * nullable
+   * nullable.
    */
   public static final SqlTypeTransform TO_NULLABLE =
       (opBinding, typeToTransform) ->
diff --git a/core/src/main/java/org/apache/calcite/sql/type/SqlTypeUtil.java b/core/src/main/java/org/apache/calcite/sql/type/SqlTypeUtil.java
index 009ef8f..eb1aafd 100644
--- a/core/src/main/java/org/apache/calcite/sql/type/SqlTypeUtil.java
+++ b/core/src/main/java/org/apache/calcite/sql/type/SqlTypeUtil.java
@@ -289,16 +289,12 @@ public abstract class SqlTypeUtil {
     return false;
   }
 
-  /**
-   * @return true if type is DATE, TIME, or TIMESTAMP
-   */
+  /** Returns whether a type is DATE, TIME, or TIMESTAMP. */
   public static boolean isDatetime(RelDataType type) {
     return SqlTypeFamily.DATETIME.contains(type);
   }
 
-  /**
-   * @return true if type is DATE
-   */
+  /** Returns whether a type is DATE. */
   public static boolean isDate(RelDataType type) {
     SqlTypeName typeName = type.getSqlTypeName();
     if (typeName == null) {
@@ -308,77 +304,57 @@ public abstract class SqlTypeUtil {
     return type.getSqlTypeName() == SqlTypeName.DATE;
   }
 
-  /**
-   * @return true if type is TIMESTAMP
-   */
+  /** Returns whether a type is TIMESTAMP. */
   public static boolean isTimestamp(RelDataType type) {
     return SqlTypeFamily.TIMESTAMP.contains(type);
   }
 
-  /**
-   * @return true if type is some kind of INTERVAL
-   */
+  /** Returns whether a type is some kind of INTERVAL. */
   public static boolean isInterval(RelDataType type) {
     return SqlTypeFamily.DATETIME_INTERVAL.contains(type);
   }
 
-  /**
-   * @return true if type is in SqlTypeFamily.Character
-   */
+  /** Returns whether a type is in SqlTypeFamily.Character. */
   public static boolean inCharFamily(RelDataType type) {
     return type.getFamily() == SqlTypeFamily.CHARACTER;
   }
 
-  /**
-   * @return true if type is in SqlTypeFamily.Character
-   */
+  /** Returns whether a type name is in SqlTypeFamily.Character. */
   public static boolean inCharFamily(SqlTypeName typeName) {
     return typeName.getFamily() == SqlTypeFamily.CHARACTER;
   }
 
-  /**
-   * @return true if type is in SqlTypeFamily.Boolean
-   */
+  /** Returns whether a type is in SqlTypeFamily.Boolean. */
   public static boolean inBooleanFamily(RelDataType type) {
     return type.getFamily() == SqlTypeFamily.BOOLEAN;
   }
 
-  /**
-   * @return true if two types are in same type family
-   */
+  /** Returns whether two types are in same type family. */
   public static boolean inSameFamily(RelDataType t1, RelDataType t2) {
     return t1.getFamily() == t2.getFamily();
   }
 
-  /**
-   * @return true if two types are in same type family, or one or the other is
-   * of type {@link SqlTypeName#NULL}.
-   */
+  /** Returns whether two types are in same type family, or one or the other is
+   * of type {@link SqlTypeName#NULL}. */
   public static boolean inSameFamilyOrNull(RelDataType t1, RelDataType t2) {
     return (t1.getSqlTypeName() == SqlTypeName.NULL)
         || (t2.getSqlTypeName() == SqlTypeName.NULL)
         || (t1.getFamily() == t2.getFamily());
   }
 
-  /**
-   * @return true if type family is either character or binary
-   */
+  /** Returns whether a type family is either character or binary. */
   public static boolean inCharOrBinaryFamilies(RelDataType type) {
     return (type.getFamily() == SqlTypeFamily.CHARACTER)
         || (type.getFamily() == SqlTypeFamily.BINARY);
   }
 
-  /**
-   * @return true if type is a LOB of some kind
-   */
+  /** Returns whether a type is a LOB of some kind. */
   public static boolean isLob(RelDataType type) {
     // TODO jvs 9-Dec-2004:  once we support LOB types
     return false;
   }
 
-  /**
-   * @return true if type is variable width with bounded precision
-   */
+  /** Returns whether a type is variable width with bounded precision. */
   public static boolean isBoundedVariableWidth(RelDataType type) {
     SqlTypeName typeName = type.getSqlTypeName();
     if (typeName == null) {
@@ -396,9 +372,7 @@ public abstract class SqlTypeUtil {
     }
   }
 
-  /**
-   * @return true if type is one of the integer types
-   */
+  /** Returns whether a type is one of the integer types. */
   public static boolean isIntType(RelDataType type) {
     SqlTypeName typeName = type.getSqlTypeName();
     if (typeName == null) {
@@ -415,9 +389,7 @@ public abstract class SqlTypeUtil {
     }
   }
 
-  /**
-   * @return true if type is decimal
-   */
+  /** Returns whether a type is DECIMAL. */
   public static boolean isDecimal(RelDataType type) {
     SqlTypeName typeName = type.getSqlTypeName();
     if (typeName == null) {
@@ -426,9 +398,7 @@ public abstract class SqlTypeUtil {
     return typeName == SqlTypeName.DECIMAL;
   }
 
-  /**
-   * @return true if type is double
-   */
+  /** Returns whether a type is DOUBLE. */
   public static boolean isDouble(RelDataType type) {
     SqlTypeName typeName = type.getSqlTypeName();
     if (typeName == null) {
@@ -437,9 +407,7 @@ public abstract class SqlTypeUtil {
     return typeName == SqlTypeName.DOUBLE;
   }
 
-  /**
-   * @return true if type is bigint
-   */
+  /** Returns whether a type is BIGINT. */
   public static boolean isBigint(RelDataType type) {
     SqlTypeName typeName = type.getSqlTypeName();
     if (typeName == null) {
@@ -448,9 +416,7 @@ public abstract class SqlTypeUtil {
     return typeName == SqlTypeName.BIGINT;
   }
 
-  /**
-   * @return true if type is numeric with exact precision
-   */
+  /** Returns whether a type is numeric with exact precision. */
   public static boolean isExactNumeric(RelDataType type) {
     SqlTypeName typeName = type.getSqlTypeName();
     if (typeName == null) {
@@ -473,9 +439,7 @@ public abstract class SqlTypeUtil {
     return type.getScale() != Integer.MIN_VALUE;
   }
 
-  /**
-   * Returns the maximum value of an integral type, as a long value
-   */
+  /** Returns the maximum value of an integral type, as a long value. */
   public static long maxValue(RelDataType type) {
     assert SqlTypeUtil.isIntType(type);
     switch (type.getSqlTypeName()) {
@@ -492,9 +456,7 @@ public abstract class SqlTypeUtil {
     }
   }
 
-  /**
-   * @return true if type is numeric with approximate precision
-   */
+  /** Returns whether a type is numeric with approximate precision. */
   public static boolean isApproximateNumeric(RelDataType type) {
     SqlTypeName typeName = type.getSqlTypeName();
     if (typeName == null) {
@@ -510,23 +472,17 @@ public abstract class SqlTypeUtil {
     }
   }
 
-  /**
-   * @return true if type is numeric
-   */
+  /** Returns whether a type is numeric. */
   public static boolean isNumeric(RelDataType type) {
     return isExactNumeric(type) || isApproximateNumeric(type);
   }
 
-  /**
-   * @return true if type is null.
-   */
+  /** Returns whether a type is null. */
   public static boolean isNull(RelDataType type) {
     SqlTypeName typeName = type.getSqlTypeName();
-
     if (typeName == null) {
       return false;
     }
-
     return typeName == SqlTypeName.NULL;
   }
 
@@ -613,8 +569,7 @@ public abstract class SqlTypeUtil {
     }
   }
 
-  /**
-   * Determines the minimum unscaled value of a numeric type
+  /** Returns the minimum unscaled value of a numeric type.
    *
    * @param type a numeric type
    */
@@ -635,8 +590,7 @@ public abstract class SqlTypeUtil {
     }
   }
 
-  /**
-   * Determines the maximum unscaled value of a numeric type
+  /** Returns the maximum unscaled value of a numeric type.
    *
    * @param type a numeric type
    */
@@ -657,10 +611,8 @@ public abstract class SqlTypeUtil {
     }
   }
 
-  /**
-   * @return true if type has a representation as a Java primitive (ignoring
-   * nullability)
-   */
+  /** Returns whether a type has a representation as a Java primitive (ignoring
+   * nullability). */
   @Deprecated // to be removed before 2.0
   public static boolean isJavaPrimitive(RelDataType type) {
     SqlTypeName typeName = type.getSqlTypeName();
@@ -684,9 +636,7 @@ public abstract class SqlTypeUtil {
     }
   }
 
-  /**
-   * @return class name of the wrapper for the primitive data type.
-   */
+  /** Returns the class name of the wrapper for the primitive data type. */
   @Deprecated // to be removed before 2.0
   public static String getPrimitiveWrapperJavaClassName(RelDataType type) {
     if (type == null) {
@@ -706,9 +656,7 @@ public abstract class SqlTypeUtil {
     }
   }
 
-  /**
-   * @return class name of the numeric data type.
-   */
+  /** Returns the class name of a numeric data type. */
   @Deprecated // to be removed before 2.0
   public static String getNumericJavaClassName(RelDataType type) {
     if (type == null) {
@@ -1601,16 +1549,12 @@ public abstract class SqlTypeUtil {
     return Integer.compare(p0, p1);
   }
 
-  /**
-   * @return true if type is ARRAY
-   */
+  /** Returns whether a type is ARRAY. */
   public static boolean isArray(RelDataType type) {
     return type.getSqlTypeName() == SqlTypeName.ARRAY;
   }
 
-  /**
-   * @return true if type is ROW
-   */
+  /** Returns whether a type is ROW. */
   public static boolean isRow(RelDataType type) {
     SqlTypeName typeName = type.getSqlTypeName();
     if (typeName == null) {
@@ -1619,58 +1563,44 @@ public abstract class SqlTypeUtil {
     return type.getSqlTypeName() == SqlTypeName.ROW;
   }
 
-  /**
-   * @return true if type is MAP
-   */
+  /** Returns whether a type is MAP. */
   public static boolean isMap(RelDataType type) {
     SqlTypeName typeName = type.getSqlTypeName();
     if (typeName == null) {
       return false;
     }
-
     return type.getSqlTypeName() == SqlTypeName.MAP;
   }
 
-  /**
-   * @return true if type is MULTISET
-   */
+  /** Returns whether a type is MULTISET. */
   public static boolean isMultiset(RelDataType type) {
     SqlTypeName typeName = type.getSqlTypeName();
     if (typeName == null) {
       return false;
     }
-
     return type.getSqlTypeName() == SqlTypeName.MULTISET;
   }
 
-  /**
-   * @return true if type is ARRAY/MULTISET
-   */
+  /** Returns whether a type is ARRAY or MULTISET. */
   public static boolean isCollection(RelDataType type) {
     SqlTypeName typeName = type.getSqlTypeName();
     if (typeName == null) {
       return false;
     }
-
     return type.getSqlTypeName() == SqlTypeName.ARRAY
         || type.getSqlTypeName() == SqlTypeName.MULTISET;
   }
 
-  /**
-   * @return true if type is CHARACTER
-   */
+  /** Returns whether a type is CHARACTER. */
   public static boolean isCharacter(RelDataType type) {
     SqlTypeName typeName = type.getSqlTypeName();
     if (typeName == null) {
       return false;
     }
-
     return SqlTypeFamily.CHARACTER.contains(type);
   }
 
-  /**
-   * @return true if the type is a CHARACTER or contains a CHARACTER type
-   */
+  /** Returns whether a type is a CHARACTER or contains a CHARACTER type. */
   public static boolean hasCharactor(RelDataType type) {
     if (isCharacter(type)) {
       return true;
@@ -1681,58 +1611,48 @@ public abstract class SqlTypeUtil {
     return false;
   }
 
-  /**
-   * @return true if type is STRING
-   */
+  /** Returns whether a type is STRING. */
   public static boolean isString(RelDataType type) {
     SqlTypeName typeName = type.getSqlTypeName();
     if (typeName == null) {
       return false;
     }
-
     return SqlTypeFamily.STRING.contains(type);
   }
 
-  /**
-   * @return true if type is BOOLEAN
-   */
+  /** Returns whether a type is BOOLEAN. */
   public static boolean isBoolean(RelDataType type) {
     SqlTypeName typeName = type.getSqlTypeName();
     if (typeName == null) {
       return false;
     }
-
     return SqlTypeFamily.BOOLEAN.contains(type);
   }
 
-  /**
-   * @return true if type is BINARY
-   */
+  /** Returns whether a type is BINARY. */
   public static boolean isBinary(RelDataType type) {
     SqlTypeName typeName = type.getSqlTypeName();
     if (typeName == null) {
       return false;
     }
-
     return SqlTypeFamily.BINARY.contains(type);
   }
 
-  /**
-   * @return true if type is Atomic
-   */
+  /** Returns whether a type is atomic (datetime, numeric, string or
+   * BOOLEAN). */
   public static boolean isAtomic(RelDataType type) {
     SqlTypeName typeName = type.getSqlTypeName();
     if (typeName == null) {
       return false;
     }
-
     return SqlTypeUtil.isDatetime(type)
         || SqlTypeUtil.isNumeric(type)
         || SqlTypeUtil.isString(type)
         || SqlTypeUtil.isBoolean(type);
   }
 
-  /** Get decimal with max precision/scale for the current type system. */
+  /** Returns a DECIMAL type with the maximum precision/scale for the current
+   * type system. */
   public static RelDataType getMaxPrecisionScaleDecimal(RelDataTypeFactory factory) {
     int maxPrecision = factory.getTypeSystem().getMaxNumericPrecision();
     int maxScale = factory.getTypeSystem().getMaxNumericScale();
diff --git a/core/src/main/java/org/apache/calcite/sql/util/SqlVisitor.java b/core/src/main/java/org/apache/calcite/sql/util/SqlVisitor.java
index 07c097d..c91beba 100644
--- a/core/src/main/java/org/apache/calcite/sql/util/SqlVisitor.java
+++ b/core/src/main/java/org/apache/calcite/sql/util/SqlVisitor.java
@@ -92,7 +92,7 @@ public interface SqlVisitor<R> {
   R visit(SqlDynamicParam param);
 
   /**
-   * Visits an interval qualifier
+   * Visits an interval qualifier.
    *
    * @param intervalQualifier Interval qualifier
    * @see SqlIntervalQualifier#accept(SqlVisitor)
diff --git a/core/src/main/java/org/apache/calcite/sql/validate/AggregatingSelectScope.java b/core/src/main/java/org/apache/calcite/sql/validate/AggregatingSelectScope.java
index aa7e199..e0e9b3c 100644
--- a/core/src/main/java/org/apache/calcite/sql/validate/AggregatingSelectScope.java
+++ b/core/src/main/java/org/apache/calcite/sql/validate/AggregatingSelectScope.java
@@ -71,7 +71,7 @@ public class AggregatingSelectScope
   //~ Constructors -----------------------------------------------------------
 
   /**
-   * Creates an AggregatingSelectScope
+   * Creates an AggregatingSelectScope.
    *
    * @param selectScope Parent scope
    * @param select      Enclosing SELECT node
diff --git a/core/src/main/java/org/apache/calcite/sql/validate/SelectScope.java b/core/src/main/java/org/apache/calcite/sql/validate/SelectScope.java
index 5cade17..2649d59 100644
--- a/core/src/main/java/org/apache/calcite/sql/validate/SelectScope.java
+++ b/core/src/main/java/org/apache/calcite/sql/validate/SelectScope.java
@@ -99,9 +99,7 @@ public class SelectScope extends ListScope {
    */
   private SqlNodeList orderList;
 
-  /**
-   * Scope to use to resolve windows
-   */
+  /** Scope to use to resolve windows. */
   private final SqlValidatorScope windowParent;
 
   //~ Constructors -----------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/sql/validate/SqlMoniker.java b/core/src/main/java/org/apache/calcite/sql/validate/SqlMoniker.java
index 7a67e77..8b8069e 100644
--- a/core/src/main/java/org/apache/calcite/sql/validate/SqlMoniker.java
+++ b/core/src/main/java/org/apache/calcite/sql/validate/SqlMoniker.java
@@ -24,7 +24,7 @@ import java.util.Comparator;
 import java.util.List;
 
 /**
- * An interface of an object identifier that represents a SqlIdentifier
+ * An interface of an object identifier that represents a SqlIdentifier.
  */
 public interface SqlMoniker {
   Comparator<SqlMoniker> COMPARATOR =
diff --git a/core/src/main/java/org/apache/calcite/sql/validate/SqlValidator.java b/core/src/main/java/org/apache/calcite/sql/validate/SqlValidator.java
index 09a50ff..cd563cb 100644
--- a/core/src/main/java/org/apache/calcite/sql/validate/SqlValidator.java
+++ b/core/src/main/java/org/apache/calcite/sql/validate/SqlValidator.java
@@ -206,7 +206,7 @@ public interface SqlValidator {
   void validateLiteral(SqlLiteral literal);
 
   /**
-   * Validates a {@link SqlIntervalQualifier}
+   * Validates a {@link SqlIntervalQualifier}.
    *
    * @param qualifier Interval qualifier
    */
@@ -301,7 +301,7 @@ public interface SqlValidator {
       SqlNodeList orderList, SqlValidatorScope scope);
 
   /**
-   * Validates a COLUMN_LIST parameter
+   * Validates a COLUMN_LIST parameter.
    *
    * @param function function containing COLUMN_LIST parameter
    * @param argTypes function arguments
@@ -384,7 +384,11 @@ public interface SqlValidator {
       SqlNode windowOrRef,
       SqlValidatorScope scope);
 
-  /** @deprecated Use {@link #resolveWindow(SqlNode, SqlValidatorScope)}, which
+  /**
+   * Converts a window specification or window name into a fully-resolved
+   * window specification.
+   *
+   * @deprecated Use {@link #resolveWindow(SqlNode, SqlValidatorScope)}, which
    * does not have the deprecated {@code populateBounds} parameter.
    *
    * @param populateBounds Whether to populate bounds. Doing so may alter the
@@ -472,7 +476,7 @@ public interface SqlValidator {
       RelDataType type);
 
   /**
-   * Removes a node from the set of validated nodes
+   * Removes a node from the set of validated nodes.
    *
    * @param node node to be removed
    */
@@ -787,9 +791,7 @@ public interface SqlValidator {
      * contain NULLS FIRST or NULLS LAST. */
     Config withDefaultNullCollation(NullCollation nullCollation);
 
-    /**
-     * Returns whether column reference expansion is enabled
-     */
+    /** Returns whether column reference expansion is enabled. */
     @ImmutableBeans.Property
     @ImmutableBeans.BooleanDefault(true)
     boolean columnReferenceExpansion();
diff --git a/core/src/main/java/org/apache/calcite/sql/validate/SqlValidatorCatalogReader.java b/core/src/main/java/org/apache/calcite/sql/validate/SqlValidatorCatalogReader.java
index 62d05e6..1dea580 100644
--- a/core/src/main/java/org/apache/calcite/sql/validate/SqlValidatorCatalogReader.java
+++ b/core/src/main/java/org/apache/calcite/sql/validate/SqlValidatorCatalogReader.java
@@ -84,6 +84,7 @@ public interface SqlValidatorCatalogReader extends Wrapper {
    */
   List<List<String>> getSchemaPaths();
 
+  // CHECKSTYLE: IGNORE 1
   /** @deprecated Use
    * {@link #nameMatcher()}.{@link SqlNameMatcher#field(RelDataType, String)} */
   @Deprecated // to be removed before 2.0
@@ -94,6 +95,7 @@ public interface SqlValidatorCatalogReader extends Wrapper {
    * that matches the case-sensitivity policy. */
   SqlNameMatcher nameMatcher();
 
+  // CHECKSTYLE: IGNORE 1
   /** @deprecated Use
    * {@link #nameMatcher()}.{@link SqlNameMatcher#matches(String, String)} */
   @Deprecated // to be removed before 2.0
@@ -102,6 +104,7 @@ public interface SqlValidatorCatalogReader extends Wrapper {
   RelDataType createTypeFromProjection(RelDataType type,
       List<String> columnNameList);
 
+  // CHECKSTYLE: IGNORE 1
   /** @deprecated Use
    * {@link #nameMatcher()}.{@link SqlNameMatcher#isCaseSensitive()} */
   @Deprecated // to be removed before 2.0
@@ -110,6 +113,6 @@ public interface SqlValidatorCatalogReader extends Wrapper {
   /** Returns the root namespace for name resolution. */
   CalciteSchema getRootSchema();
 
-  /** Returns Config settings */
+  /** Returns Config settings. */
   CalciteConnectionConfig getConfig();
 }
diff --git a/core/src/main/java/org/apache/calcite/sql/validate/SqlValidatorImpl.java b/core/src/main/java/org/apache/calcite/sql/validate/SqlValidatorImpl.java
index 32dc272..bf59a4f 100644
--- a/core/src/main/java/org/apache/calcite/sql/validate/SqlValidatorImpl.java
+++ b/core/src/main/java/org/apache/calcite/sql/validate/SqlValidatorImpl.java
@@ -168,8 +168,8 @@ public class SqlValidatorImpl implements SqlValidatorWithHints {
   final SqlValidatorCatalogReader catalogReader;
 
   /**
-   * Maps ParsePosition strings to the {@link SqlIdentifier} identifier
-   * objects at these positions
+   * Maps {@link SqlParserPos} strings to the {@link SqlIdentifier} identifier
+   * objects at these positions.
    */
   protected final Map<String, IdInfo> idPositions = new HashMap<>();
 
@@ -208,7 +208,7 @@ public class SqlValidatorImpl implements SqlValidatorWithHints {
 
   /**
    * Maps a {@link SqlSelect} node that is the argument to a CURSOR
-   * constructor to the scope of the result of that select node
+   * constructor to the scope of the result of that select node.
    */
   private final Map<SqlSelect, SqlValidatorScope> cursorScopes =
       new IdentityHashMap<>();
@@ -5709,9 +5709,7 @@ public class SqlValidatorImpl implements SqlValidatorWithHints {
     }
   }
 
-  /**
-   * retrieve pattern variables defined
-   */
+  /** Visitor that retrieves pattern variables defined. */
   private static class PatternVarVisitor implements SqlVisitor<Void> {
     private MatchRecognizeScope scope;
     PatternVarVisitor(MatchRecognizeScope scope) {
@@ -6368,9 +6366,8 @@ public class SqlValidatorImpl implements SqlValidatorWithHints {
     }
   }
 
-  /**
-   * Within one navigation function, the pattern var should be same
-   */
+  /** Validates that within one navigation function, the pattern var is the
+   * same. */
   private class PatternValidator extends SqlBasicVisitor<Set<String>> {
     private final boolean isMeasure;
     int firstLastCount;
diff --git a/core/src/main/java/org/apache/calcite/sql/validate/SqlValidatorScope.java b/core/src/main/java/org/apache/calcite/sql/validate/SqlValidatorScope.java
index c3847c0..dc7f1cb 100644
--- a/core/src/main/java/org/apache/calcite/sql/validate/SqlValidatorScope.java
+++ b/core/src/main/java/org/apache/calcite/sql/validate/SqlValidatorScope.java
@@ -70,6 +70,7 @@ public interface SqlValidatorScope {
   void resolve(List<String> names, SqlNameMatcher nameMatcher, boolean deep,
       Resolved resolved);
 
+  // CHECKSTYLE: IGNORE 1
   /** @deprecated Use
    * {@link #findQualifyingTableNames(String, SqlNode, SqlNameMatcher)} */
   @Deprecated // to be removed before 2.0
@@ -177,6 +178,7 @@ public interface SqlValidatorScope {
    */
   void validateExpr(SqlNode expr);
 
+  // CHECKSTYLE: IGNORE 1
   /** @deprecated Use
    * {@link #resolveTable(List, SqlNameMatcher, Path, Resolved)}. */
   @Deprecated // to be removed before 2.0
diff --git a/core/src/main/java/org/apache/calcite/sql/validate/SqlValidatorTable.java b/core/src/main/java/org/apache/calcite/sql/validate/SqlValidatorTable.java
index 7f9204b..f1d0423 100644
--- a/core/src/main/java/org/apache/calcite/sql/validate/SqlValidatorTable.java
+++ b/core/src/main/java/org/apache/calcite/sql/validate/SqlValidatorTable.java
@@ -42,7 +42,7 @@ public interface SqlValidatorTable extends Wrapper {
   SqlMonotonicity getMonotonicity(String columnName);
 
   /**
-   * Returns the access type of the table
+   * Returns the access type of the table.
    */
   SqlAccessType getAllowedAccess();
 
diff --git a/core/src/main/java/org/apache/calcite/sql/validate/SqlValidatorWithHints.java b/core/src/main/java/org/apache/calcite/sql/validate/SqlValidatorWithHints.java
index cd15ff0..f5d4fca 100644
--- a/core/src/main/java/org/apache/calcite/sql/validate/SqlValidatorWithHints.java
+++ b/core/src/main/java/org/apache/calcite/sql/validate/SqlValidatorWithHints.java
@@ -24,8 +24,8 @@ import java.util.List;
 
 /**
  * Extends {@link SqlValidator} to allow discovery of useful data such as fully
- * qualified names of sql objects, alternative valid sql objects that can be
- * used in the SQL statement (dubbed as hints)
+ * qualified names of SQL objects, alternative valid SQL objects that can be
+ * used in the SQL statement (dubbed as hints).
  */
 public interface SqlValidatorWithHints extends SqlValidator {
   //~ Methods ----------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/sql/validate/TableNamespace.java b/core/src/main/java/org/apache/calcite/sql/validate/TableNamespace.java
index cf3bf64..8a8c59a 100644
--- a/core/src/main/java/org/apache/calcite/sql/validate/TableNamespace.java
+++ b/core/src/main/java/org/apache/calcite/sql/validate/TableNamespace.java
@@ -111,8 +111,8 @@ class TableNamespace extends AbstractNamespace {
   }
 
   /**
-   * Gets the data-type of all columns in a table (for a view table: including
-   * columns of the underlying table)
+   * Gets the data-type of all columns in a table. For a view table, includes
+   * columns of the underlying table.
    */
   private RelDataType getBaseRowType() {
     final Table schemaTable = table.unwrap(Table.class);
diff --git a/core/src/main/java/org/apache/calcite/sql/validate/implicit/TypeCoercionImpl.java b/core/src/main/java/org/apache/calcite/sql/validate/implicit/TypeCoercionImpl.java
index e122152..312683a 100644
--- a/core/src/main/java/org/apache/calcite/sql/validate/implicit/TypeCoercionImpl.java
+++ b/core/src/main/java/org/apache/calcite/sql/validate/implicit/TypeCoercionImpl.java
@@ -409,7 +409,9 @@ public class TypeCoercionImpl extends AbstractTypeCoercion {
   }
 
   /**
-   * STRATEGIES
+   * {@inheritDoc}
+   *
+   * <p>STRATEGIES
    *
    * <p>With(Without) sub-query:
    *
@@ -442,9 +444,8 @@ public class TypeCoercionImpl extends AbstractTypeCoercion {
    *                     |                          |
    *                     +-------------type3--------+
    *</pre>
-   *   </li>
    *   <li>For both basic sql types(LHS and RHS),
-   *   find the common type of LHS and RHS nodes.</li>
+   *   find the common type of LHS and RHS nodes.
    * </ul>
    */
   public boolean inOperationCoercion(SqlCallBinding binding) {
diff --git a/core/src/main/java/org/apache/calcite/sql2rel/ReflectiveConvertletTable.java b/core/src/main/java/org/apache/calcite/sql2rel/ReflectiveConvertletTable.java
index b9c571e..d8243c8 100644
--- a/core/src/main/java/org/apache/calcite/sql2rel/ReflectiveConvertletTable.java
+++ b/core/src/main/java/org/apache/calcite/sql2rel/ReflectiveConvertletTable.java
@@ -165,7 +165,7 @@ public class ReflectiveConvertletTable implements SqlRexConvertletTable {
   }
 
   /**
-   * Registers a convertlet for a given operator instance
+   * Registers a convertlet for a given operator instance.
    *
    * @param op         Operator instance, say
    * {@link org.apache.calcite.sql.fun.SqlStdOperatorTable#MINUS}
diff --git a/core/src/main/java/org/apache/calcite/sql2rel/RelDecorrelator.java b/core/src/main/java/org/apache/calcite/sql2rel/RelDecorrelator.java
index ff418bf..161439f 100644
--- a/core/src/main/java/org/apache/calcite/sql2rel/RelDecorrelator.java
+++ b/core/src/main/java/org/apache/calcite/sql2rel/RelDecorrelator.java
@@ -1499,7 +1499,7 @@ public class RelDecorrelator implements ReflectiveVisitor {
   }
 
   /**
-   * Remove correlated variables from the tree at root corRel
+   * Removes correlated variables from the tree at root corRel.
    *
    * @param correlate Correlate
    */
diff --git a/core/src/main/java/org/apache/calcite/sql2rel/SqlToRelConverter.java b/core/src/main/java/org/apache/calcite/sql2rel/SqlToRelConverter.java
index 6982de1..8b696b5 100644
--- a/core/src/main/java/org/apache/calcite/sql2rel/SqlToRelConverter.java
+++ b/core/src/main/java/org/apache/calcite/sql2rel/SqlToRelConverter.java
@@ -335,9 +335,7 @@ public class SqlToRelConverter {
 
   //~ Methods ----------------------------------------------------------------
 
-  /**
-   * @return the RelOptCluster in use.
-   */
+  /** Returns the RelOptCluster in use. */
   public RelOptCluster getCluster() {
     return cluster;
   }
@@ -388,10 +386,8 @@ public class SqlToRelConverter {
     return retVal;
   }
 
-  /**
-   * @return mapping of non-correlated sub-queries that have been converted to
-   * the constants that they evaluate to
-   */
+  /** Returns the mapping of non-correlated sub-queries that have been converted
+   * to the constants that they evaluate to. */
   public Map<SqlNode, RexNode> getMapConvertedNonCorrSubqs() {
     return mapConvertedNonCorrSubqs;
   }
@@ -5132,7 +5128,7 @@ public class SqlToRelConverter {
     private final Map<AggregateCall, RexNode> aggCallMapping =
         new HashMap<>();
 
-    /** Are we directly inside a windowed aggregate? */
+    /** Whether we are directly inside a windowed aggregate. */
     private boolean inOver = false;
 
     /**
diff --git a/core/src/main/java/org/apache/calcite/sql2rel/SubQueryConverter.java b/core/src/main/java/org/apache/calcite/sql2rel/SubQueryConverter.java
index 5a76644..ec3054c 100644
--- a/core/src/main/java/org/apache/calcite/sql2rel/SubQueryConverter.java
+++ b/core/src/main/java/org/apache/calcite/sql2rel/SubQueryConverter.java
@@ -26,9 +26,7 @@ import org.apache.calcite.sql.SqlCall;
 public interface SubQueryConverter {
   //~ Methods ----------------------------------------------------------------
 
-  /**
-   * @return Whether the sub-query can be converted
-   */
+  /** Returns whether the sub-query can be converted. */
   boolean canConvertSubQuery();
 
   /**
diff --git a/core/src/main/java/org/apache/calcite/tools/FrameworkConfig.java b/core/src/main/java/org/apache/calcite/tools/FrameworkConfig.java
index 67b21d6..63db0e8 100644
--- a/core/src/main/java/org/apache/calcite/tools/FrameworkConfig.java
+++ b/core/src/main/java/org/apache/calcite/tools/FrameworkConfig.java
@@ -112,7 +112,7 @@ public interface FrameworkConfig {
 
   /**
    * Returns the convertlet table that should be used when converting from SQL
-   * to row expressions
+   * to row expressions.
    */
   SqlRexConvertletTable getConvertletTable();
 
diff --git a/core/src/main/java/org/apache/calcite/tools/PigRelBuilder.java b/core/src/main/java/org/apache/calcite/tools/PigRelBuilder.java
index eae5822..0c46267 100644
--- a/core/src/main/java/org/apache/calcite/tools/PigRelBuilder.java
+++ b/core/src/main/java/org/apache/calcite/tools/PigRelBuilder.java
@@ -183,11 +183,11 @@ public class PigRelBuilder extends RelBuilder {
     return super.as(alias);
   }
 
-  /** Partitioner for group and join */
+  /** Partitioner for group and join. */
   interface Partitioner {
   }
 
-  /** Option for performing group efficiently if data set is already sorted */
+  /** Option for performing group efficiently if data set is already sorted. */
   public enum GroupOption {
     MERGE,
     COLLECTED
diff --git a/core/src/main/java/org/apache/calcite/tools/Planner.java b/core/src/main/java/org/apache/calcite/tools/Planner.java
index a9b584c..dccd414 100644
--- a/core/src/main/java/org/apache/calcite/tools/Planner.java
+++ b/core/src/main/java/org/apache/calcite/tools/Planner.java
@@ -90,6 +90,7 @@ public interface Planner extends AutoCloseable {
    */
   RelRoot rel(SqlNode sql) throws RelConversionException;
 
+  // CHECKSTYLE: IGNORE 1
   /** @deprecated Use {@link #rel}. */
   @Deprecated // to removed before 2.0
   RelNode convert(SqlNode sql) throws RelConversionException;
diff --git a/core/src/main/java/org/apache/calcite/tools/RelBuilder.java b/core/src/main/java/org/apache/calcite/tools/RelBuilder.java
index f3b8188..9644253 100644
--- a/core/src/main/java/org/apache/calcite/tools/RelBuilder.java
+++ b/core/src/main/java/org/apache/calcite/tools/RelBuilder.java
@@ -728,6 +728,7 @@ public class RelBuilder {
     return groupKey_(nodes, nodeLists);
   }
 
+  // CHECKSTYLE: IGNORE 1
   /** @deprecated Now that indicator is deprecated, use
    * {@link #groupKey(Iterable, Iterable)}, which has the same behavior as
    * calling this method with {@code indicator = false}. */
@@ -779,6 +780,7 @@ public class RelBuilder {
     return groupKey_(groupSet, ImmutableList.copyOf(groupSets));
   }
 
+  // CHECKSTYLE: IGNORE 1
   /** @deprecated Use {@link #groupKey(ImmutableBitSet)}
    * or {@link #groupKey(ImmutableBitSet, Iterable)}. */
   @Deprecated // to be removed before 2.0
@@ -788,6 +790,7 @@ public class RelBuilder {
         ? ImmutableList.of(groupSet) : ImmutableList.copyOf(groupSets));
   }
 
+  // CHECKSTYLE: IGNORE 1
   /** @deprecated Use {@link #groupKey(ImmutableBitSet, Iterable)}. */
   @Deprecated // to be removed before 2.0
   public GroupKey groupKey(ImmutableBitSet groupSet, boolean indicator,
@@ -2045,7 +2048,7 @@ public class RelBuilder {
   }
 
   /**
-   * Auxiliary class to find a certain RelOptTable based on its name
+   * Auxiliary class to find a certain RelOptTable based on its name.
    */
   private static final class RelOptTableFinder extends RelHomogeneousShuttle {
     private RelOptTable relOptTable = null;
diff --git a/core/src/main/java/org/apache/calcite/util/BlackholeMap.java b/core/src/main/java/org/apache/calcite/util/BlackholeMap.java
index 56b8765..cd7b9e0 100644
--- a/core/src/main/java/org/apache/calcite/util/BlackholeMap.java
+++ b/core/src/main/java/org/apache/calcite/util/BlackholeMap.java
@@ -101,7 +101,7 @@ final class BlackholeMap<K, V> extends AbstractMap<K, V> {
   }
 
   /**
-   * Gets an instance of {@code BlackholeMap}
+   * Gets an instance of {@code BlackholeMap}.
    *
    * @param <K> type of the keys for the map
    * @param <V> type of the values for the map
diff --git a/core/src/main/java/org/apache/calcite/util/Bug.java b/core/src/main/java/org/apache/calcite/util/Bug.java
index a1e41ce..a7a502a 100644
--- a/core/src/main/java/org/apache/calcite/util/Bug.java
+++ b/core/src/main/java/org/apache/calcite/util/Bug.java
@@ -168,8 +168,7 @@ public abstract class Bug {
 
   /** Whether
    * <a href="https://issues.apache.org/jira/browse/CALCITE-2401">[CALCITE-2401]
-   * Improve RelMdPredicates performance</a>
-   */
+   * Improve RelMdPredicates performance</a> is fixed. */
   public static final boolean CALCITE_2401_FIXED = false;
 
   /** Whether
diff --git a/core/src/main/java/org/apache/calcite/util/CancelFlag.java b/core/src/main/java/org/apache/calcite/util/CancelFlag.java
index 7a44a34..35a8bbc 100644
--- a/core/src/main/java/org/apache/calcite/util/CancelFlag.java
+++ b/core/src/main/java/org/apache/calcite/util/CancelFlag.java
@@ -40,9 +40,7 @@ public class CancelFlag {
 
   //~ Methods ----------------------------------------------------------------
 
-  /**
-   * @return whether a cancellation has been requested
-   */
+  /** Returns whether a cancellation has been requested. */
   public boolean isCancelRequested() {
     return atomicBoolean.get();
   }
diff --git a/core/src/main/java/org/apache/calcite/util/ConversionUtil.java b/core/src/main/java/org/apache/calcite/util/ConversionUtil.java
index f4d313a..6646e23 100644
--- a/core/src/main/java/org/apache/calcite/util/ConversionUtil.java
+++ b/core/src/main/java/org/apache/calcite/util/ConversionUtil.java
@@ -23,7 +23,7 @@ import java.util.Locale;
 import static org.apache.calcite.util.Static.RESOURCE;
 
 /**
- * Utility functions for converting from one type to another
+ * Utility functions for converting from one type to another.
  */
 public class ConversionUtil {
   private ConversionUtil() {}
@@ -116,7 +116,7 @@ public class ConversionUtil {
   }
 
   /**
-   * Converts a string into a boolean
+   * Converts a string into a BOOLEAN.
    */
   public static Boolean toBoolean(String str) {
     if (str == null) {
diff --git a/core/src/main/java/org/apache/calcite/util/DateTimeStringUtils.java b/core/src/main/java/org/apache/calcite/util/DateTimeStringUtils.java
index 03f9a82..aff1229 100644
--- a/core/src/main/java/org/apache/calcite/util/DateTimeStringUtils.java
+++ b/core/src/main/java/org/apache/calcite/util/DateTimeStringUtils.java
@@ -29,7 +29,8 @@ public class DateTimeStringUtils {
 
   private DateTimeStringUtils() {}
 
-  /** The SimpleDateFormat string for ISO timestamps, "yyyy-MM-dd'T'HH:mm:ss'Z'"*/
+  /** The SimpleDateFormat string for ISO timestamps,
+   * "yyyy-MM-dd'T'HH:mm:ss'Z'". */
   public static final String ISO_DATETIME_FORMAT =
       "yyyy-MM-dd'T'HH:mm:ss'Z'";
 
diff --git a/core/src/main/java/org/apache/calcite/util/ImmutableBeans.java b/core/src/main/java/org/apache/calcite/util/ImmutableBeans.java
index a16a0e1..ea1343f 100644
--- a/core/src/main/java/org/apache/calcite/util/ImmutableBeans.java
+++ b/core/src/main/java/org/apache/calcite/util/ImmutableBeans.java
@@ -363,7 +363,7 @@ public class ImmutableBeans {
     }
   }
 
-  /** Is the method reading or writing? */
+  /** Whether the method is reading or writing. */
   private enum Mode {
     GET, SET, WITH
   }
diff --git a/core/src/main/java/org/apache/calcite/util/NumberUtil.java b/core/src/main/java/org/apache/calcite/util/NumberUtil.java
index 99fe861..b5dade9 100644
--- a/core/src/main/java/org/apache/calcite/util/NumberUtil.java
+++ b/core/src/main/java/org/apache/calcite/util/NumberUtil.java
@@ -114,11 +114,9 @@ public class NumberUtil {
     }
   }
 
-  /**
-   * @return whether a BigDecimal is a valid Farrago decimal. If a
+  /** Returns whether a {@link BigDecimal} is a valid Farrago decimal. If a
    * BigDecimal's unscaled value overflows a long, then it is not a valid
-   * Farrago decimal.
-   */
+   * Farrago decimal. */
   public static boolean isValidDecimal(BigDecimal bd) {
     BigInteger usv = bd.unscaledValue();
     long usvl = usv.longValue();
diff --git a/core/src/main/java/org/apache/calcite/util/Sources.java b/core/src/main/java/org/apache/calcite/util/Sources.java
index b647559..4a8c514de 100644
--- a/core/src/main/java/org/apache/calcite/util/Sources.java
+++ b/core/src/main/java/org/apache/calcite/util/Sources.java
@@ -94,9 +94,7 @@ public abstract class Sources {
     return source.protocol().equals("file");
   }
 
-  /**
-   * Adapter for {@link CharSource}
-   */
+  /** Adapter for {@link CharSource}. */
   private static class GuavaCharSource implements Source {
     private final CharSource charSource;
 
@@ -155,7 +153,8 @@ public abstract class Sources {
     }
   }
 
-  /** Implementation of {@link Source} on the top of a {@link File} or {@link URL} */
+  /** Implementation of {@link Source} on the top of a {@link File} or
+   * {@link URL}. */
   private static class FileSource implements Source {
     private final File file;
     private final URL url;
diff --git a/core/src/main/java/org/apache/calcite/util/StackWriter.java b/core/src/main/java/org/apache/calcite/util/StackWriter.java
index 13f9f67..9c66762 100644
--- a/core/src/main/java/org/apache/calcite/util/StackWriter.java
+++ b/core/src/main/java/org/apache/calcite/util/StackWriter.java
@@ -60,42 +60,42 @@ public class StackWriter extends FilterWriter {
   //~ Static fields/initializers ---------------------------------------------
 
   /**
-   * directive for increasing the indentation level
+   * Directive for increasing the indentation level.
    */
   public static final int INDENT = 0xF0000001;
 
   /**
-   * directive for decreasing the indentation level
+   * Directive for decreasing the indentation level.
    */
   public static final int OUTDENT = 0xF0000002;
 
   /**
-   * directive for beginning an SQL string literal
+   * Directive for beginning an SQL string literal.
    */
   public static final int OPEN_SQL_STRING_LITERAL = 0xF0000003;
 
   /**
-   * directive for ending an SQL string literal
+   * Directive for ending an SQL string literal.
    */
   public static final int CLOSE_SQL_STRING_LITERAL = 0xF0000004;
 
   /**
-   * directive for beginning an SQL identifier
+   * Directive for beginning an SQL identifier.
    */
   public static final int OPEN_SQL_IDENTIFIER = 0xF0000005;
 
   /**
-   * directive for ending an SQL identifier
+   * Directive for ending an SQL identifier.
    */
   public static final int CLOSE_SQL_IDENTIFIER = 0xF0000006;
 
   /**
-   * tab indentation
+   * Tab indentation.
    */
   public static final String INDENT_TAB = "\t";
 
   /**
-   * four-space indentation
+   * Four-space indentation.
    */
   public static final String INDENT_SPACE4 = "    ";
   private static final Character SINGLE_QUOTE = '\'';
diff --git a/core/src/main/java/org/apache/calcite/util/Util.java b/core/src/main/java/org/apache/calcite/util/Util.java
index d5b9198..0469083 100644
--- a/core/src/main/java/org/apache/calcite/util/Util.java
+++ b/core/src/main/java/org/apache/calcite/util/Util.java
@@ -762,7 +762,7 @@ public class Util {
   }
 
   /**
-   * @return true if s==null or if s.length()==0
+   * Returns whether s == null or if s.length() == 0.
    */
   public static boolean isNullOrEmpty(String s) {
     return (null == s) || (s.length() == 0);
@@ -803,7 +803,7 @@ public class Util {
 
   /**
    * Returns the {@link Charset} object representing the value of
-   * {@link CalciteSystemProperty#DEFAULT_CHARSET}
+   * {@link CalciteSystemProperty#DEFAULT_CHARSET}.
    *
    * @throws java.nio.charset.IllegalCharsetNameException If the given charset
    *                                                      name is illegal
@@ -817,18 +817,21 @@ public class Util {
     return DEFAULT_CHARSET;
   }
 
+  // CHECKSTYLE: IGNORE 1
   /** @deprecated Throw new {@link AssertionError} */
   @Deprecated // to be removed before 2.0
   public static Error newInternal() {
     return new AssertionError("(unknown cause)");
   }
 
+  // CHECKSTYLE: IGNORE 1
   /** @deprecated Throw new {@link AssertionError} */
   @Deprecated // to be removed before 2.0
   public static Error newInternal(String s) {
     return new AssertionError(s);
   }
 
+  // CHECKSTYLE: IGNORE 1
   /** @deprecated Throw new {@link RuntimeException} if checked; throw raw
    * exception if unchecked or {@link Error} */
   @Deprecated // to be removed before 2.0
@@ -836,6 +839,7 @@ public class Util {
     return new AssertionError(e);
   }
 
+  // CHECKSTYLE: IGNORE 1
   /** @deprecated Throw new {@link AssertionError} if applicable;
    * or {@link RuntimeException} if e is checked;
    * or raw exception if e is unchecked or {@link Error}. */
@@ -908,6 +912,7 @@ public class Util {
     return sw.toString();
   }
 
+  // CHECKSTYLE: IGNORE 1
   /** @deprecated Use {@link Preconditions#checkArgument}
    * or {@link Objects#requireNonNull(Object)} */
   @Deprecated // to be removed before 2.0
@@ -917,6 +922,7 @@ public class Util {
     }
   }
 
+  // CHECKSTYLE: IGNORE 1
   /** @deprecated Use {@link Preconditions#checkArgument}
    * or {@link Objects#requireNonNull(Object)} */
   @Deprecated // to be removed before 2.0
@@ -926,6 +932,7 @@ public class Util {
     }
   }
 
+  // CHECKSTYLE: IGNORE 1
   /** @deprecated Use {@link Preconditions#checkArgument} */
   @Deprecated // to be removed before 2.0
   public static void permAssert(boolean b, String description) {
diff --git a/core/src/main/java/org/apache/calcite/util/XmlOutput.java b/core/src/main/java/org/apache/calcite/util/XmlOutput.java
index 241070f..debda9d 100644
--- a/core/src/main/java/org/apache/calcite/util/XmlOutput.java
+++ b/core/src/main/java/org/apache/calcite/util/XmlOutput.java
@@ -49,15 +49,14 @@ public class XmlOutput {
   // is used to monitor changes to the output
   private int tagsWritten;
 
-  // This flag is set to true if the output should be compacted.
-  // Compacted output is free of extraneous whitespace and is designed
-  // for easier transport.
+  /** Whehter output should be compacted.  Compacted output is free of
+   * extraneous whitespace and is designed for easier transport. */
   private boolean compact;
 
-  /** @see #setIndentString */
+  /** String to write for each indent level; see {@link #setIndentString}. */
   private String indentString = "\t";
 
-  /** @see #setGlob */
+  /** Whether to detect that tags are empty; see {@link #setGlob}. */
   private boolean glob;
 
   /**
@@ -68,14 +67,17 @@ public class XmlOutput {
    */
   private boolean inTag;
 
-  /** @see #setAlwaysQuoteCData */
+  /** Whether to always quote CDATA segments (even if they don't contain
+   * special characters); see {@link #setAlwaysQuoteCData}. */
   private boolean alwaysQuoteCData;
 
-  /** @see #setIgnorePcdata */
+  /** Whether to ignore unquoted text, such as whitespace; see
+   * {@link #setIgnorePcdata}. */
   private boolean ignorePcdata;
 
   /**
-   * Private helper function to display a degree of indentation
+   * Private helper function to display a degree of indentation.
+   *
    * @param out the PrintWriter to which to display output.
    * @param indent the degree of indentation.
    */
@@ -443,14 +445,15 @@ public class XmlOutput {
   }
 
   /**
-   * Get the total number of tags written
+   * Returns the total number of tags written.
+   *
    * @return the total number of tags written to the XML stream.
    */
   public int numTagsWritten() {
     return tagsWritten;
   }
 
-  /** Print an XML attribute name and value for string val */
+  /** Prints an XML attribute name and value for string {@code val}. */
   private static void printAtt(PrintWriter pw, String name, String val) {
     if (val != null /* && !val.equals("") */) {
       pw.print(" ");
@@ -543,15 +546,13 @@ public class XmlOutput {
     public static final StringEscaper URL_ARG_ESCAPER;
     public static final StringEscaper URL_ESCAPER;
 
-    /**
-     * Identity transform
-     */
+    /** Identity transform. */
     StringEscaper() {
       translationVector = new ArrayList<>();
     }
 
     /**
-     * Map character "from" to escape sequence "to"
+     * Map character "from" to escape sequence "to".
      */
     public void defineEscape(char from, String to) {
       int i = (int) from;
diff --git a/core/src/main/java/org/apache/calcite/util/graph/AttributedDirectedGraph.java b/core/src/main/java/org/apache/calcite/util/graph/AttributedDirectedGraph.java
index 756d973..2bae5b7 100644
--- a/core/src/main/java/org/apache/calcite/util/graph/AttributedDirectedGraph.java
+++ b/core/src/main/java/org/apache/calcite/util/graph/AttributedDirectedGraph.java
@@ -50,6 +50,7 @@ public class AttributedDirectedGraph<V, E extends DefaultEdge>
     return null;
   }
 
+  // CHECKSTYLE: IGNORE 1
   /** @deprecated Use {@link #addEdge(Object, Object, Object...)}. */
   @Deprecated
   public E addEdge(V vertex, V targetVertex) {
diff --git a/core/src/main/java/org/apache/calcite/util/mapping/AbstractSourceMapping.java b/core/src/main/java/org/apache/calcite/util/mapping/AbstractSourceMapping.java
index 928a893..c4329e3 100644
--- a/core/src/main/java/org/apache/calcite/util/mapping/AbstractSourceMapping.java
+++ b/core/src/main/java/org/apache/calcite/util/mapping/AbstractSourceMapping.java
@@ -21,8 +21,8 @@ import java.util.Iterator;
 /**
  * Simple implementation of
  * {@link org.apache.calcite.util.mapping.Mappings.TargetMapping} where the
- * number of sources and targets are specified as constructor parameters and you
- * just need to implement one method,
+ * number of sources and targets are specified as constructor parameters, and you
+ * just need to implement one method.
  */
 public abstract class AbstractSourceMapping
     extends Mappings.AbstractMapping
diff --git a/core/src/main/java/org/apache/calcite/util/mapping/AbstractTargetMapping.java b/core/src/main/java/org/apache/calcite/util/mapping/AbstractTargetMapping.java
index d0099fc..cd4db08 100644
--- a/core/src/main/java/org/apache/calcite/util/mapping/AbstractTargetMapping.java
+++ b/core/src/main/java/org/apache/calcite/util/mapping/AbstractTargetMapping.java
@@ -21,8 +21,8 @@ import java.util.Iterator;
 /**
  * Simple implementation of
  * {@link org.apache.calcite.util.mapping.Mappings.TargetMapping} where the
- * number of sources and targets are specified as constructor parameters and you
- * just need to implement one method,
+ * number of sources and targets are specified as constructor parameters, and you
+ * just need to implement one method.
  */
 public abstract class AbstractTargetMapping
     extends Mappings.AbstractMapping
diff --git a/core/src/main/java/org/apache/calcite/util/trace/CalciteLogger.java b/core/src/main/java/org/apache/calcite/util/trace/CalciteLogger.java
index 1e0b253..8cb1c50 100644
--- a/core/src/main/java/org/apache/calcite/util/trace/CalciteLogger.java
+++ b/core/src/main/java/org/apache/calcite/util/trace/CalciteLogger.java
@@ -60,7 +60,7 @@ public class CalciteLogger {
   // WARN
 
   /**
-   * Logs a WARN message with two Object parameters
+   * Logs a WARN message with two Object parameters.
    */
   public void warn(String format, Object arg1, Object arg2) {
     // slf4j already avoids the array creation for 1 or 2 arg invocations
@@ -68,7 +68,7 @@ public class CalciteLogger {
   }
 
   /**
-   * Conditionally logs a WARN message with three Object parameters
+   * Conditionally logs a WARN message with three Object parameters.
    */
   public void warn(String format, Object arg1, Object arg2, Object arg3) {
     if (logger.isWarnEnabled()) {
@@ -77,7 +77,7 @@ public class CalciteLogger {
   }
 
   /**
-   * Conditionally logs a WARN message with four Object parameters
+   * Conditionally logs a WARN message with four Object parameters.
    */
   public void warn(String format, Object arg1, Object arg2, Object arg3, Object arg4) {
     if (logger.isWarnEnabled()) {
@@ -94,7 +94,7 @@ public class CalciteLogger {
   // INFO
 
   /**
-   * Logs an INFO message with two Object parameters
+   * Logs an INFO message with two Object parameters.
    */
   public void info(String format, Object arg1, Object arg2) {
     // slf4j already avoids the array creation for 1 or 2 arg invocations
@@ -102,7 +102,7 @@ public class CalciteLogger {
   }
 
   /**
-   * Conditionally logs an INFO message with three Object parameters
+   * Conditionally logs an INFO message with three Object parameters.
    */
   public void info(String format, Object arg1, Object arg2, Object arg3) {
     if (logger.isInfoEnabled()) {
@@ -111,7 +111,7 @@ public class CalciteLogger {
   }
 
   /**
-   * Conditionally logs an INFO message with four Object parameters
+   * Conditionally logs an INFO message with four Object parameters.
    */
   public void info(String format, Object arg1, Object arg2, Object arg3, Object arg4) {
     if (logger.isInfoEnabled()) {
@@ -128,7 +128,7 @@ public class CalciteLogger {
   // DEBUG
 
   /**
-   * Logs a DEBUG message with two Object parameters
+   * Logs a DEBUG message with two Object parameters.
    */
   public void debug(String format, Object arg1, Object arg2) {
     // slf4j already avoids the array creation for 1 or 2 arg invocations
@@ -136,7 +136,7 @@ public class CalciteLogger {
   }
 
   /**
-   * Conditionally logs a DEBUG message with three Object parameters
+   * Conditionally logs a DEBUG message with three Object parameters.
    */
   public void debug(String format, Object arg1, Object arg2, Object arg3) {
     if (logger.isDebugEnabled()) {
@@ -145,7 +145,7 @@ public class CalciteLogger {
   }
 
   /**
-   * Conditionally logs a DEBUG message with four Object parameters
+   * Conditionally logs a DEBUG message with four Object parameters.
    */
   public void debug(String format, Object arg1, Object arg2, Object arg3, Object arg4) {
     if (logger.isDebugEnabled()) {
@@ -162,7 +162,7 @@ public class CalciteLogger {
   // TRACE
 
   /**
-   * Logs a TRACE message with two Object parameters
+   * Logs a TRACE message with two Object parameters.
    */
   public void trace(String format, Object arg1, Object arg2) {
     // slf4j already avoids the array creation for 1 or 2 arg invocations
@@ -170,7 +170,7 @@ public class CalciteLogger {
   }
 
   /**
-   * Conditionally logs a TRACE message with three Object parameters
+   * Conditionally logs a TRACE message with three Object parameters.
    */
   public void trace(String format, Object arg1, Object arg2, Object arg3) {
     if (logger.isTraceEnabled()) {
@@ -179,7 +179,7 @@ public class CalciteLogger {
   }
 
   /**
-   * Conditionally logs a TRACE message with four Object parameters
+   * Conditionally logs a TRACE message with four Object parameters.
    */
   public void trace(String format, Object arg1, Object arg2, Object arg3, Object arg4) {
     if (logger.isTraceEnabled()) {
diff --git a/core/src/test/java/org/apache/calcite/adapter/enumerable/TypeFinderTest.java b/core/src/test/java/org/apache/calcite/adapter/enumerable/TypeFinderTest.java
index 179982a..ed91f2c 100644
--- a/core/src/test/java/org/apache/calcite/adapter/enumerable/TypeFinderTest.java
+++ b/core/src/test/java/org/apache/calcite/adapter/enumerable/TypeFinderTest.java
@@ -40,7 +40,8 @@ import static org.hamcrest.CoreMatchers.containsString;
 import static org.hamcrest.MatcherAssert.assertThat;
 
 /**
- * Test for {@link org.apache.calcite.adapter.enumerable.EnumerableRelImplementor.TypeFinder}
+ * Test for
+ * {@link org.apache.calcite.adapter.enumerable.EnumerableRelImplementor.TypeFinder}.
  */
 class TypeFinderTest {
 
diff --git a/core/src/test/java/org/apache/calcite/jdbc/CalciteRemoteDriverTest.java b/core/src/test/java/org/apache/calcite/jdbc/CalciteRemoteDriverTest.java
index 469f7b8..6bba338 100644
--- a/core/src/test/java/org/apache/calcite/jdbc/CalciteRemoteDriverTest.java
+++ b/core/src/test/java/org/apache/calcite/jdbc/CalciteRemoteDriverTest.java
@@ -859,7 +859,7 @@ class CalciteRemoteDriverTest {
   }
 
   /**
-   * Remote PreparedStatement insert WITHOUT bind variables
+   * Remote PreparedStatement insert WITHOUT bind variables.
    */
   @Test void testRemotePreparedStatementInsert() throws Exception {
     final Connection connection = DriverManager.getConnection(
@@ -880,7 +880,7 @@ class CalciteRemoteDriverTest {
   }
 
   /**
-   * Remote PreparedStatement insert WITH bind variables
+   * Remote PreparedStatement insert WITH bind variables.
    */
   @Test void testRemotePreparedStatementInsert2() throws Exception {
   }
diff --git a/core/src/test/java/org/apache/calcite/jdbc/JavaTypeFactoryTest.java b/core/src/test/java/org/apache/calcite/jdbc/JavaTypeFactoryTest.java
index cbb8631..ccda983 100644
--- a/core/src/test/java/org/apache/calcite/jdbc/JavaTypeFactoryTest.java
+++ b/core/src/test/java/org/apache/calcite/jdbc/JavaTypeFactoryTest.java
@@ -95,12 +95,12 @@ public final class JavaTypeFactoryTest {
         () -> "Type {" + actual.getTypeName() + "} is not a subtype of Types.RecordType");
   }
 
-  /***/
+  /** Struct with one field. */
   private static class OneFieldStruct {
     public Integer intField;
   }
 
-  /***/
+  /** Struct with two fields. */
   private static class TwoFieldStruct {
     public Integer intField;
     public String strField;
diff --git a/core/src/test/java/org/apache/calcite/materialize/NormalizationTrimFieldTest.java b/core/src/test/java/org/apache/calcite/materialize/NormalizationTrimFieldTest.java
index 567d952..b625862 100644
--- a/core/src/test/java/org/apache/calcite/materialize/NormalizationTrimFieldTest.java
+++ b/core/src/test/java/org/apache/calcite/materialize/NormalizationTrimFieldTest.java
@@ -49,6 +49,7 @@ import static org.apache.calcite.test.Matchers.isLinux;
 import static org.hamcrest.CoreMatchers.is;
 import static org.hamcrest.MatcherAssert.assertThat;
 
+/** Tests trimming unused fields before materialized view matching. */
 public class NormalizationTrimFieldTest extends SqlToRelTestBase {
 
   public static Frameworks.ConfigBuilder config() {
@@ -68,7 +69,7 @@ public class NormalizationTrimFieldTest extends SqlToRelTestBase {
         .traitDefs((List<RelTraitDef>) null);
   }
 
-  @Test public void testMVTrimUnusedFiled() {
+  @Test void testMVTrimUnusedFiled() {
     final RelBuilder relBuilder = RelBuilder.create(config().build());
     final LogicalProject project = (LogicalProject) relBuilder.scan("EMP")
         .project(relBuilder.field("EMPNO"),
@@ -78,21 +79,25 @@ public class NormalizationTrimFieldTest extends SqlToRelTestBase {
             relBuilder.field("DEPTNO")).build();
     final LogicalAggregate aggregate = (LogicalAggregate) relBuilder.push(project)
         .aggregate(
-            relBuilder.groupKey(relBuilder.field(1, 0, "DEPTNO")
-        ), relBuilder.count(relBuilder.field(1, 0, "SAL"))).build();
+            relBuilder.groupKey(relBuilder.field(1, 0, "DEPTNO")),
+            relBuilder.count(relBuilder.field(1, 0, "SAL")))
+        .build();
     final ImmutableBitSet groupSet = ImmutableBitSet.of(4);
     final AggregateCall count = aggregate.getAggCallList().get(0);
-    final AggregateCall call = AggregateCall.create(count.getAggregation(), count.isDistinct(),
-        count.isApproximate(), count.ignoreNulls(), ImmutableList.of(3),
+    final AggregateCall call = AggregateCall.create(count.getAggregation(),
+        count.isDistinct(), count.isApproximate(),
+        count.ignoreNulls(), ImmutableList.of(3),
         count.filterArg, count.collation, count.getType(), count.getName());
     final RelNode query = LogicalAggregate.create(project, aggregate.getHints(),
         groupSet, ImmutableList.of(groupSet), ImmutableList.of(call));
     final RelNode target = aggregate;
     final RelNode replacement = relBuilder.scan("mv0").build();
-    final RelOptMaterialization relOptMaterialization = new RelOptMaterialization(replacement,
-        target, null, Lists.newArrayList("mv0"));
-    final List<Pair<RelNode, List<RelOptMaterialization>>> relOptimized = RelOptMaterializations
-        .useMaterializedViews(query, ImmutableList.of(relOptMaterialization));
+    final RelOptMaterialization relOptMaterialization =
+        new RelOptMaterialization(replacement,
+            target, null, Lists.newArrayList("mv0"));
+    final List<Pair<RelNode, List<RelOptMaterialization>>> relOptimized =
+        RelOptMaterializations.useMaterializedViews(query,
+            ImmutableList.of(relOptMaterialization));
 
     final String optimized = ""
         + "LogicalProject(deptno=[CAST($0):TINYINT], count_sal=[$1])\n"
diff --git a/core/src/test/java/org/apache/calcite/plan/RelOptUtilTest.java b/core/src/test/java/org/apache/calcite/plan/RelOptUtilTest.java
index 63e7e25..58d772d 100644
--- a/core/src/test/java/org/apache/calcite/plan/RelOptUtilTest.java
+++ b/core/src/test/java/org/apache/calcite/plan/RelOptUtilTest.java
@@ -236,8 +236,8 @@ class RelOptUtilTest {
   }
 
   /**
-   * Test {@link RelOptUtil#splitJoinCondition(RelNode, RelNode, RexNode, List, List, List)}
-   * where the join condition contains an expanded version of IS NOT DISTINCT
+   * Tests {@link RelOptUtil#splitJoinCondition(RelNode, RelNode, RexNode, List, List, List)}
+   * where the join condition contains an expanded version of IS NOT DISTINCT.
    */
   @Test void testSplitJoinConditionExpandedIsNotDistinctFrom() {
     int leftJoinIndex = empScan.getRowType().getFieldNames().indexOf("DEPTNO");
@@ -261,8 +261,9 @@ class RelOptUtilTest {
   }
 
   /**
-   * Test {@link RelOptUtil#splitJoinCondition(RelNode, RelNode, RexNode, List, List, List)}
-   * where the join condition contains an expanded version of IS NOT DISTINCT using CASE
+   * Tests {@link RelOptUtil#splitJoinCondition(RelNode, RelNode, RexNode, List, List, List)}
+   * where the join condition contains an expanded version of IS NOT DISTINCT
+   * using CASE.
    */
   @Test void testSplitJoinConditionExpandedIsNotDistinctFromUsingCase() {
     int leftJoinIndex = empScan.getRowType().getFieldNames().indexOf("DEPTNO");
@@ -287,8 +288,9 @@ class RelOptUtilTest {
   }
 
   /**
-   * Test {@link RelOptUtil#splitJoinCondition(RelNode, RelNode, RexNode, List, List, List)}
-   * where the join condition contains an expanded version of IS NOT DISTINCT using CASE
+   * Tests {@link RelOptUtil#splitJoinCondition(RelNode, RelNode, RexNode, List, List, List)}
+   * where the join condition contains an expanded version of IS NOT DISTINCT
+   * using CASE.
    */
   @Test void testSplitJoinConditionExpandedIsNotDistinctFromUsingCase2() {
     int leftJoinIndex = empScan.getRowType().getFieldNames().indexOf("DEPTNO");
@@ -328,8 +330,8 @@ class RelOptUtilTest {
   }
 
   /**
-   * Test {@link RelOptUtil#pushDownJoinConditions(org.apache.calcite.rel.core.Join, RelBuilder)}
-   * where the join condition contains a complex expression
+   * Tests {@link RelOptUtil#pushDownJoinConditions(org.apache.calcite.rel.core.Join, RelBuilder)}
+   * where the join condition contains a complex expression.
    */
   @Test void testPushDownJoinConditions() {
     int leftJoinIndex = empScan.getRowType().getFieldNames().indexOf("DEPTNO");
@@ -373,8 +375,8 @@ class RelOptUtilTest {
   }
 
   /**
-   * Test {@link RelOptUtil#pushDownJoinConditions(org.apache.calcite.rel.core.Join, RelBuilder)}
-   * where the join condition contains a complex expression
+   * Tests {@link RelOptUtil#pushDownJoinConditions(org.apache.calcite.rel.core.Join, RelBuilder)}
+   * where the join condition contains a complex expression.
    */
   @Test void testPushDownJoinConditionsWithIsNotDistinct() {
     int leftJoinIndex = empScan.getRowType().getFieldNames().indexOf("DEPTNO");
@@ -415,12 +417,11 @@ class RelOptUtilTest {
     assertThat(leftInput.getProjects().get(empRow.getFieldCount()).toString(),
         is(relBuilder.call(SqlStdOperatorTable.PLUS, leftKeyInputRef, relBuilder.literal(1))
             .toString()));
-
   }
 
   /**
-   * Test {@link RelOptUtil#pushDownJoinConditions(org.apache.calcite.rel.core.Join, RelBuilder)}
-   * where the join condition contains a complex expression
+   * Tests {@link RelOptUtil#pushDownJoinConditions(org.apache.calcite.rel.core.Join, RelBuilder)}
+   * where the join condition contains a complex expression.
    */
   @Test void testPushDownJoinConditionsWithExpandedIsNotDistinct() {
     int leftJoinIndex = empScan.getRowType().getFieldNames().indexOf("DEPTNO");
@@ -470,8 +471,8 @@ class RelOptUtilTest {
   }
 
   /**
-   * Test {@link RelOptUtil#pushDownJoinConditions(org.apache.calcite.rel.core.Join, RelBuilder)}
-   * where the join condition contains a complex expression
+   * Tests {@link RelOptUtil#pushDownJoinConditions(org.apache.calcite.rel.core.Join, RelBuilder)}
+   * where the join condition contains a complex expression.
    */
   @Test void testPushDownJoinConditionsWithExpandedIsNotDistinctUsingCase() {
     int leftJoinIndex = empScan.getRowType().getFieldNames().indexOf("DEPTNO");
diff --git a/core/src/test/java/org/apache/calcite/plan/RelWriterTest.java b/core/src/test/java/org/apache/calcite/plan/RelWriterTest.java
index 73e9bcd..340eab3 100644
--- a/core/src/test/java/org/apache/calcite/plan/RelWriterTest.java
+++ b/core/src/test/java/org/apache/calcite/plan/RelWriterTest.java
@@ -901,13 +901,18 @@ class RelWriterTest {
   }
 
   /**
-   * Mock a {@link RelNode} for sql:
-   * select count(*) over (partition by {@code partitionKeyNames}
+   * Creates a mock {@link RelNode} that contains OVER. The SQL is as follows:
+   *
+   * <blockquote>
+   * select count(*) over (partition by {@code partitionKeyNames}<br>
    * order by {@code orderKeyNames}) from {@code table}
+   * </blockquote>
+   *
    * @param table Table name
-   * @param partitionKeyNames Partition by column names, may empty, can not be null
+   * @param partitionKeyNames Partition by column names, may empty, can not be
+   * null
    * @param orderKeyNames Order by column names, may empty, can not be null
-   * @return RelNode for the sql
+   * @return RelNode for the SQL
    */
   private RelNode mockCountOver(String table,
       List<String> partitionKeyNames, List<String> orderKeyNames) {
diff --git a/core/src/test/java/org/apache/calcite/plan/volcano/ComboRuleTest.java b/core/src/test/java/org/apache/calcite/plan/volcano/ComboRuleTest.java
index 101d66e..4d13b6f 100644
--- a/core/src/test/java/org/apache/calcite/plan/volcano/ComboRuleTest.java
+++ b/core/src/test/java/org/apache/calcite/plan/volcano/ComboRuleTest.java
@@ -45,7 +45,7 @@ import static org.apache.calcite.plan.volcano.PlannerTests.newCluster;
 import static org.junit.jupiter.api.Assertions.assertTrue;
 
 /**
- * Unit test for {@link VolcanoPlanner}
+ * Unit test for {@link VolcanoPlanner}.
  */
 class ComboRuleTest {
 
diff --git a/core/src/test/java/org/apache/calcite/plan/volcano/TraitConversionTest.java b/core/src/test/java/org/apache/calcite/plan/volcano/TraitConversionTest.java
index fe28158..0906a1e 100644
--- a/core/src/test/java/org/apache/calcite/plan/volcano/TraitConversionTest.java
+++ b/core/src/test/java/org/apache/calcite/plan/volcano/TraitConversionTest.java
@@ -240,9 +240,8 @@ class TraitConversionTest {
     @Override public void register(RelOptPlanner planner) {}
   }
 
-  /**
-   * Dummy distribution trait def for test (handles conversion of SimpleDistribution)
-   */
+  /** Dummy distribution trait def for test (handles conversion of
+   * SimpleDistribution). */
   private static class ConvertRelDistributionTraitDef
       extends RelTraitDef<SimpleDistribution> {
 
diff --git a/core/src/test/java/org/apache/calcite/plan/volcano/TraitPropagationTest.java b/core/src/test/java/org/apache/calcite/plan/volcano/TraitPropagationTest.java
index 2f0b3aa..39b3399 100644
--- a/core/src/test/java/org/apache/calcite/plan/volcano/TraitPropagationTest.java
+++ b/core/src/test/java/org/apache/calcite/plan/volcano/TraitPropagationTest.java
@@ -117,7 +117,7 @@ class TraitPropagationTest {
   }
 
   /**
-   * Materialized anonymous class for simplicity
+   * Materialized anonymous class for simplicity.
    */
   private static class PropAction {
     public RelNode apply(RelOptCluster cluster, RelOptSchema relOptSchema,
@@ -282,7 +282,7 @@ class TraitPropagationTest {
     }
   }
 
-  /** Rule for PhysSort */
+  /** Rule for PhysSort. */
   private static class PhysSortRule extends ConverterRule {
     static final PhysSortRule INSTANCE = Config.INSTANCE
         .withConversion(Sort.class, Convention.NONE, PHYSICAL, "PhysSortRule")
@@ -335,10 +335,10 @@ class TraitPropagationTest {
   }
 
   /* RELS */
-  /** Market interface for Phys nodes */
+  /** Market interface for Phys nodes. */
   private interface Phys extends RelNode { }
 
-  /** Physical Aggregate RelNode */
+  /** Physical Aggregate RelNode. */
   private static class PhysAgg extends Aggregate implements Phys {
     PhysAgg(RelOptCluster cluster, RelTraitSet traitSet, RelNode input,
         ImmutableBitSet groupSet,
@@ -359,7 +359,7 @@ class TraitPropagationTest {
     }
   }
 
-  /** Physical Project RelNode */
+  /** Physical Project RelNode. */
   private static class PhysProj extends Project implements Phys {
     PhysProj(RelOptCluster cluster, RelTraitSet traits, RelNode child,
         List<RexNode> exps, RelDataType rowType) {
@@ -389,7 +389,7 @@ class TraitPropagationTest {
     }
   }
 
-  /** Physical Sort RelNode */
+  /** Physical Sort RelNode. */
   private static class PhysSort extends Sort implements Phys {
     PhysSort(RelOptCluster cluster, RelTraitSet traits, RelNode child,
         RelCollation collation, RexNode offset,
@@ -411,7 +411,7 @@ class TraitPropagationTest {
     }
   }
 
-  /** Physical Table RelNode */
+  /** Physical Table RelNode. */
   private static class PhysTable extends AbstractRelNode implements Phys {
     PhysTable(RelOptCluster cluster) {
       super(cluster, cluster.traitSet().replace(PHYSICAL).replace(COLLATION));
diff --git a/core/src/test/java/org/apache/calcite/plan/volcano/VolcanoPlannerTest.java b/core/src/test/java/org/apache/calcite/plan/volcano/VolcanoPlannerTest.java
index 1709322..b45dced 100644
--- a/core/src/test/java/org/apache/calcite/plan/volcano/VolcanoPlannerTest.java
+++ b/core/src/test/java/org/apache/calcite/plan/volcano/VolcanoPlannerTest.java
@@ -220,8 +220,8 @@ class VolcanoPlannerTest {
   }
 
   /**
-   * Tests a rule that is fired once per subset (whereas most rules are fired
-   * once per rel in a set or rel in a subset)
+   * Tests a rule that is fired once per subset. (Whereas most rules are fired
+   * once per rel in a set or rel in a subset.)
    */
   @Test void testSubsetRule() {
     VolcanoPlanner planner = new VolcanoPlanner();
@@ -712,6 +712,7 @@ class VolcanoPlannerTest {
     }
   }
 
+  /** Rule that matches a PhysSingle on a RelSubset. */
   public static class PhysSingleSubsetRule
       extends RelRule<PhysSingleSubsetRule.Config> {
     static Config config(List<String> buf) {
@@ -753,9 +754,7 @@ class VolcanoPlannerTest {
     }
   }
 
-  /**
-   * Create an artificial RelSet merge in the PhysSingleRel's input RelSet
-   */
+  /** Creates an artificial RelSet merge in the PhysSingleRel's input RelSet. */
   public static class PhysSingleInputSetMergeRule
       extends RelRule<PhysSingleInputSetMergeRule.Config> {
     static final PhysSingleInputSetMergeRule INSTANCE =
@@ -989,6 +988,7 @@ class VolcanoPlannerTest {
     }
   }
 
+  /** Rule that converts a physical RelNode to an iterator. */
   private static class PhysToIteratorRule extends ConverterRule {
     static final PhysToIteratorRule INSTANCE = Config.INSTANCE
         .withConversion(RelNode.class, PlannerTests.PHYS_CALLING_CONVENTION,
diff --git a/core/src/test/java/org/apache/calcite/rel/rel2sql/RelToSqlConverterStructsTest.java b/core/src/test/java/org/apache/calcite/rel/rel2sql/RelToSqlConverterStructsTest.java
index 8fed7c6..a7f7049 100644
--- a/core/src/test/java/org/apache/calcite/rel/rel2sql/RelToSqlConverterStructsTest.java
+++ b/core/src/test/java/org/apache/calcite/rel/rel2sql/RelToSqlConverterStructsTest.java
@@ -101,16 +101,22 @@ class RelToSqlConverterStructsTest {
 
   private static final Table TABLE = new Table() {
     /**
-     * Table schema is as following:
+     * {@inheritDoc}
+     *
+     * <p>Table schema is as follows:
+     *
+     * <blockquote>
+     * <pre>
      *  myTable(
      *          a: BIGINT,
-     *          n1: STRUCT<
-     *                n11: STRUCT<b: BIGINT>,
-     *                n12: STRUCT<c: BIGINT>
-     *              >,
-     *          n2: STRUCT<d: BIGINT>,
-     *          e: BIGINT
-     *  )
+     *          n1: STRUCT&lt;
+     *                n11: STRUCT&lt;b: BIGINT&gt;,
+     *                n12: STRUCT&lt;c: BIGINT&gt;
+     *              &gt;,
+     *          n2: STRUCT&lt;d: BIGINT&gt;,
+     *          e: BIGINT)
+     * </pre>
+     * </blockquote>
      */
     @Override public RelDataType getRowType(RelDataTypeFactory tf) {
       RelDataType bigint = tf.createSqlType(SqlTypeName.BIGINT);
diff --git a/core/src/test/java/org/apache/calcite/rel/rel2sql/RelToSqlConverterTest.java b/core/src/test/java/org/apache/calcite/rel/rel2sql/RelToSqlConverterTest.java
index b1402a2..26cae12 100644
--- a/core/src/test/java/org/apache/calcite/rel/rel2sql/RelToSqlConverterTest.java
+++ b/core/src/test/java/org/apache/calcite/rel/rel2sql/RelToSqlConverterTest.java
@@ -2824,8 +2824,8 @@ class RelToSqlConverterTest {
 
   /** Test case for
    * <a href="https://issues.apache.org/jira/browse/CALCITE-2625">[CALCITE-2625]
-   * Removing Window Boundaries from SqlWindow of Aggregate Function which do not allow Framing</a>
-   * */
+   * Removing Window Boundaries from SqlWindow of Aggregate Function which do
+   * not allow Framing</a>. */
   @Test void testRowNumberFunctionForPrintingOfFrameBoundary() {
     String query = "SELECT row_number() over (order by \"hire_date\") FROM \"employee\"";
     String expected = "SELECT ROW_NUMBER() OVER (ORDER BY \"hire_date\")\n"
@@ -4534,9 +4534,10 @@ class RelToSqlConverterTest {
         .ok(expectedPostgresql);
   }
 
-  /** Test case fo
+  /** Test case for
    * <a href="https://issues.apache.org/jira/browse/CALCITE-3840">[CALCITE-3840]
-   * Re-aliasing of VALUES that has column aliases produces wrong SQL in the JDBC adapter</a>*/
+   * Re-aliasing of VALUES that has column aliases produces wrong SQL in the
+   * JDBC adapter</a>. */
   @Test void testValuesReAlias() {
     final RelBuilder builder = relBuilder();
     final RelNode root = builder
diff --git a/core/src/test/java/org/apache/calcite/rex/RexBuilderTest.java b/core/src/test/java/org/apache/calcite/rex/RexBuilderTest.java
index 8c82755..49b5e6b 100644
--- a/core/src/test/java/org/apache/calcite/rex/RexBuilderTest.java
+++ b/core/src/test/java/org/apache/calcite/rex/RexBuilderTest.java
@@ -604,7 +604,7 @@ class RexBuilderTest {
     checkBigDecimalLiteral(builder, "-73786976294838206464");
   }
 
-  /** Tests {@link RexCopier#visitOver(RexOver)} */
+  /** Tests {@link RexCopier#visitOver(RexOver)}. */
   @Test void testCopyOver() {
     final RelDataTypeFactory sourceTypeFactory =
         new SqlTypeFactoryImpl(RelDataTypeSystem.DEFAULT);
@@ -643,7 +643,7 @@ class RexBuilderTest {
     }
   }
 
-  /** Tests {@link RexCopier#visitCorrelVariable(RexCorrelVariable)} */
+  /** Tests {@link RexCopier#visitCorrelVariable(RexCorrelVariable)}. */
   @Test void testCopyCorrelVariable() {
     final RelDataTypeFactory sourceTypeFactory =
         new SqlTypeFactoryImpl(RelDataTypeSystem.DEFAULT);
@@ -664,7 +664,7 @@ class RexBuilderTest {
     assertThat(result.getType().getPrecision(), is(PRECISION));
   }
 
-  /** Tests {@link RexCopier#visitLocalRef(RexLocalRef)} */
+  /** Tests {@link RexCopier#visitLocalRef(RexLocalRef)}. */
   @Test void testCopyLocalRef() {
     final RelDataTypeFactory sourceTypeFactory =
         new SqlTypeFactoryImpl(RelDataTypeSystem.DEFAULT);
@@ -684,7 +684,7 @@ class RexBuilderTest {
     assertThat(result.getType().getPrecision(), is(PRECISION));
   }
 
-  /** Tests {@link RexCopier#visitDynamicParam(RexDynamicParam)} */
+  /** Tests {@link RexCopier#visitDynamicParam(RexDynamicParam)}. */
   @Test void testCopyDynamicParam() {
     final RelDataTypeFactory sourceTypeFactory =
         new SqlTypeFactoryImpl(RelDataTypeSystem.DEFAULT);
@@ -704,7 +704,7 @@ class RexBuilderTest {
     assertThat(result.getType().getPrecision(), is(PRECISION));
   }
 
-  /** Tests {@link RexCopier#visitRangeRef(RexRangeRef)} */
+  /** Tests {@link RexCopier#visitRangeRef(RexRangeRef)}. */
   @Test void testCopyRangeRef() {
     final RelDataTypeFactory sourceTypeFactory =
         new SqlTypeFactoryImpl(RelDataTypeSystem.DEFAULT);
diff --git a/core/src/test/java/org/apache/calcite/rex/RexProgramTest.java b/core/src/test/java/org/apache/calcite/rex/RexProgramTest.java
index 0c56498..1fdd8d1 100644
--- a/core/src/test/java/org/apache/calcite/rex/RexProgramTest.java
+++ b/core/src/test/java/org/apache/calcite/rex/RexProgramTest.java
@@ -192,7 +192,8 @@ class RexProgramTest extends RexProgramTestBase {
   }
 
   /**
-   * Creates a program, depending on variant:
+   * Creates one of several programs. The program generated depends on the
+   * {@code variant} parameter, as follows:
    *
    * <ol>
    * <li><code>select (x + y) + (x + 1) as a, (x + x) as b from t(x, y)</code>
@@ -1707,7 +1708,7 @@ class RexProgramTest extends RexProgramTestBase {
     checkSimplifyUnchanged(caseNode);
   }
 
-  /** Tests a CASE condition that contains division, */
+  /** Tests a CASE condition that contains division. */
   @Test void testSimplifyCaseDiv2() {
     // FIXME: RexInterpreter currently evaluates children beforehand.
     simplify = simplify.withParanoid(false);
@@ -2672,6 +2673,8 @@ class RexProgramTest extends RexProgramTestBase {
     checkSimplifyUnchanged(expr);
   }
 
+  /** An operator that overrides the {@link #getStrongPolicyInference}
+   * method. */
   private static class SqlSpecialOperatorWithPolicy extends SqlSpecialOperator {
     private final Strong.Policy policy;
     private SqlSpecialOperatorWithPolicy(String name, SqlKind kind, int prec, boolean leftAssoc,
diff --git a/core/src/test/java/org/apache/calcite/rex/RexProgramTestBase.java b/core/src/test/java/org/apache/calcite/rex/RexProgramTestBase.java
index c3a634c..0680b45 100644
--- a/core/src/test/java/org/apache/calcite/rex/RexProgramTestBase.java
+++ b/core/src/test/java/org/apache/calcite/rex/RexProgramTestBase.java
@@ -27,6 +27,7 @@ import static org.hamcrest.CoreMatchers.is;
 import static org.hamcrest.MatcherAssert.assertThat;
 import static org.junit.jupiter.api.Assertions.assertEquals;
 
+/** Base class for tests of {@link RexProgram}. */
 class RexProgramTestBase extends RexProgramBuilderBase {
 
   protected void checkDigest(RexNode node, String expected) {
@@ -55,7 +56,9 @@ class RexProgramTestBase extends RexProgramBuilderBase {
   }
 
   /**
-   * Asserts that given node has expected string representation with account of node type
+   * Asserts that a given node has expected string representation with account
+   * of node type.
+   *
    * @param message extra message that clarifies where the node came from
    * @param expected expected string representation of the node
    * @param node node to check
diff --git a/core/src/test/java/org/apache/calcite/runtime/DeterministicAutomatonTest.java b/core/src/test/java/org/apache/calcite/runtime/DeterministicAutomatonTest.java
index d74929f..ab1f5d1 100644
--- a/core/src/test/java/org/apache/calcite/runtime/DeterministicAutomatonTest.java
+++ b/core/src/test/java/org/apache/calcite/runtime/DeterministicAutomatonTest.java
@@ -21,7 +21,7 @@ import org.junit.jupiter.api.Test;
 import static org.hamcrest.MatcherAssert.assertThat;
 import static org.hamcrest.core.Is.is;
 
-/** Tests for the {@link DeterministicAutomaton} */
+/** Tests for the {@link DeterministicAutomaton}. */
 class DeterministicAutomatonTest {
   @Test void convertAutomaton() {
     final Pattern.PatternBuilder builder = Pattern.builder();
diff --git a/core/src/test/java/org/apache/calcite/sql/parser/SqlParserTest.java b/core/src/test/java/org/apache/calcite/sql/parser/SqlParserTest.java
index badb4ce..603beec 100644
--- a/core/src/test/java/org/apache/calcite/sql/parser/SqlParserTest.java
+++ b/core/src/test/java/org/apache/calcite/sql/parser/SqlParserTest.java
@@ -1287,14 +1287,14 @@ public class SqlParserTest {
     sqlList(sql).ok(expected, expected1, expected2);
   }
 
-  /** Should fail since the first statement lacks semicolon */
+  /** Should fail since the first statement lacks semicolon. */
   @Test void testStmtListWithoutSemiColon1() {
     sqlList("select * from emp where name like 'toto' "
         + "^delete^ from emp")
         .fails("(?s).*Encountered \"delete\" at .*");
   }
 
-  /** Should fail since the third statement lacks semicolon */
+  /** Should fail since the third statement lacks semicolon. */
   @Test void testStmtListWithoutSemiColon2() {
     sqlList("select * from emp where name like 'toto'; "
         + "delete from emp; "
@@ -7424,9 +7424,8 @@ public class SqlParserTest {
         .fails("(?s).*Encountered \"from\" at .*");
   }
 
-  /**
-   * Tests that applying member function of a specific type as a suffix function
-   */
+  /** Tests applying a member function of a specific type as a suffix
+   * function. */
   @Test void testMemberFunction() {
     sql("SELECT myColumn.func(a, b) FROM tbl")
         .ok("SELECT `MYCOLUMN`.`FUNC`(`A`, `B`)\n"
diff --git a/core/src/test/java/org/apache/calcite/sql/test/SqlTests.java b/core/src/test/java/org/apache/calcite/sql/test/SqlTests.java
index d7f0ef6..47b5c34 100644
--- a/core/src/test/java/org/apache/calcite/sql/test/SqlTests.java
+++ b/core/src/test/java/org/apache/calcite/sql/test/SqlTests.java
@@ -89,7 +89,7 @@ public abstract class SqlTests {
 
   /**
    * Helper function to get the string representation of a RelDataType
-   * (include precision/scale but no charset or collation)
+   * (include precision/scale but no charset or collation).
    *
    * @param sqlType Type
    * @return String representation of type
@@ -505,7 +505,7 @@ public abstract class SqlTests {
     }
   }
 
-  /** Stage of query processing */
+  /** Stage of query processing. */
   public enum Stage {
     PARSE("Parser"),
     VALIDATE("Validator"),
diff --git a/core/src/test/java/org/apache/calcite/sql/type/RelDataTypeSystemTest.java b/core/src/test/java/org/apache/calcite/sql/type/RelDataTypeSystemTest.java
index 62550f9..8eec8e8 100644
--- a/core/src/test/java/org/apache/calcite/sql/type/RelDataTypeSystemTest.java
+++ b/core/src/test/java/org/apache/calcite/sql/type/RelDataTypeSystemTest.java
@@ -28,7 +28,7 @@ import org.junit.jupiter.api.Test;
 import static org.junit.jupiter.api.Assertions.assertEquals;
 
 /**
- * Tests return type inference using {@code RelDataTypeSystem}
+ * Tests the inference of return types using {@code RelDataTypeSystem}.
  */
 class RelDataTypeSystemTest {
 
diff --git a/core/src/test/java/org/apache/calcite/sql2rel/RelFieldTrimmerTest.java b/core/src/test/java/org/apache/calcite/sql2rel/RelFieldTrimmerTest.java
index 94f555e..d04fc94 100644
--- a/core/src/test/java/org/apache/calcite/sql2rel/RelFieldTrimmerTest.java
+++ b/core/src/test/java/org/apache/calcite/sql2rel/RelFieldTrimmerTest.java
@@ -51,6 +51,7 @@ import static org.apache.calcite.test.Matchers.hasTree;
 import static org.hamcrest.MatcherAssert.assertThat;
 import static org.junit.jupiter.api.Assertions.assertTrue;
 
+/** Test for {@link RelFieldTrimmer}. */
 class RelFieldTrimmerTest {
   public static Frameworks.ConfigBuilder config() {
     final SchemaPlus rootSchema = Frameworks.createRootSchema(true);
diff --git a/core/src/test/java/org/apache/calcite/test/BookstoreSchema.java b/core/src/test/java/org/apache/calcite/test/BookstoreSchema.java
index 5d4e7a7..a000b56 100644
--- a/core/src/test/java/org/apache/calcite/test/BookstoreSchema.java
+++ b/core/src/test/java/org/apache/calcite/test/BookstoreSchema.java
@@ -70,8 +70,7 @@ public final class BookstoreSchema {
           Collections.emptyList())
   };
 
-  /**
-   */
+  /** Author. */
   public static class Author {
     public final int aid;
     public final String name;
@@ -87,8 +86,7 @@ public final class BookstoreSchema {
     }
   }
 
-  /**
-   */
+  /** Place. */
   public static class Place {
     public final Coordinate coords;
     public final String city;
@@ -102,8 +100,7 @@ public final class BookstoreSchema {
 
   }
 
-  /**
-   */
+  /** Coordinate. */
   public static class Coordinate {
     public final BigDecimal latitude;
     public final BigDecimal longtitude;
@@ -114,8 +111,7 @@ public final class BookstoreSchema {
     }
   }
 
-  /**
-   */
+  /** Book. */
   public static class Book {
     public final String title;
     public final int publishYear;
@@ -129,8 +125,7 @@ public final class BookstoreSchema {
     }
   }
 
-  /**
-   */
+  /** Page. */
   public static class Page {
     public final int pageNo;
     public final String contentType;
diff --git a/core/src/test/java/org/apache/calcite/test/CalciteAssert.java b/core/src/test/java/org/apache/calcite/test/CalciteAssert.java
index 6d4965d..26e96f6 100644
--- a/core/src/test/java/org/apache/calcite/test/CalciteAssert.java
+++ b/core/src/test/java/org/apache/calcite/test/CalciteAssert.java
@@ -385,12 +385,18 @@ public class CalciteAssert {
     return buf.toString();
   }
 
-  /** @see Matchers#returnsUnordered(String...) */
+  /** Checks that the {@link ResultSet} returns the given set of lines, in no
+   * particular order.
+   *
+   * @see Matchers#returnsUnordered(String...) */
   static Consumer<ResultSet> checkResultUnordered(final String... lines) {
     return checkResult(true, false, lines);
   }
 
-  /** @see Matchers#returnsUnordered(String...) */
+  /** Checks that the {@link ResultSet} returns the given set of lines,
+   * optionally sorting.
+   *
+   * @see Matchers#returnsUnordered(String...) */
   static Consumer<ResultSet> checkResult(final boolean sort,
       final boolean head, final String... lines) {
     return resultSet -> {
@@ -1042,7 +1048,7 @@ public class CalciteAssert {
       }
     }
 
-    /** Creates a copy of this AssertThat, adding more schemas */
+    /** Creates a copy of this AssertThat, adding more schemas. */
     public AssertThat with(SchemaSpec... specs) {
       AssertThat next = this;
       for (SchemaSpec spec : specs) {
@@ -1075,7 +1081,7 @@ public class CalciteAssert {
       return new AssertThat(connectionFactory.with(property, value));
     }
 
-    /** Sets Lex property **/
+    /** Sets the Lex property. **/
     public AssertThat with(Lex lex) {
       return with(CalciteConnectionProperty.LEX, lex);
     }
@@ -1269,7 +1275,7 @@ public class CalciteAssert {
     }
   }
 
-  /** Connection post processor */
+  /** Connection post-processor. */
   @FunctionalInterface
   public interface ConnectionPostProcessor {
     Connection apply(Connection connection) throws SQLException;
@@ -1542,10 +1548,8 @@ public class CalciteAssert {
             hooks, null, null, checkValidationException(optionalMessage)));
     }
 
-    /**
-     * Utility method so that one doesn't have to call
-     * {@link #failsAtValidation} with {@code null}
-     * */
+    /** Utility method so that one doesn't have to call
+     * {@link #failsAtValidation} with {@code null}. */
     public AssertQuery failsAtValidation() {
       return failsAtValidation(null);
     }
@@ -1764,6 +1768,7 @@ public class CalciteAssert {
       });
     }
 
+    // CHECKSTYLE: IGNORE 1
     /** @deprecated Use {@link #queryContains(Consumer)}. */
     @SuppressWarnings("Guava")
     @Deprecated // to be removed before 2.0
@@ -2114,9 +2119,7 @@ public class CalciteAssert {
     }
   }
 
-  /**
-   * We want a consumer which can throw SqlException
-   */
+  /** We want a consumer that can throw SqlException. */
   public interface PreparedStatementConsumer {
     void accept(PreparedStatement statement) throws SQLException;
   }
diff --git a/core/src/test/java/org/apache/calcite/test/DiffTestCase.java b/core/src/test/java/org/apache/calcite/test/DiffTestCase.java
index 4b4551e..24ddbbc 100644
--- a/core/src/test/java/org/apache/calcite/test/DiffTestCase.java
+++ b/core/src/test/java/org/apache/calcite/test/DiffTestCase.java
@@ -69,10 +69,7 @@ public abstract class DiffTestCase {
    */
   protected OutputStream logOutputStream;
 
-  /**
-   * Diff masks defined so far
-   */
-  // private List diffMasks;
+  /** Diff masks defined so far. */
   private String diffMasks;
   Pattern compiledDiffPattern;
   Matcher compiledDiffMatcher;
@@ -149,9 +146,7 @@ public abstract class DiffTestCase {
         openTestLogOutputStream(testLogFile), StandardCharsets.UTF_8);
   }
 
-  /**
-   * @return the root under which testlogs should be written
-   */
+  /** Returns the root directory under which testlogs should be written. */
   protected abstract File getTestlogRoot() throws Exception;
 
   /**
diff --git a/core/src/test/java/org/apache/calcite/test/HierarchySchema.java b/core/src/test/java/org/apache/calcite/test/HierarchySchema.java
index 868edd7..3721f02 100644
--- a/core/src/test/java/org/apache/calcite/test/HierarchySchema.java
+++ b/core/src/test/java/org/apache/calcite/test/HierarchySchema.java
@@ -59,9 +59,7 @@ public class HierarchySchema {
       new Hierarchy(1, 4),
   };
 
-  /**
-   * Hierarchy representing manager - subordinate
-   */
+  /** Hierarchy representing manager - subordinate. */
   public static class Hierarchy {
     public final int managerid;
     public final int subordinateid;
diff --git a/core/src/test/java/org/apache/calcite/test/JdbcAdapterTest.java b/core/src/test/java/org/apache/calcite/test/JdbcAdapterTest.java
index 67b3938..8502c33 100644
--- a/core/src/test/java/org/apache/calcite/test/JdbcAdapterTest.java
+++ b/core/src/test/java/org/apache/calcite/test/JdbcAdapterTest.java
@@ -85,11 +85,10 @@ class JdbcAdapterTest {
             + "FROM \"foodmart\".\"sales_fact_1998\"");
   }
 
-  /**
-   * Test case for
+  /** Test case for
    * <a href="https://issues.apache.org/jira/browse/CALCITE-3115">[CALCITE-3115]
    * Cannot add JdbcRules which have different JdbcConvention
-   * to same VolcanoPlanner's RuleSet.</a>*/
+   * to same VolcanoPlanner's RuleSet</a>. */
   @Test void testUnionPlan2() {
     CalciteAssert.model(JdbcTest.FOODMART_SCOTT_MODEL)
         .query("select \"store_name\" from \"foodmart\".\"store\" where \"store_id\" < 10\n"
diff --git a/core/src/test/java/org/apache/calcite/test/JdbcFrontLinqBackTest.java b/core/src/test/java/org/apache/calcite/test/JdbcFrontLinqBackTest.java
index ff8dce3..90d1352 100644
--- a/core/src/test/java/org/apache/calcite/test/JdbcFrontLinqBackTest.java
+++ b/core/src/test/java/org/apache/calcite/test/JdbcFrontLinqBackTest.java
@@ -394,9 +394,7 @@ public class JdbcFrontLinqBackTest {
         .returns("C=6\n");
   }
 
-  /**
-   * Local Statement insert
-   */
+  /** Local Statement insert. */
   @Test void testInsert3() throws Exception {
     Connection connection = makeConnection(new ArrayList<JdbcTest.Employee>());
     String sql = "insert into \"foo\".\"bar\" values (1, 1, 'second', 2, 2)";
@@ -410,9 +408,7 @@ public class JdbcFrontLinqBackTest {
     assertTrue(updateCount == 1);
   }
 
-  /**
-   * Local PreparedStatement insert WITHOUT bind variables
-   */
+  /** Local PreparedStatement insert WITHOUT bind variables. */
   @Test void testPreparedStatementInsert() throws Exception {
     Connection connection = makeConnection(new ArrayList<JdbcTest.Employee>());
     assertFalse(connection.isClosed());
@@ -429,9 +425,7 @@ public class JdbcFrontLinqBackTest {
     assertTrue(updateCount == 1);
   }
 
-  /**
-   * Local PreparedStatement insert WITH bind variables
-   */
+  /** Local PreparedStatement insert WITH bind variables. */
   @Test void testPreparedStatementInsert2() throws Exception {
   }
 
diff --git a/core/src/test/java/org/apache/calcite/test/JdbcTest.java b/core/src/test/java/org/apache/calcite/test/JdbcTest.java
index dc87f16..2322d89 100644
--- a/core/src/test/java/org/apache/calcite/test/JdbcTest.java
+++ b/core/src/test/java/org/apache/calcite/test/JdbcTest.java
@@ -933,7 +933,7 @@ public class JdbcTest {
    * and also
    * <a href="https://issues.apache.org/jira/browse/CALCITE-1222">[CALCITE-1222]
    * DatabaseMetaData.getColumnLabel returns null when query has ORDER
-   * BY</a>, */
+   * BY</a>. */
   @Test void testResultSetMetaData()
       throws ClassNotFoundException, SQLException {
     try (Connection connection =
@@ -1115,6 +1115,25 @@ public class JdbcTest {
             + "c0=1998\n");
   }
 
+  /** Test case for
+   * <a href="https://issues.apache.org/jira/browse/CALCITE-2894">[CALCITE-2894]
+   * NullPointerException thrown by RelMdPercentageOriginalRows when explaining
+   * plan with all attributes</a>. */
+  @Test void testExplainAllAttributesSemiJoinUnionCorrelate() {
+    final String sql = "select deptno, name from depts where deptno in (\n"
+        + "  select e.deptno from emps e where exists (\n"
+        + "     select 1 from depts d where d.deptno = e.deptno)\n"
+        + "   union\n"
+        + "   select e.deptno from emps e where e.salary > 10000)";
+    CalciteAssert.that()
+        .with(CalciteConnectionProperty.LEX, Lex.JAVA)
+        .with(CalciteConnectionProperty.FORCE_DECORRELATE, false)
+        .withSchema("s", new ReflectiveSchema(new JdbcTest.HrSchema()))
+        .query(sql)
+        .explainMatches("including all attributes ",
+            CalciteAssert.checkResultContains("EnumerableCorrelate"));
+  }
+
   /** Just short of bushy. */
   @Test void testAlmostBushy() {
     CalciteAssert.that()
@@ -3858,7 +3877,7 @@ public class JdbcTest {
             "deptno=20; empid=200; commission=500; RCNF=1; RCNL=1; R=1; RD=1");
   }
 
-  /** Tests for RANK with same values */
+  /** Tests for RANK with same values. */
   @Test void testWinAggRankValues() {
     CalciteAssert.hr()
         .query("select  \"deptno\",\n"
@@ -3873,7 +3892,7 @@ public class JdbcTest {
             "deptno=20; R=4"); // 4 for rank and 2 for dense_rank
   }
 
-  /** Tests for RANK with same values */
+  /** Tests for RANK with same values. */
   @Test void testWinAggRankValuesDesc() {
     CalciteAssert.hr()
         .query("select  \"deptno\",\n"
@@ -3888,7 +3907,7 @@ public class JdbcTest {
             "deptno=20; R=1");
   }
 
-  /** Tests for DENSE_RANK with same values */
+  /** Tests for DENSE_RANK with same values. */
   @Test void testWinAggDenseRankValues() {
     CalciteAssert.hr()
         .query("select  \"deptno\",\n"
@@ -3903,7 +3922,7 @@ public class JdbcTest {
             "deptno=20; R=2");
   }
 
-  /** Tests for DENSE_RANK with same values */
+  /** Tests for DENSE_RANK with same values. */
   @Test void testWinAggDenseRankValuesDesc() {
     CalciteAssert.hr()
         .query("select  \"deptno\",\n"
@@ -3918,7 +3937,7 @@ public class JdbcTest {
             "deptno=20; R=1");
   }
 
-  /** Tests for DATE +- INTERVAL window frame */
+  /** Tests for DATE +- INTERVAL window frame. */
   @Test void testWinIntervalFrame() {
     CalciteAssert.hr()
         .query("select  \"deptno\",\n"
@@ -4296,7 +4315,7 @@ public class JdbcTest {
             "Cannot apply 'NTILE' to arguments of type 'NTILE(<DECIMAL(10, 9)>)'");
   }
 
-  /** Tests for FIRST_VALUE */
+  /** Tests for FIRST_VALUE. */
   @Test void testWinAggFirstValue() {
     CalciteAssert.hr()
         .query("select  \"deptno\",\n"
@@ -4313,7 +4332,7 @@ public class JdbcTest {
             "deptno=20; empid=200; commission=500; R=500");
   }
 
-  /** Tests for FIRST_VALUE desc */
+  /** Tests for FIRST_VALUE desc. */
   @Test void testWinAggFirstValueDesc() {
     CalciteAssert.hr()
         .query("select  \"deptno\",\n"
@@ -4330,7 +4349,7 @@ public class JdbcTest {
             "deptno=20; empid=200; commission=500; R=500");
   }
 
-  /** Tests for FIRST_VALUE empty window */
+  /** Tests for FIRST_VALUE empty window. */
   @Test void testWinAggFirstValueEmptyWindow() {
     CalciteAssert.hr()
         .query("select \"deptno\",\n"
@@ -4347,7 +4366,7 @@ public class JdbcTest {
             "deptno=20; empid=200; commission=500; R=null");
   }
 
-  /** Tests for ROW_NUMBER */
+  /** Tests for ROW_NUMBER. */
   @Test void testWinRowNumber() {
     CalciteAssert.hr()
         .query("select \"deptno\",\n"
@@ -4637,7 +4656,9 @@ public class JdbcTest {
             "DEPTNO=null; G=2; C=14");
   }
 
-  /** Tests CALCITE-980: Not (C='a' or C='b') causes NPE */
+  /** Tests
+   * <a href="https://issues.apache.org/jira/browse/CALCITE-980">[CALCITE-980]
+   * Not (C='a' or C='b') causes NPE</a>. */
   @Test void testWhereOrAndNullable() {
     /* Generates the following code:
        public boolean moveNext() {
diff --git a/core/src/test/java/org/apache/calcite/test/LogicalProjectDigestTest.java b/core/src/test/java/org/apache/calcite/test/LogicalProjectDigestTest.java
index 043271d..78c1920 100644
--- a/core/src/test/java/org/apache/calcite/test/LogicalProjectDigestTest.java
+++ b/core/src/test/java/org/apache/calcite/test/LogicalProjectDigestTest.java
@@ -34,9 +34,7 @@ import static org.hamcrest.MatcherAssert.assertThat;
  * Verifies digest for {@link LogicalProject}.
  */
 class LogicalProjectDigestTest {
-  /**
-   * Planner does not compare
-   */
+  /** Planner does not compare. */
   @Test void fieldNamesDoNotInfluenceDigest() {
     final RelBuilder rb = RelBuilder.create(Frameworks.newConfigBuilder().build());
     final RelNode xAsEmpid = rb.values(new String[]{"x", "y", "z"}, 1, 2, 3)
diff --git a/core/src/test/java/org/apache/calcite/test/Matchers.java b/core/src/test/java/org/apache/calcite/test/Matchers.java
index 48ca5e3..5566cf4 100644
--- a/core/src/test/java/org/apache/calcite/test/Matchers.java
+++ b/core/src/test/java/org/apache/calcite/test/Matchers.java
@@ -278,8 +278,8 @@ public class Matchers {
     };
   }
 
-  /**
-   * Is the numeric value within a given difference another value?
+  /** Matcher that tests whether the numeric value is within a given difference
+   * another value.
    *
    * @param <T> Value type
    */
diff --git a/core/src/test/java/org/apache/calcite/test/MutableRelTest.java b/core/src/test/java/org/apache/calcite/test/MutableRelTest.java
index 90e90b2..27f6fa8 100644
--- a/core/src/test/java/org/apache/calcite/test/MutableRelTest.java
+++ b/core/src/test/java/org/apache/calcite/test/MutableRelTest.java
@@ -219,7 +219,7 @@ class MutableRelTest {
     assertEquals(mutableRel1, mutableRel2);
   }
 
-  /** Verifies equivalence of {@link MutableScan} */
+  /** Verifies equivalence of {@link MutableScan}. */
   @Test public void testMutableScanEquivalence() {
     final FrameworkConfig config = RelBuilderTest.config().build();
     final RelBuilder builder = RelBuilder.create(config);
diff --git a/core/src/test/java/org/apache/calcite/test/ReflectiveSchemaTest.java b/core/src/test/java/org/apache/calcite/test/ReflectiveSchemaTest.java
index 812060a..859d2fb 100644
--- a/core/src/test/java/org/apache/calcite/test/ReflectiveSchemaTest.java
+++ b/core/src/test/java/org/apache/calcite/test/ReflectiveSchemaTest.java
@@ -296,8 +296,8 @@ public class ReflectiveSchemaTest {
             + "primitiveBoolean=true; primitiveByte=127; primitiveChar=\uffff; primitiveShort=32767; primitiveInt=2147483647; primitiveLong=9223372036854775807; primitiveFloat=3.4028235E38; primitiveDouble=1.7976931348623157E308; wrapperBoolean=null; wrapperByte=null; wrapperCharacter=null; wrapperShort=null; wrapperInteger=null; wrapperLong=null; wrapperFloat=null; wrapperDouble=null; sqlDate=null; sqlTime=null; sqlTimestamp=null; utilDate=null; string=null; bigDecimal=null\n");
   }
 
-  /**
-   * Tests NOT for nullable columns
+  /** Tests NOT for nullable columns.
+   *
    * @see CatchallSchema#everyTypes */
   @Test void testWhereNOT() throws Exception {
     final CalciteAssert.AssertThat with =
@@ -307,8 +307,8 @@ public class ReflectiveSchemaTest {
         .returnsUnordered("wrapperByte=0");
   }
 
-  /**
-   * Tests NOT for nullable columns
+  /** Tests NOT for nullable columns.
+   *
    * @see CatchallSchema#everyTypes */
   @Test void testSelectNOT() throws Exception {
     final CalciteAssert.AssertThat with =
@@ -761,7 +761,7 @@ public class ReflectiveSchemaTest {
                 + "empid=4; deptno=10; name=Abd; salary=0.0; commission=null\n");
   }
 
-  /** Table with single field as Integer[] */
+  /** Table with single field as Integer[]. */
   @Disabled(
       "java.lang.AssertionError RelDataTypeImpl.getFieldList(RelDataTypeImpl.java:99)")
   @Test void testArrayOfBoxedPrimitives() {
@@ -771,7 +771,7 @@ public class ReflectiveSchemaTest {
         .returnsUnordered("value=1", "value=3", "value=7");
   }
 
-  /** Table with single field as int[] */
+  /** Table with single field as int[]. */
   @Disabled(
       "java.lang.AssertionError RelDataTypeImpl.getFieldList(RelDataTypeImpl.java:99)")
   @Test void testArrayOfPrimitives() {
@@ -977,7 +977,7 @@ public class ReflectiveSchemaTest {
     public final BitSet bitSet = new BitSet(0);
   }
 
-  /** Table that has integer and string fields */
+  /** Table that has integer and string fields. */
   public static class IntAndString {
     public final int id;
     public final String value;
@@ -1062,7 +1062,9 @@ public class ReflectiveSchemaTest {
     };
   }
 
-  /** CALCITE-2611 unknown on one side of an or may lead to uncompilable code */
+  /** Tests
+   * <a href="https://issues.apache.org/jira/browse/CALCITE-2611">[CALCITE-2611]
+   * UNKNOWN on one side of an OR may lead to uncompilable code</a>. */
   @Test void testUnknownInOr() {
     CalciteAssert.that()
         .withSchema("s", CATCHALL)
diff --git a/core/src/test/java/org/apache/calcite/test/RelBuilderTest.java b/core/src/test/java/org/apache/calcite/test/RelBuilderTest.java
index 8c213f7..fa80fd6 100644
--- a/core/src/test/java/org/apache/calcite/test/RelBuilderTest.java
+++ b/core/src/test/java/org/apache/calcite/test/RelBuilderTest.java
@@ -3177,7 +3177,7 @@ public class RelBuilderTest {
     assertThat(root, hasTree(expected));
   }
 
-  /** Tests filter builder with correlation variables */
+  /** Tests filter builder with correlation variables. */
   @Test void testFilterWithCorrelationVariables() {
     final RelBuilder builder = RelBuilder.create(config().build());
     final Holder<RexCorrelVariable> v = Holder.of(null);
@@ -3225,7 +3225,9 @@ public class RelBuilderTest {
     assertThat(root, hasTree("LogicalTableScan(table=[[scott, EMP]])\n"));
   }
 
-  /** Checks if simplification is run in {@link org.apache.calcite.rex.RexUnknownAs#FALSE} mode for filter conditions */
+  /** Checks if simplification is run in
+   * {@link org.apache.calcite.rex.RexUnknownAs#FALSE} mode for filter
+   * conditions. */
   @Test void testFilterSimplification() {
     final RelBuilder builder = RelBuilder.create(config().build());
     final RelNode root =
diff --git a/core/src/test/java/org/apache/calcite/test/RelMdPercentageOriginalRowsTest.java b/core/src/test/java/org/apache/calcite/test/RelMdPercentageOriginalRowsTest.java
deleted file mode 100644
index c3afeb9..0000000
--- a/core/src/test/java/org/apache/calcite/test/RelMdPercentageOriginalRowsTest.java
+++ /dev/null
@@ -1,43 +0,0 @@
-/*
- * 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.test;
-
-import org.apache.calcite.adapter.java.ReflectiveSchema;
-import org.apache.calcite.config.CalciteConnectionProperty;
-import org.apache.calcite.config.Lex;
-
-import org.junit.jupiter.api.Test;
-
-/** Test case for CALCITE-2894 */
-class RelMdPercentageOriginalRowsTest {
-  /** Test case for
-   * <a href="https://issues.apache.org/jira/browse/CALCITE-2894">[CALCITE-2894]
-   * NullPointerException thrown by RelMdPercentageOriginalRows when explaining
-   * plan with all attributes</a>. */
-  @Test void testExplainAllAttributesSemiJoinUnionCorrelate() {
-    CalciteAssert.that()
-            .with(CalciteConnectionProperty.LEX, Lex.JAVA)
-            .with(CalciteConnectionProperty.FORCE_DECORRELATE, false)
-            .withSchema("s", new ReflectiveSchema(new JdbcTest.HrSchema()))
-            .query(
-                    "select deptno, name from depts where deptno in (\n"
-                            + " select e.deptno from emps e where exists (select 1 from depts d where d.deptno=e.deptno)\n"
-                            + " union select e.deptno from emps e where e.salary > 10000) ")
-            .explainMatches("including all attributes ",
-                    CalciteAssert.checkResultContains("EnumerableCorrelate"));
-  }
-}
diff --git a/core/src/test/java/org/apache/calcite/test/RelOptRulesTest.java b/core/src/test/java/org/apache/calcite/test/RelOptRulesTest.java
index 4040570..5c6b99a 100644
--- a/core/src/test/java/org/apache/calcite/test/RelOptRulesTest.java
+++ b/core/src/test/java/org/apache/calcite/test/RelOptRulesTest.java
@@ -2158,7 +2158,7 @@ class RelOptRulesTest extends RelOptTestBase {
     sql(sql).with(program).check();
   }
 
-  /** Tests to see if the final branch of union is missed */
+  /** Tests to see if the final branch of union is missed. */
   @Test void testUnionMergeRule() {
     HepProgram program = new HepProgramBuilder()
         .addRuleInstance(CoreRules.PROJECT_SET_OP_TRANSPOSE)
@@ -4357,8 +4357,7 @@ class RelOptRulesTest extends RelOptTestBase {
 
   /** Test case for
    * <a href="https://issues.apache.org/jira/browse/CALCITE-3111">[CALCITE-3111]
-   * Allow custom implementations of Correlate in RelDecorrelator </a>
-   */
+   * Allow custom implementations of Correlate in RelDecorrelator</a>. */
   @Test void testCustomDecorrelate() {
     final String sql = "SELECT e1.empno\n"
         + "FROM emp e1, dept d1 where e1.deptno = d1.deptno\n"
@@ -4510,9 +4509,9 @@ class RelOptRulesTest extends RelOptTestBase {
         .check();
   }
 
-  /** Test case for DX-11490
+  /** Test case for DX-11490:
    * Make sure the planner doesn't fail over wrong push down
-   * of is null */
+   * of is null. */
   @Test void testIsNullPushDown() {
     HepProgramBuilder preBuilder = new HepProgramBuilder();
     preBuilder.addRuleInstance(CoreRules.PROJECT_TO_LOGICAL_PROJECT_AND_WINDOW);
@@ -4591,8 +4590,8 @@ class RelOptRulesTest extends RelOptTestBase {
         .check();
   }
 
-  /** Test case for
-   * outer join, group by on non-join keys, group by on non-null generating side only */
+  /** Test case for outer join, group by on non-join keys, group by on
+   * non-null generating side only. */
   @Test void testPushAggregateThroughOuterJoin1() {
     final HepProgram preProgram = new HepProgramBuilder()
         .addRuleInstance(CoreRules.AGGREGATE_PROJECT_MERGE)
@@ -4606,8 +4605,8 @@ class RelOptRulesTest extends RelOptTestBase {
         .check();
   }
 
-  /** Test case for
-   * outer join, group by on non-join keys, on null generating side only */
+  /** Test case for outer join, group by on non-join keys, on null
+   * generating side only. */
   @Test void testPushAggregateThroughOuterJoin2() {
     final HepProgram preProgram = new HepProgramBuilder()
         .addRuleInstance(CoreRules.AGGREGATE_PROJECT_MERGE)
@@ -4621,8 +4620,8 @@ class RelOptRulesTest extends RelOptTestBase {
         .check();
   }
 
-  /** Test case for
-   * outer join, group by on both side on non-join keys */
+  /** Test case for outer join, group by on both side on non-join
+   * keys. */
   @Test void testPushAggregateThroughOuterJoin3() {
     final HepProgram preProgram = new HepProgramBuilder()
         .addRuleInstance(CoreRules.AGGREGATE_PROJECT_MERGE)
@@ -4636,8 +4635,8 @@ class RelOptRulesTest extends RelOptTestBase {
         .check();
   }
 
-  /** Test case for
-   * outer join, group by on key same as join key, group by on non-null generating side */
+  /** Test case for outer join, group by on key same as join key,
+   * group by on non-null generating side. */
   @Test void testPushAggregateThroughOuterJoin4() {
     final HepProgram preProgram = new HepProgramBuilder()
         .addRuleInstance(CoreRules.AGGREGATE_PROJECT_MERGE)
@@ -4651,8 +4650,8 @@ class RelOptRulesTest extends RelOptTestBase {
         .check();
   }
 
-  /** Test case for
-   * outer join, group by on key same as join key, group by on null generating side */
+  /** Test case for outer join, group by on key same as join key,
+   * group by on null generating side. */
   @Test void testPushAggregateThroughOuterJoin5() {
     final HepProgram preProgram = new HepProgramBuilder()
         .addRuleInstance(CoreRules.AGGREGATE_PROJECT_MERGE)
@@ -4666,8 +4665,8 @@ class RelOptRulesTest extends RelOptTestBase {
         .check();
   }
 
-  /** Test case for
-   * outer join, group by on key same as join key, group by on both side */
+  /** Test case for outer join, group by on key same as join key,
+   * group by on both side. */
   @Test void testPushAggregateThroughOuterJoin6() {
     final HepProgram preProgram = new HepProgramBuilder()
         .addRuleInstance(CoreRules.AGGREGATE_PROJECT_MERGE)
@@ -4681,8 +4680,8 @@ class RelOptRulesTest extends RelOptTestBase {
         .check();
   }
 
-  /** Test case for
-   * outer join, group by key is susbset of join keys, group by on non-null generating side */
+  /** Test case for outer join, group by key is susbset of join keys,
+   * group by on non-null generating side. */
   @Test void testPushAggregateThroughOuterJoin7() {
     final HepProgram preProgram = new HepProgramBuilder()
         .addRuleInstance(CoreRules.AGGREGATE_PROJECT_MERGE)
@@ -4697,8 +4696,8 @@ class RelOptRulesTest extends RelOptTestBase {
         .check();
   }
 
-  /** Test case for
-   * outer join, group by key is susbset of join keys, group by on null generating side */
+  /** Test case for outer join, group by key is susbset of join keys,
+   * group by on null generating side. */
   @Test void testPushAggregateThroughOuterJoin8() {
     final HepProgram preProgram = new HepProgramBuilder()
         .addRuleInstance(CoreRules.AGGREGATE_PROJECT_MERGE)
@@ -4713,8 +4712,8 @@ class RelOptRulesTest extends RelOptTestBase {
         .check();
   }
 
-  /** Test case for
-   * outer join, group by key is susbset of join keys, group by on both sides */
+  /** Test case for outer join, group by key is susbset of join keys,
+   * group by on both sides. */
   @Test void testPushAggregateThroughOuterJoin9() {
     final HepProgram preProgram = new HepProgramBuilder()
         .addRuleInstance(CoreRules.AGGREGATE_PROJECT_MERGE)
@@ -4729,8 +4728,7 @@ class RelOptRulesTest extends RelOptTestBase {
         .check();
   }
 
-  /** Test case for
-   * outer join, with aggregate functions */
+  /** Test case for outer join, with aggregate functions. */
   @Test void testPushAggregateThroughOuterJoin10() {
     final HepProgram preProgram = new HepProgramBuilder()
         .addRuleInstance(CoreRules.AGGREGATE_PROJECT_MERGE)
@@ -4744,8 +4742,7 @@ class RelOptRulesTest extends RelOptTestBase {
         .checkUnchanged();
   }
 
-  /** Test case for
-   * non-equi outer join */
+  /** Test case for non-equi outer join. */
   @Test void testPushAggregateThroughOuterJoin11() {
     final HepProgram preProgram = new HepProgramBuilder()
         .addRuleInstance(CoreRules.AGGREGATE_PROJECT_MERGE)
@@ -4759,8 +4756,8 @@ class RelOptRulesTest extends RelOptTestBase {
         .checkUnchanged();
   }
 
-  /** Test case for
-   * right outer join, group by on key same as join key, group by on (left)null generating side */
+  /** Test case for right outer join, group by on key same as join
+   * key, group by on (left)null generating side. */
   @Test void testPushAggregateThroughOuterJoin12() {
     final HepProgram preProgram = new HepProgramBuilder()
         .addRuleInstance(CoreRules.AGGREGATE_PROJECT_MERGE)
@@ -4774,8 +4771,8 @@ class RelOptRulesTest extends RelOptTestBase {
         .check();
   }
 
-  /** Test case for
-   * full outer join, group by on key same as join key, group by on one side */
+  /** Test case for full outer join, group by on key same as join key,
+   * group by on one side. */
   @Test void testPushAggregateThroughOuterJoin13() {
     final HepProgram preProgram = new HepProgramBuilder()
         .addRuleInstance(CoreRules.AGGREGATE_PROJECT_MERGE)
@@ -4789,8 +4786,8 @@ class RelOptRulesTest extends RelOptTestBase {
         .check();
   }
 
-  /** Test case for
-   * full outer join, group by on key same as join key, group by on both side */
+  /** Test case for full outer join, group by on key same as join key,
+   * group by on both side. */
   @Test void testPushAggregateThroughOuterJoin14() {
     final HepProgram preProgram = new HepProgramBuilder()
         .addRuleInstance(CoreRules.AGGREGATE_PROJECT_MERGE)
@@ -4804,8 +4801,8 @@ class RelOptRulesTest extends RelOptTestBase {
         .check();
   }
 
-  /** Test case for
-   * full outer join, group by on both side on non-join keys */
+  /** Test case for full outer join, group by on both side on non-join
+   * keys. */
   @Test void testPushAggregateThroughOuterJoin15() {
     final HepProgram preProgram = new HepProgramBuilder()
         .addRuleInstance(CoreRules.AGGREGATE_PROJECT_MERGE)
@@ -4819,8 +4816,8 @@ class RelOptRulesTest extends RelOptTestBase {
         .check();
   }
 
-  /** Test case for
-   * full outer join, group by key is susbset of join keys */
+  /** Test case for full outer join, group by key is susbset of join
+   * keys. */
   @Test void testPushAggregateThroughOuterJoin16() {
     final HepProgram preProgram = new HepProgramBuilder()
         .addRuleInstance(CoreRules.AGGREGATE_PROJECT_MERGE)
@@ -5377,10 +5374,8 @@ class RelOptRulesTest extends RelOptTestBase {
         .checkUnchanged();
   }
 
-  /**
-   * The top Aggregate should be removed -- given "deptno=100",
-   * the input of top Aggregate must be already distinct by "mgr"
-   */
+  /** Tests that top Aggregate is removed. Given "deptno=100", the
+   * input of top Aggregate must be already distinct by "mgr". */
   @Test void testAggregateRemove7() {
     final HepProgram program = new HepProgramBuilder()
         .addRuleInstance(CoreRules.AGGREGATE_REMOVE)
@@ -6705,8 +6700,8 @@ class RelOptRulesTest extends RelOptTestBase {
 
   /** Test case for
    * <a href="https://issues.apache.org/jira/browse/CALCITE-3151">[CALCITE-3151]
-   * RexCall's Monotonicity is not considered in determining a Calc's collation</a>
-   */
+   * RexCall's Monotonicity is not considered in determining a Calc's
+   * collation</a>. */
   @Test void testMonotonicityUDF() {
     final SqlFunction monotonicityFun =
         new SqlFunction("MONOFUN", SqlKind.OTHER_FUNCTION, ReturnTypes.BIGINT, null,
@@ -6761,8 +6756,8 @@ class RelOptRulesTest extends RelOptTestBase {
 
   /** Test case for
    * <a href="https://issues.apache.org/jira/browse/CALCITE-3997">[CALCITE-3997]
-   * Logical rules applied on physical operator but failed handle traits</a>
-   */
+   * Logical rules applied on physical operator but failed handle
+   * traits</a>. */
   @Test void testMergeJoinCollation() {
     final String sql = "select r.ename, s.sal from\n"
         + "sales.emp r join sales.bonus s\n"
@@ -6790,7 +6785,7 @@ class RelOptRulesTest extends RelOptTestBase {
    * Custom implementation of {@link Filter} for use
    * in test case to verify that {@link FilterMultiJoinMergeRule}
    * can be created with any {@link Filter} and not limited to
-   * {@link org.apache.calcite.rel.logical.LogicalFilter}
+   * {@link org.apache.calcite.rel.logical.LogicalFilter}.
    */
   private static class MyFilter extends Filter {
 
@@ -6844,7 +6839,7 @@ class RelOptRulesTest extends RelOptTestBase {
    * Custom implementation of {@link Project} for use
    * in test case to verify that {@link ProjectMultiJoinMergeRule}
    * can be created with any {@link Project} and not limited to
-   * {@link org.apache.calcite.rel.logical.LogicalProject}
+   * {@link org.apache.calcite.rel.logical.LogicalProject}.
    */
   private static class MyProject extends Project {
     MyProject(
@@ -6967,8 +6962,8 @@ class RelOptRulesTest extends RelOptTestBase {
 
   /** Test case for
    * <a href="https://issues.apache.org/jira/browse/CALCITE-3296">[CALCITE-3296]
-   * Decorrelator gives empty result
-   * after decorrelating sort rel with null offset and fetch </a>
+   * Decorrelator gives empty result after decorrelating sort rel with
+   * null offset and fetch</a>.
    */
   @Test void testDecorrelationWithSort() {
     final String sql = "SELECT e1.empno\n"
@@ -6985,8 +6980,7 @@ class RelOptRulesTest extends RelOptTestBase {
   /**
    * Test case for
    * <a href="https://issues.apache.org/jira/browse/CALCITE-3319">[CALCITE-3319]
-   * AssertionError for ReduceDecimalsRule </a>
-   */
+   * AssertionError for ReduceDecimalsRule</a>. */
   @Test void testReduceDecimal() {
     final HepProgram program = new HepProgramBuilder()
         .addRuleInstance(CoreRules.FILTER_TO_CALC)
@@ -7033,10 +7027,9 @@ class RelOptRulesTest extends RelOptTestBase {
   /**
    * Test case for
    * <a href="https://issues.apache.org/jira/browse/CALCITE-3404">[CALCITE-3404]
-   * Treat agg expressions that can ignore distinct constraint as distinct
-   * in AggregateExpandDistinctAggregatesRule
-   * when all the other agg expressions are distinct and have same arguments</a>
-   */
+   * Treat agg expressions that can ignore distinct constraint as
+   * distinct in AggregateExpandDistinctAggregatesRule when all the
+   * other agg expressions are distinct and have same arguments</a>. */
   @Test void testMaxReuseDistinctAttrWithMixedOptionality() {
     final String sql = "select sum(distinct deptno), count(distinct deptno), "
         + "max(deptno) from emp";
diff --git a/core/src/test/java/org/apache/calcite/test/RelOptTestBase.java b/core/src/test/java/org/apache/calcite/test/RelOptTestBase.java
index 6f3ee5c..beb3a9c 100644
--- a/core/src/test/java/org/apache/calcite/test/RelOptTestBase.java
+++ b/core/src/test/java/org/apache/calcite/test/RelOptTestBase.java
@@ -203,6 +203,7 @@ abstract class RelOptTestBase extends SqlToRelTestBase {
           FlatLists.append(hooks, hook, handler), transforms);
     }
 
+    // CHECKSTYLE: IGNORE 1
     /** @deprecated Use {@link #withHook(Hook, Consumer)}. */
     @SuppressWarnings("Guava")
     @Deprecated // to be removed before 2.0
diff --git a/core/src/test/java/org/apache/calcite/test/RexShuttleTest.java b/core/src/test/java/org/apache/calcite/test/RexShuttleTest.java
index 0399cb8..aed5304 100644
--- a/core/src/test/java/org/apache/calcite/test/RexShuttleTest.java
+++ b/core/src/test/java/org/apache/calcite/test/RexShuttleTest.java
@@ -38,7 +38,7 @@ import static org.hamcrest.CoreMatchers.is;
 import static org.hamcrest.MatcherAssert.assertThat;
 
 /**
- * Unit tests for {@link RexShuttle}
+ * Unit tests for {@link RexShuttle}.
  */
 class RexShuttleTest {
 
diff --git a/core/src/test/java/org/apache/calcite/test/SqlToRelConverterTest.java b/core/src/test/java/org/apache/calcite/test/SqlToRelConverterTest.java
index b0d2396..e05ef07 100644
--- a/core/src/test/java/org/apache/calcite/test/SqlToRelConverterTest.java
+++ b/core/src/test/java/org/apache/calcite/test/SqlToRelConverterTest.java
@@ -1319,7 +1319,7 @@ class SqlToRelConverterTest extends SqlToRelTestBase {
    * <a href="https://issues.apache.org/jira/browse/CALCITE-3789">[CALCITE-3789]
    * Support validation of UNNEST multiple array columns like Presto</a>.
    */
-  @Test public void testAliasUnnestArrayPlanWithSingleColumn() {
+  @Test void testAliasUnnestArrayPlanWithSingleColumn() {
     final String sql = "select d.deptno, employee.empno\n"
         + "from dept_nested_expanded as d,\n"
         + " UNNEST(d.employees) as t(employee)";
@@ -1331,7 +1331,7 @@ class SqlToRelConverterTest extends SqlToRelTestBase {
    * <a href="https://issues.apache.org/jira/browse/CALCITE-3789">[CALCITE-3789]
    * Support validation of UNNEST multiple array columns like Presto</a>.
    */
-  @Test public void testAliasUnnestArrayPlanWithDoubleColumn() {
+  @Test void testAliasUnnestArrayPlanWithDoubleColumn() {
     final String sql = "select d.deptno, e, k.empno\n"
         + "from dept_nested_expanded as d CROSS JOIN\n"
         + " UNNEST(d.admins, d.employees) as t(e, k)";
@@ -1828,21 +1828,21 @@ class SqlToRelConverterTest extends SqlToRelTestBase {
     sql(sql).ok();
   }
 
-  @Test public void testTableFunctionHop() {
+  @Test void testTableFunctionHop() {
     final String sql = "select *\n"
         + "from table(hop(table Shipments, descriptor(rowtime), "
         + "INTERVAL '1' MINUTE, INTERVAL '2' MINUTE))";
     sql(sql).ok();
   }
 
-  @Test public void testTableFunctionHopWithOffset() {
+  @Test void testTableFunctionHopWithOffset() {
     final String sql = "select *\n"
         + "from table(hop(table Shipments, descriptor(rowtime), "
         + "INTERVAL '1' MINUTE, INTERVAL '5' MINUTE, INTERVAL '3' MINUTE))";
     sql(sql).ok();
   }
 
-  @Test public void testTableFunctionSession() {
+  @Test void testTableFunctionSession() {
     final String sql = "select *\n"
         + "from table(session(table Shipments, descriptor(rowtime), "
         + "descriptor(orderId), INTERVAL '10' MINUTE))";
@@ -1855,21 +1855,21 @@ class SqlToRelConverterTest extends SqlToRelTestBase {
     sql(sql).ok();
   }
 
-  @Test public void testTableFunctionHopWithSubQueryParam() {
+  @Test void testTableFunctionHopWithSubQueryParam() {
     final String sql = "select *\n"
         + "from table(hop((select * from Shipments), descriptor(rowtime), "
         + "INTERVAL '1' MINUTE, INTERVAL '2' MINUTE))";
     sql(sql).ok();
   }
 
-  @Test public void testTableFunctionSessionWithSubQueryParam() {
+  @Test void testTableFunctionSessionWithSubQueryParam() {
     final String sql = "select *\n"
         + "from table(session((select * from Shipments), descriptor(rowtime), "
         + "descriptor(orderId), INTERVAL '10' MINUTE))";
     sql(sql).ok();
   }
 
-  @Test public void testTableFunctionSessionCompoundSessionKey() {
+  @Test void testTableFunctionSessionCompoundSessionKey() {
     final String sql = "select *\n"
         + "from table(session(table Orders, descriptor(rowtime), "
         + "descriptor(orderId, productId), INTERVAL '10' MINUTE))";
@@ -2151,7 +2151,7 @@ class SqlToRelConverterTest extends SqlToRelTestBase {
       @Override public RelOptPlanner createPlanner() {
         return new MockRelOptPlanner(Contexts.empty()) {
           @Override public List<RelTraitDef> getRelTraitDefs() {
-            return ImmutableList.<RelTraitDef>of(RelCollationTraitDef.INSTANCE);
+            return ImmutableList.of(RelCollationTraitDef.INSTANCE);
           }
           @Override public RelTraitSet emptyTraitSet() {
             return RelTraitSet.createEmpty().plus(
@@ -2232,9 +2232,7 @@ class SqlToRelConverterTest extends SqlToRelTestBase {
     sql(sql).ok();
   }
 
-  /**
-   * Test group-by CASE expression involving a non-query IN
-   */
+  /** Tests group-by CASE expression involving a non-query IN. */
   @Test void testGroupByCaseSubQuery() {
     final String sql = "SELECT CASE WHEN emp.empno IN (3) THEN 0 ELSE 1 END\n"
         + "FROM emp\n"
@@ -2242,9 +2240,8 @@ class SqlToRelConverterTest extends SqlToRelTestBase {
     sql(sql).ok();
   }
 
-  /**
-   * Test aggregate function on a CASE expression involving a non-query IN
-   */
+  /** Tests an aggregate function on a CASE expression involving a non-query
+   * IN. */
   @Test void testAggCaseSubQuery() {
     final String sql =
         "SELECT SUM(CASE WHEN empno IN (3) THEN 0 ELSE 1 END) FROM emp";
@@ -2965,21 +2962,16 @@ class SqlToRelConverterTest extends SqlToRelTestBase {
     sql(sql).ok();
   }
 
-  /**
-   * Test case for
+  /** Test case for
    * <a href="https://issues.apache.org/jira/browse/CALCITE-1150">[CALCITE-1150]
-   * Dynamic Table / Dynamic Star support</a>
-   */
-  @Test void testSelectFromDynamicTable() throws Exception {
+   * Dynamic Table / Dynamic Star support</a>. */
+  @Test void testSelectFromDynamicTable() {
     final String sql = "select n_nationkey, n_name from SALES.NATION";
     sql(sql).with(getTesterWithDynamicTable()).ok();
   }
 
-  /**
-   * Test case for Dynamic Table / Dynamic Star support
-   * <a href="https://issues.apache.org/jira/browse/CALCITE-1150">[CALCITE-1150]</a>
-   */
-  @Test void testSelectStarFromDynamicTable() throws Exception {
+  /** As {@link #testSelectFromDynamicTable} but "SELECT *". */
+  @Test void testSelectStarFromDynamicTable() {
     final String sql = "select * from SALES.NATION";
     sql(sql).with(getTesterWithDynamicTable()).ok();
   }
@@ -2997,22 +2989,16 @@ class SqlToRelConverterTest extends SqlToRelTestBase {
     sql(sql).with(getTesterWithDynamicTable()).ok();
   }
 
-  /**
-   * Test case for Dynamic Table / Dynamic Star support
-   * <a href="https://issues.apache.org/jira/browse/CALCITE-1150">[CALCITE-1150]</a>
-   */
-  @Test void testReferDynamicStarInSelectOB() throws Exception {
+  /** As {@link #testSelectFromDynamicTable} but with ORDER BY. */
+  @Test void testReferDynamicStarInSelectOB() {
     final String sql = "select n_nationkey, n_name\n"
         + "from (select * from SALES.NATION)\n"
         + "order by n_regionkey";
     sql(sql).with(getTesterWithDynamicTable()).ok();
   }
 
-  /**
-   * Test case for Dynamic Table / Dynamic Star support
-   * <a href="https://issues.apache.org/jira/browse/CALCITE-1150">[CALCITE-1150]</a>
-   */
-  @Test void testDynamicStarInTableJoin() throws Exception {
+  /** As {@link #testSelectFromDynamicTable} but with join. */
+  @Test void testDynamicStarInTableJoin() {
     final String sql = "select * from "
         + " (select * from SALES.NATION) T1, "
         + " (SELECT * from SALES.CUSTOMER) T2 "
@@ -3028,13 +3014,13 @@ class SqlToRelConverterTest extends SqlToRelTestBase {
     sql(sql).with(getTesterWithDynamicTable()).ok();
   }
 
-  /**
-   * Test case for <a href="https://issues.apache.org/jira/browse/CALCITE-2900">[CALCITE-2900]
-   * RelStructuredTypeFlattener generates wrong types on nested columns</a>.
-   */
+  /** Test case for
+   * <a href="https://issues.apache.org/jira/browse/CALCITE-2900">[CALCITE-2900]
+   * RelStructuredTypeFlattener generates wrong types on nested columns</a>. */
   @Test void testNestedColumnType() {
-    final String sql =
-        "select empa.home_address.zip from sales.emp_address empa where empa.home_address.city = 'abc'";
+    final String sql = "select empa.home_address.zip\n"
+        + "from sales.emp_address empa\n"
+        + "where empa.home_address.city = 'abc'";
     sql(sql).ok();
   }
 
@@ -3220,54 +3206,34 @@ class SqlToRelConverterTest extends SqlToRelTestBase {
     sql(sql3).with(getTesterWithDynamicTable()).ok();
   }
 
-  /**
-   * Test case for Dynamic Table / Dynamic Star support
-   * <a href="https://issues.apache.org/jira/browse/CALCITE-1150">[CALCITE-1150]</a>
-   */
-  @Test void testReferDynamicStarInSelectWhereGB() throws Exception {
+  @Test void testReferDynamicStarInSelectWhereGB() {
     final String sql = "select n_regionkey, count(*) as cnt from "
         + "(select * from SALES.NATION) where n_nationkey > 5 "
         + "group by n_regionkey";
     sql(sql).with(getTesterWithDynamicTable()).ok();
   }
 
-  /**
-   * Test case for Dynamic Table / Dynamic Star support
-   * <a href="https://issues.apache.org/jira/browse/CALCITE-1150">[CALCITE-1150]</a>
-   */
-  @Test void testDynamicStarInJoinAndSubQ() throws Exception {
+  @Test void testDynamicStarInJoinAndSubQ() {
     final String sql = "select * from "
         + " (select * from SALES.NATION T1, "
         + " SALES.CUSTOMER T2 where T1.n_nationkey = T2.c_nationkey)";
     sql(sql).with(getTesterWithDynamicTable()).ok();
   }
 
-  /**
-   * Test case for Dynamic Table / Dynamic Star support
-   * <a href="https://issues.apache.org/jira/browse/CALCITE-1150">[CALCITE-1150]</a>
-   */
-  @Test void testStarJoinStaticDynTable() throws Exception {
+  @Test void testStarJoinStaticDynTable() {
     final String sql = "select * from SALES.NATION N, SALES.REGION as R "
         + "where N.n_regionkey = R.r_regionkey";
     sql(sql).with(getTesterWithDynamicTable()).ok();
   }
 
-  /**
-   * Test case for Dynamic Table / Dynamic Star support
-   * <a href="https://issues.apache.org/jira/browse/CALCITE-1150">[CALCITE-1150]</a>
-   */
-  @Test void testGrpByColFromStarInSubQuery() throws Exception {
+  @Test void testGrpByColFromStarInSubQuery() {
     final String sql = "SELECT n.n_nationkey AS col "
         + " from (SELECT * FROM SALES.NATION) as n "
         + " group by n.n_nationkey";
     sql(sql).with(getTesterWithDynamicTable()).ok();
   }
 
-  /**
-   * Test case for Dynamic Table / Dynamic Star support
-   * <a href="https://issues.apache.org/jira/browse/CALCITE-1150">[CALCITE-1150]</a>
-   */
-  @Test void testDynStarInExistSubQ() throws Exception {
+  @Test void testDynStarInExistSubQ() {
     final String sql = "select *\n"
         + "from SALES.REGION where exists (select * from SALES.NATION)";
     sql(sql).with(getTesterWithDynamicTable()).ok();
@@ -3277,7 +3243,7 @@ class SqlToRelConverterTest extends SqlToRelTestBase {
    * <a href="https://issues.apache.org/jira/browse/CALCITE-1150">[CALCITE-1150]
    * Create the a new DynamicRecordType, avoiding star expansion when working
    * with this type</a>. */
-  @Test void testSelectDynamicStarOrderBy() throws Exception {
+  @Test void testSelectDynamicStarOrderBy() {
     final String sql = "SELECT * from SALES.NATION order by n_nationkey";
     sql(sql).with(getTesterWithDynamicTable()).ok();
   }
@@ -3301,7 +3267,7 @@ class SqlToRelConverterTest extends SqlToRelTestBase {
    * <a href="https://issues.apache.org/jira/browse/CALCITE-1944">[CALCITE-1944]
    * Window function applied to sub-query with dynamic star gets wrong
    * plan</a>. */
-  @Test void testWindowOnDynamicStar() throws Exception {
+  @Test void testWindowOnDynamicStar() {
     final String sql = "SELECT SUM(n_nationkey) OVER w\n"
         + "FROM (SELECT * FROM SALES.NATION) subQry\n"
         + "WINDOW w AS (PARTITION BY REGION ORDER BY n_nationkey)";
@@ -3325,17 +3291,17 @@ class SqlToRelConverterTest extends SqlToRelTestBase {
   /** Test case for
    * <a href="https://issues.apache.org/jira/browse/CALCITE-2366">[CALCITE-2366]
    * Add support for ANY_VALUE aggregate function</a>. */
-  @Test void testAnyValueAggregateFunctionNoGroupBy() throws Exception {
+  @Test void testAnyValueAggregateFunctionNoGroupBy() {
     final String sql = "SELECT any_value(empno) as anyempno FROM emp AS e";
     sql(sql).ok();
   }
 
-  @Test void testAnyValueAggregateFunctionGroupBy() throws Exception {
+  @Test void testAnyValueAggregateFunctionGroupBy() {
     final String sql = "SELECT any_value(empno) as anyempno FROM emp AS e group by e.sal";
     sql(sql).ok();
   }
 
-  @Test void testSomeAndEveryAggregateFunctions() throws Exception {
+  @Test void testSomeAndEveryAggregateFunctions() {
     final String sql = "SELECT some(empno = 130) as someempnoexists,\n"
         + " every(empno > 0) as everyempnogtzero\n"
         + " FROM emp AS e group by e.sal";
@@ -3712,9 +3678,7 @@ class SqlToRelConverterTest extends SqlToRelTestBase {
     sql(sql).ok();
   }
 
-  /**
-   * Tests left join lateral with using
-   */
+  /** Tests LEFT JOIN LATERAL with USING. */
   @Test void testLeftJoinLateral1() {
     final String sql = "select * from (values 4) as t(c)\n"
         + " left join lateral\n"
@@ -3723,9 +3687,7 @@ class SqlToRelConverterTest extends SqlToRelTestBase {
     sql(sql).ok();
   }
 
-  /**
-   * Tests left join lateral with natural join
-   */
+  /** Tests LEFT JOIN LATERAL with NATURAL JOIN. */
   @Test void testLeftJoinLateral2() {
     final String sql = "select * from (values 4) as t(c)\n"
         + " natural left join lateral\n"
@@ -3733,9 +3695,7 @@ class SqlToRelConverterTest extends SqlToRelTestBase {
     sql(sql).ok();
   }
 
-  /**
-   * Tests left join lateral with on condition
-   */
+  /** Tests LEFT JOIN LATERAL with ON condition. */
   @Test void testLeftJoinLateral3() {
     final String sql = "select * from (values 4) as t(c)\n"
         + " left join lateral\n"
@@ -3744,9 +3704,7 @@ class SqlToRelConverterTest extends SqlToRelTestBase {
     sql(sql).ok();
   }
 
-  /**
-   * Tests left join lateral with multiple columns from outer
-   */
+  /** Tests LEFT JOIN LATERAL with multiple columns from outer. */
   @Test void testLeftJoinLateral4() {
     final String sql = "select * from (values (4,5)) as t(c,d)\n"
         + " left join lateral\n"
@@ -3755,10 +3713,8 @@ class SqlToRelConverterTest extends SqlToRelTestBase {
     sql(sql).ok();
   }
 
-  /**
-   * Tests left join lateral with correlate variable coming
-   * from one level up join scope
-   */
+  /** Tests LEFT JOIN LATERAL with correlating variable coming
+   * from one level up join scope. */
   @Test void testLeftJoinLateral5() {
     final String sql = "select * from (values 4) as t (c)\n"
         + "left join lateral\n"
@@ -3770,9 +3726,7 @@ class SqlToRelConverterTest extends SqlToRelTestBase {
     sql(sql).ok();
   }
 
-  /**
-   * Tests cross join lateral with multiple columns from outer
-   */
+  /** Tests CROSS JOIN LATERAL with multiple columns from outer. */
   @Test void testCrossJoinLateral1() {
     final String sql = "select * from (values (4,5)) as t(c,d)\n"
         + " cross join lateral\n"
@@ -3781,10 +3735,8 @@ class SqlToRelConverterTest extends SqlToRelTestBase {
     sql(sql).ok();
   }
 
-  /**
-   * Tests cross join lateral with correlate variable coming
-   * from one level up join scope
-   */
+  /** Tests CROSS JOIN LATERAL with correlating variable coming
+   * from one level up join scope. */
   @Test void testCrossJoinLateral2() {
     final String sql = "select * from (values 4) as t (c)\n"
         + "cross join lateral\n"
diff --git a/core/src/test/java/org/apache/calcite/test/SqlValidatorTest.java b/core/src/test/java/org/apache/calcite/test/SqlValidatorTest.java
index 57a4ec3..48c2958 100644
--- a/core/src/test/java/org/apache/calcite/test/SqlValidatorTest.java
+++ b/core/src/test/java/org/apache/calcite/test/SqlValidatorTest.java
@@ -95,6 +95,7 @@ import static org.junit.jupiter.api.Assumptions.assumeTrue;
 class SqlValidatorTest extends SqlValidatorTestCase {
   //~ Static fields/initializers ---------------------------------------------
 
+  // CHECKSTYLE: IGNORE 1
   /**
    * @deprecated Deprecated so that usages of this constant will show up in
    * yellow in Intellij and maybe someone will fix them.
diff --git a/core/src/test/java/org/apache/calcite/test/TypeCoercionTest.java b/core/src/test/java/org/apache/calcite/test/TypeCoercionTest.java
index ddd0a47..d70a5dc 100644
--- a/core/src/test/java/org/apache/calcite/test/TypeCoercionTest.java
+++ b/core/src/test/java/org/apache/calcite/test/TypeCoercionTest.java
@@ -650,7 +650,7 @@ class TypeCoercionTest extends SqlValidatorTestCase {
         .type("RecordType(TIMESTAMP(0) NOT NULL EXPR$0) NOT NULL");
   }
 
-  /** Test case for {@link AbstractTypeCoercion#implicitCast} */
+  /** Test for {@link AbstractTypeCoercion#implicitCast}. */
   @Test void testImplicitCasts() {
     // TINYINT
     RelDataType checkedType1 = dataTypeFactory.createSqlType(SqlTypeName.TINYINT);
diff --git a/core/src/test/java/org/apache/calcite/test/catalog/Fixture.java b/core/src/test/java/org/apache/calcite/test/catalog/Fixture.java
index 7968d7c..2cf7cd8 100644
--- a/core/src/test/java/org/apache/calcite/test/catalog/Fixture.java
+++ b/core/src/test/java/org/apache/calcite/test/catalog/Fixture.java
@@ -157,7 +157,7 @@ final class Fixture extends AbstractFixture {
 }
 
 /**
- * Just a little trick to store factory ref before field init in fixture
+ * Just a little trick to store factory ref before field init in fixture.
  */
 abstract class AbstractFixture {
   final RelDataTypeFactory typeFactory;
diff --git a/core/src/test/java/org/apache/calcite/test/catalog/MockCatalogReader.java b/core/src/test/java/org/apache/calcite/test/catalog/MockCatalogReader.java
index fcad4d1..8c764c7 100644
--- a/core/src/test/java/org/apache/calcite/test/catalog/MockCatalogReader.java
+++ b/core/src/test/java/org/apache/calcite/test/catalog/MockCatalogReader.java
@@ -261,7 +261,7 @@ public abstract class MockCatalogReader extends CalciteCatalogReader {
 
   //~ Inner Classes ----------------------------------------------------------
 
-  /** Column resolver*/
+  /** Column resolver. */
   public interface ColumnResolver {
     List<Pair<RelDataTypeField, List<String>>> resolveColumn(
         RelDataType rowType, RelDataTypeFactory typeFactory, List<String> names);
diff --git a/core/src/test/java/org/apache/calcite/test/concurrent/ConcurrentTestCommand.java b/core/src/test/java/org/apache/calcite/test/concurrent/ConcurrentTestCommand.java
index 1f1598b..d3b898d 100644
--- a/core/src/test/java/org/apache/calcite/test/concurrent/ConcurrentTestCommand.java
+++ b/core/src/test/java/org/apache/calcite/test/concurrent/ConcurrentTestCommand.java
@@ -76,7 +76,7 @@ public interface ConcurrentTestCommand {
 
   /**
    * Indicates that a command should have failed, but instead succeeded, which
-   * is a test error
+   * is a test error.
    */
   class ShouldHaveFailedException extends RuntimeException {
     private final String description;
diff --git a/core/src/test/java/org/apache/calcite/test/concurrent/ConcurrentTestCommandExecutor.java b/core/src/test/java/org/apache/calcite/test/concurrent/ConcurrentTestCommandExecutor.java
index 0b523e9..7da70aa 100644
--- a/core/src/test/java/org/apache/calcite/test/concurrent/ConcurrentTestCommandExecutor.java
+++ b/core/src/test/java/org/apache/calcite/test/concurrent/ConcurrentTestCommandExecutor.java
@@ -80,7 +80,7 @@ class ConcurrentTestCommandExecutor extends Thread {
   private final PrintStream debugPrintStream;
 
   /**
-   * Command throwing error *
+   * Command throwing error.
    */
   private ConcurrentTestCommand errorCommand;
 
diff --git a/core/src/test/java/org/apache/calcite/test/concurrent/ConcurrentTestCommandGenerator.java b/core/src/test/java/org/apache/calcite/test/concurrent/ConcurrentTestCommandGenerator.java
index da25588..01b63bf 100644
--- a/core/src/test/java/org/apache/calcite/test/concurrent/ConcurrentTestCommandGenerator.java
+++ b/core/src/test/java/org/apache/calcite/test/concurrent/ConcurrentTestCommandGenerator.java
@@ -83,7 +83,7 @@ public class ConcurrentTestCommandGenerator {
   private TreeMap<Integer, String> threadNameMap;
 
   /**
-   * Describes a thread that failed
+   * Describes a thread that failed.
    */
   static class FailedThread {
     public final String name;
@@ -466,9 +466,7 @@ public class ConcurrentTestCommandGenerator {
     return !failedThreads.isEmpty();
   }
 
-  /**
-   * @return the list of failed threads (unmodifiable)
-   */
+  /** Returns the list of failed threads (unmodifiable). */
   public List<FailedThread> getFailedThreads() {
     return ImmutableList.copyOf(failedThreads);
   }
@@ -627,9 +625,7 @@ public class ConcurrentTestCommandGenerator {
 
   //~ Inner Classes ----------------------------------------------------------
 
-  /**
-   * abstract base to handle SQLExceptions
-   */
+  /** Abstract base to handle {@link SQLException}s. */
   protected abstract static class AbstractCommand
       implements ConcurrentTestCommand {
     private boolean shouldFail = false;
diff --git a/core/src/test/java/org/apache/calcite/test/concurrent/ConcurrentTestCommandScript.java b/core/src/test/java/org/apache/calcite/test/concurrent/ConcurrentTestCommandScript.java
index c684ad7..76ff9fc 100644
--- a/core/src/test/java/org/apache/calcite/test/concurrent/ConcurrentTestCommandScript.java
+++ b/core/src/test/java/org/apache/calcite/test/concurrent/ConcurrentTestCommandScript.java
@@ -343,7 +343,7 @@ public class ConcurrentTestCommandScript
 
   /**
    * Gets ready to execute: loads script FILENAME applying external variable
-   * BINDINGS
+   * BINDINGS.
    */
   private void prepare(String filename, List<String> bindings)
       throws IOException {
@@ -367,7 +367,7 @@ public class ConcurrentTestCommandScript
   }
 
   /**
-   * Executes the script
+   * Executes the script.
    */
   public void execute() throws Exception {
     scriptStartTime = System.currentTimeMillis();
@@ -497,7 +497,7 @@ public class ConcurrentTestCommandScript
   }
 
   /**
-   * Identifies the start of a comment line; same rules as sqlline
+   * Identifies the start of a comment line; same rules as sqlline.
    */
   private boolean isComment(String line) {
     return line.startsWith("--") || line.startsWith("#");
diff --git a/core/src/test/java/org/apache/calcite/test/concurrent/ConcurrentTestPlugin.java b/core/src/test/java/org/apache/calcite/test/concurrent/ConcurrentTestPlugin.java
index 62e9bd6..83fc7d3 100644
--- a/core/src/test/java/org/apache/calcite/test/concurrent/ConcurrentTestPlugin.java
+++ b/core/src/test/java/org/apache/calcite/test/concurrent/ConcurrentTestPlugin.java
@@ -24,7 +24,7 @@ import java.util.ArrayList;
 public abstract class ConcurrentTestPlugin {
 
   /**
-   * Should containing test be disabled?
+   * Returns whether the containing test should be disabled.
    *
    * @return true if containing test should be disabled
    */
diff --git a/core/src/test/java/org/apache/calcite/test/enumerable/EnumerableBatchNestedLoopJoinTest.java b/core/src/test/java/org/apache/calcite/test/enumerable/EnumerableBatchNestedLoopJoinTest.java
index 120a43e..3c6095e 100644
--- a/core/src/test/java/org/apache/calcite/test/enumerable/EnumerableBatchNestedLoopJoinTest.java
+++ b/core/src/test/java/org/apache/calcite/test/enumerable/EnumerableBatchNestedLoopJoinTest.java
@@ -33,7 +33,7 @@ import java.util.function.Consumer;
 
 /**
  * Unit test for
- * {@link org.apache.calcite.adapter.enumerable.EnumerableBatchNestedLoopJoin}
+ * {@link org.apache.calcite.adapter.enumerable.EnumerableBatchNestedLoopJoin}.
  */
 class EnumerableBatchNestedLoopJoinTest {
 
diff --git a/core/src/test/java/org/apache/calcite/test/enumerable/EnumerableCalcTest.java b/core/src/test/java/org/apache/calcite/test/enumerable/EnumerableCalcTest.java
index de35291..ce4feab 100644
--- a/core/src/test/java/org/apache/calcite/test/enumerable/EnumerableCalcTest.java
+++ b/core/src/test/java/org/apache/calcite/test/enumerable/EnumerableCalcTest.java
@@ -25,7 +25,7 @@ import org.junit.jupiter.api.Test;
 
 /**
  * Unit test for
- * {@link org.apache.calcite.adapter.enumerable.EnumerableCalc}
+ * {@link org.apache.calcite.adapter.enumerable.EnumerableCalc}.
  */
 class EnumerableCalcTest {
 
diff --git a/core/src/test/java/org/apache/calcite/test/enumerable/EnumerableCorrelateTest.java b/core/src/test/java/org/apache/calcite/test/enumerable/EnumerableCorrelateTest.java
index 1432d0e..f326e9e 100644
--- a/core/src/test/java/org/apache/calcite/test/enumerable/EnumerableCorrelateTest.java
+++ b/core/src/test/java/org/apache/calcite/test/enumerable/EnumerableCorrelateTest.java
@@ -39,7 +39,8 @@ import java.util.function.Consumer;
 class EnumerableCorrelateTest {
   /** Test case for
    * <a href="https://issues.apache.org/jira/browse/CALCITE-2605">[CALCITE-2605]
-   * NullPointerException when left outer join implemented with EnumerableCorrelate</a> */
+   * NullPointerException when left outer join implemented with
+   * EnumerableCorrelate</a>. */
   @Test void leftOuterJoinCorrelate() {
     tester(false, new JdbcTest.HrSchema())
         .query(
@@ -82,7 +83,7 @@ class EnumerableCorrelateTest {
 
   /** Test case for
    * <a href="https://issues.apache.org/jira/browse/CALCITE-2621">[CALCITE-2621]
-   * Add rule to execute semi joins with correlation</a> */
+   * Add rule to execute semi joins with correlation</a>. */
   @Test void semiJoinCorrelate() {
     tester(false, new JdbcTest.HrSchema())
         .query(
@@ -109,8 +110,8 @@ class EnumerableCorrelateTest {
 
   /** Test case for
    * <a href="https://issues.apache.org/jira/browse/CALCITE-2930">[CALCITE-2930]
-   * FilterCorrelateRule on a Correlate with SemiJoinType SEMI (or ANTI)
-   * throws IllegalStateException</a> */
+   * FilterCorrelateRule on a Correlate with SemiJoinType SEMI (or ANTI) throws
+   * IllegalStateException</a>. */
   @Test void semiJoinCorrelateWithFilterCorrelateRule() {
     tester(false, new JdbcTest.HrSchema())
         .query(
@@ -236,7 +237,7 @@ class EnumerableCorrelateTest {
 
   /** Test case for
    * <a href="https://issues.apache.org/jira/browse/CALCITE-2920">[CALCITE-2920]
-   * RelBuilder: new method to create an antijoin</a> */
+   * RelBuilder: new method to create an antijoin</a>. */
   @Test void antiJoinCorrelateWithNullValues() {
     final Integer salesDeptNo = 10;
     tester(false, new JdbcTest.HrSchema())
diff --git a/core/src/test/java/org/apache/calcite/test/enumerable/EnumerableSortedAggregateTest.java b/core/src/test/java/org/apache/calcite/test/enumerable/EnumerableSortedAggregateTest.java
index 225aa28..f39696b 100644
--- a/core/src/test/java/org/apache/calcite/test/enumerable/EnumerableSortedAggregateTest.java
+++ b/core/src/test/java/org/apache/calcite/test/enumerable/EnumerableSortedAggregateTest.java
@@ -29,11 +29,12 @@ import org.junit.jupiter.api.Test;
 
 import java.util.function.Consumer;
 
+/** Test for
+ * {@link org.apache.calcite.adapter.enumerable.EnumerableSortedAggregate}. */
 public class EnumerableSortedAggregateTest {
   @Test void sortedAgg() {
     tester(false, new JdbcTest.HrSchema())
-        .query(
-            "select deptno, "
+        .query("select deptno, "
             + "max(salary) as max_salary, count(name) as num_employee "
             + "from emps group by deptno")
         .withHook(Hook.PLANNER, (Consumer<RelOptPlanner>) planner -> {
diff --git a/core/src/test/java/org/apache/calcite/test/enumerable/EnumerableUncollectTest.java b/core/src/test/java/org/apache/calcite/test/enumerable/EnumerableUncollectTest.java
index d898575..92508c9 100644
--- a/core/src/test/java/org/apache/calcite/test/enumerable/EnumerableUncollectTest.java
+++ b/core/src/test/java/org/apache/calcite/test/enumerable/EnumerableUncollectTest.java
@@ -23,6 +23,7 @@ import org.apache.calcite.test.CalciteAssert;
 import org.junit.jupiter.api.Disabled;
 import org.junit.jupiter.api.Test;
 
+/** Test for {@link EnumerableUncollect}. */
 class EnumerableUncollectTest {
 
   @Test void simpleUnnestArray() {
diff --git a/core/src/test/java/org/apache/calcite/test/fuzzer/RexProgramFuzzyTest.java b/core/src/test/java/org/apache/calcite/test/fuzzer/RexProgramFuzzyTest.java
index f380327..41f95f0 100644
--- a/core/src/test/java/org/apache/calcite/test/fuzzer/RexProgramFuzzyTest.java
+++ b/core/src/test/java/org/apache/calcite/test/fuzzer/RexProgramFuzzyTest.java
@@ -91,7 +91,7 @@ class RexProgramFuzzyTest extends RexProgramBuilderBase {
   private static final Strong STRONG = Strong.of(ImmutableBitSet.of());
 
   /**
-   * A bounded variation of {@link PriorityQueue}
+   * A bounded variation of {@link PriorityQueue}.
    *
    * @param <E> the type of elements held in this collection
    */
diff --git a/core/src/test/java/org/apache/calcite/tools/PlannerTest.java b/core/src/test/java/org/apache/calcite/tools/PlannerTest.java
index 5d787e4..b3ddf40 100644
--- a/core/src/test/java/org/apache/calcite/tools/PlannerTest.java
+++ b/core/src/test/java/org/apache/calcite/tools/PlannerTest.java
@@ -542,7 +542,7 @@ class PlannerTest {
   }
 
   /** Unit test that parses, validates, converts and
-   * plans for query using order by */
+   * plans for query using ORDER BY. */
   @Test void testSortPlan() throws Exception {
     RuleSet ruleSet =
         RuleSets.ofList(
@@ -647,7 +647,7 @@ class PlannerTest {
   }
 
   /** Tests that outer order by is not removed since window function
-   * might reorder the rows in-between */
+   * might reorder the rows in-between. */
   @Test void testDuplicateSortPlanWithOver() throws Exception {
     runDuplicateSortCheck("select emp_cnt, empid+deptno from ( "
         + "select empid, deptno, count(*) over (partition by deptno) emp_cnt from ( "
@@ -796,7 +796,7 @@ class PlannerTest {
   }
 
   /** Unit test that calls {@link Planner#transform} twice with
-   * rule name conflicts */
+   * rule name conflicts. */
   @Test void testPlanTransformWithRuleNameConflicts() throws Exception {
     // Create two dummy rules with identical rules.
     RelOptRule rule1 = MyProjectFilterRule.config("MYRULE").toRule();
@@ -1166,6 +1166,7 @@ class PlannerTest {
     assertThat(toString(transform), containsString(expected));
   }
 
+  /** Rule that matches a Project on a Filter. */
   public static class MyProjectFilterRule
       extends RelRule<MyProjectFilterRule.Config> {
     static Config config(String description) {
@@ -1197,6 +1198,7 @@ class PlannerTest {
     }
   }
 
+  /** Rule that matches a Filter on a Project. */
   public static class MyFilterProjectRule
       extends RelRule<MyFilterProjectRule.Config> {
     static Config config(String description) {
diff --git a/core/src/test/java/org/apache/calcite/util/BitSetsTest.java b/core/src/test/java/org/apache/calcite/util/BitSetsTest.java
index d4bad57..a372f8d 100644
--- a/core/src/test/java/org/apache/calcite/util/BitSetsTest.java
+++ b/core/src/test/java/org/apache/calcite/util/BitSetsTest.java
@@ -187,9 +187,7 @@ class BitSetsTest {
     assertThat(BitSets.previousClearBit(BitSets.of(1, 3, 4), 1), equalTo(0));
   }
 
-  /**
-   * Tests the method {@link BitSets#closure(java.util.SortedMap)}
-   */
+  /** Tests the method {@link BitSets#closure(java.util.SortedMap)}. */
   @Test void testClosure() {
     final SortedMap<Integer, BitSet> empty = new TreeMap<>();
     assertThat(BitSets.closure(empty), equalTo(empty));
diff --git a/core/src/test/java/org/apache/calcite/util/Smalls.java b/core/src/test/java/org/apache/calcite/util/Smalls.java
index 19c434d..77859e1 100644
--- a/core/src/test/java/org/apache/calcite/util/Smalls.java
+++ b/core/src/test/java/org/apache/calcite/util/Smalls.java
@@ -297,9 +297,7 @@ public class Smalls {
     };
   }
 
-  /**
-   * A function that adds a number to the first column of input cursor
-   */
+  /** Table function that adds a number to the first column of input cursor. */
   public static QueryableTable processCursor(final int offset,
       final Enumerable<Object[]> a) {
     return new AbstractQueryableTable(Object[].class) {
@@ -606,9 +604,9 @@ public class Smalls {
     public static java.sql.Time toTimeFun(Long v) {
       return v == null ? null : SqlFunctions.internalToTime(v.intValue());
     }
-    /** for Overloaded user-defined functions that have Double and BigDecimal
-     * arguments will goes wrong
-     * */
+
+    /** For overloaded user-defined functions that have {@code double} and
+     * {@code BigDecimal} arguments will go wrong. */
     public static double toDouble(BigDecimal var) {
       return var == null ? null : var.doubleValue();
     }
@@ -661,7 +659,7 @@ public class Smalls {
     }
   }
 
-  /** A generic interface for defining user defined aggregate functions
+  /** A generic interface for defining user-defined aggregate functions.
    *
    * @param <A> accumulator type
    * @param <V> value type
diff --git a/core/src/test/java/org/apache/calcite/util/SourceTest.java b/core/src/test/java/org/apache/calcite/util/SourceTest.java
index 1be1b31..6546dcb 100644
--- a/core/src/test/java/org/apache/calcite/util/SourceTest.java
+++ b/core/src/test/java/org/apache/calcite/util/SourceTest.java
@@ -63,7 +63,7 @@ class SourceTest {
   }
 
   /**
-   * Read lines from {@link CharSource}
+   * Read lines from {@link CharSource}.
    */
   @Test void charSource() throws IOException {
     Source source = Sources.fromCharSource(CharSource.wrap("a\nb"));
diff --git a/core/src/test/java/org/apache/calcite/util/TestUtilTest.java b/core/src/test/java/org/apache/calcite/util/TestUtilTest.java
index 43c96c4..2708d96 100644
--- a/core/src/test/java/org/apache/calcite/util/TestUtilTest.java
+++ b/core/src/test/java/org/apache/calcite/util/TestUtilTest.java
@@ -22,7 +22,7 @@ import static org.junit.jupiter.api.Assertions.assertEquals;
 import static org.junit.jupiter.api.Assertions.assertTrue;
 
 /**
- * Tests for TestUtil
+ * Tests for TestUtil.
  */
 class TestUtilTest {
 
diff --git a/druid/src/main/java/org/apache/calcite/adapter/druid/BinaryOperatorConversion.java b/druid/src/main/java/org/apache/calcite/adapter/druid/BinaryOperatorConversion.java
index f8260b9..64a0d1b 100644
--- a/druid/src/main/java/org/apache/calcite/adapter/druid/BinaryOperatorConversion.java
+++ b/druid/src/main/java/org/apache/calcite/adapter/druid/BinaryOperatorConversion.java
@@ -24,7 +24,8 @@ import org.apache.calcite.sql.SqlOperator;
 import java.util.List;
 
 /**
- * Binary operator conversion utility class used to convert expression like exp1 Operator exp2
+ * Binary operator conversion utility class; used to convert expressions like
+ * {@code exp1 Operator exp2}.
  */
 public class BinaryOperatorConversion implements DruidSqlOperatorConverter {
   private final SqlOperator operator;
diff --git a/druid/src/main/java/org/apache/calcite/adapter/druid/CeilOperatorConversion.java b/druid/src/main/java/org/apache/calcite/adapter/druid/CeilOperatorConversion.java
index df840e4..3207759 100644
--- a/druid/src/main/java/org/apache/calcite/adapter/druid/CeilOperatorConversion.java
+++ b/druid/src/main/java/org/apache/calcite/adapter/druid/CeilOperatorConversion.java
@@ -30,7 +30,8 @@ import java.util.TimeZone;
 import javax.annotation.Nullable;
 
 /**
- * DruidSqlOperatorConverter implementation that handles Ceil operations conversions
+ * DruidSqlOperatorConverter implementation that handles Ceil operations
+ * conversions.
  */
 public class CeilOperatorConversion implements DruidSqlOperatorConverter {
   @Override public SqlOperator calciteOperator() {
diff --git a/druid/src/main/java/org/apache/calcite/adapter/druid/ComplexMetric.java b/druid/src/main/java/org/apache/calcite/adapter/druid/ComplexMetric.java
index 0b5577b..27a58f4 100644
--- a/druid/src/main/java/org/apache/calcite/adapter/druid/ComplexMetric.java
+++ b/druid/src/main/java/org/apache/calcite/adapter/druid/ComplexMetric.java
@@ -20,18 +20,14 @@ import org.apache.calcite.rel.core.AggregateCall;
 import org.apache.calcite.sql.SqlKind;
 
 /**
- * Used to store information about available complex metrics in the Druid Adapter
- * */
+ * Stores information about available complex metrics in the Druid Adapter.
+ */
 public class ComplexMetric {
 
-  /**
-   * The underlying metric column this complex metric represents
-   * */
+  /** The underlying metric column that this complex metric represents. */
   private final String metricName;
 
-  /**
-   * The type of this metric
-   * */
+  /** The type of this metric. */
   private final DruidType type;
 
   public ComplexMetric(String metricName, DruidType type) {
diff --git a/druid/src/main/java/org/apache/calcite/adapter/druid/DruidDateTimeUtils.java b/druid/src/main/java/org/apache/calcite/adapter/druid/DruidDateTimeUtils.java
index 30a77d7..db7997f 100644
--- a/druid/src/main/java/org/apache/calcite/adapter/druid/DruidDateTimeUtils.java
+++ b/druid/src/main/java/org/apache/calcite/adapter/druid/DruidDateTimeUtils.java
@@ -330,9 +330,12 @@ public class DruidDateTimeUtils {
   }
 
   /**
+   * Converts a granularity to ISO period format.
+   *
    * @param type Druid Granularity  to translate as period of time
    *
-   * @return String representing the granularity as ISO8601 Period of Time, null for unknown case.
+   * @return String representing the granularity as ISO8601 Period of Time; null
+   * for unknown case
    */
   @Nullable
   public static String toISOPeriodFormat(Granularity.Type type) {
@@ -359,7 +362,8 @@ public class DruidDateTimeUtils {
   }
 
   /**
-   * Translates Calcite TimeUnitRange to Druid {@link Granularity}
+   * Translates a Calcite {@link TimeUnitRange} to a Druid {@link Granularity}.
+   *
    * @param timeUnit Calcite Time unit to convert
    *
    * @return Druid Granularity or null
diff --git a/druid/src/main/java/org/apache/calcite/adapter/druid/DruidExpressions.java b/druid/src/main/java/org/apache/calcite/adapter/druid/DruidExpressions.java
index ff2805f..5bd22c9 100644
--- a/druid/src/main/java/org/apache/calcite/adapter/druid/DruidExpressions.java
+++ b/druid/src/main/java/org/apache/calcite/adapter/druid/DruidExpressions.java
@@ -44,10 +44,10 @@ import javax.annotation.Nullable;
  */
 public class DruidExpressions {
 
-  /**
-   * Type mapping between Calcite SQL family types and native Druid expression types
-   */
+  /** Type mapping between Calcite SQL family types and native Druid expression
+   * types. */
   static final Map<SqlTypeName, DruidType> EXPRESSION_TYPES;
+
   /**
    * Druid expression safe chars, must be sorted.
    */
@@ -88,12 +88,14 @@ public class DruidExpressions {
 
 
   /**
-   * Translates Calcite rexNode to Druid Expression when possible
-   * @param rexNode rexNode to convert to a Druid Expression
-   * @param inputRowType input row type of the rexNode to translate
+   * Translates a Calcite {@link RexNode} to a Druid expression, if possible;
+   * returns null if not possible.
+   *
+   * @param rexNode RexNode to convert to a Druid Expression
+   * @param inputRowType Input row type of the rexNode to translate
    * @param druidRel Druid query
    *
-   * @return Druid Expression or null when can not convert the RexNode
+   * @return Druid Expression, or null when can not convert the RexNode
    */
   @Nullable
   public static String toDruidExpression(
diff --git a/druid/src/main/java/org/apache/calcite/adapter/druid/DruidJsonFilter.java b/druid/src/main/java/org/apache/calcite/adapter/druid/DruidJsonFilter.java
index 10ef5ca..669d745 100644
--- a/druid/src/main/java/org/apache/calcite/adapter/druid/DruidJsonFilter.java
+++ b/druid/src/main/java/org/apache/calcite/adapter/druid/DruidJsonFilter.java
@@ -50,16 +50,19 @@ abstract class DruidJsonFilter implements DruidJson {
       getDateFormatter(ISO_DATETIME_FRACTIONAL_SECOND_FORMAT);
 
   /**
-   * @param rexNode    rexNode to translate to Druid Json Filter
-   * @param rowType    rowType associated to rexNode
-   * @param druidQuery druid query
+   * Converts a {@link RexNode} to a Druid JSON filter.
    *
-   * @return Druid Json filter or null if it can not translate
+   * @param rexNode    RexNode to translate to Druid Json Filter
+   * @param rowType    Row type associated to rexNode
+   * @param druidQuery Druid query
+   *
+   * @return Druid JSON filter, or null if it cannot translate
    */
   @Nullable
-  private static DruidJsonFilter toEqualityKindDruidFilter(RexNode rexNode, RelDataType rowType,
-      DruidQuery druidQuery) {
-    if (rexNode.getKind() != SqlKind.EQUALS && rexNode.getKind() != SqlKind.NOT_EQUALS) {
+  private static DruidJsonFilter toEqualityKindDruidFilter(RexNode rexNode,
+      RelDataType rowType, DruidQuery druidQuery) {
+    if (rexNode.getKind() != SqlKind.EQUALS
+        && rexNode.getKind() != SqlKind.NOT_EQUALS) {
       throw new AssertionError(
           DruidQuery.format("Expecting EQUALS or NOT_EQUALS but got [%s]", rexNode.getKind()));
     }
@@ -88,7 +91,7 @@ abstract class DruidJsonFilter implements DruidJson {
     }
     final String literalValue = toDruidLiteral(rexLiteral, rowType, druidQuery);
     if (literalValue == null) {
-      // can not translate literal better bail out
+      // cannot translate literal; better bail out
       return null;
     }
     final boolean isNumeric = refNode.getType().getFamily() == SqlTypeFamily.NUMERIC
@@ -118,11 +121,14 @@ abstract class DruidJsonFilter implements DruidJson {
 
 
   /**
-   * @param rexNode    rexNode to translate
-   * @param rowType    row type associated to Filter
-   * @param druidQuery druid query
+   * Converts a {@link RexNode} to a Druid JSON bound filter.
+   *
+   * @param rexNode    RexNode to translate
+   * @param rowType    Row type associated to Filter
+   * @param druidQuery Druid query
    *
-   * @return valid Druid Json Bound Filter or null if it can not translate the rexNode.
+   * @return valid Druid JSON Bound Filter, or null if it cannot translate the
+   * RexNode
    */
   @Nullable
   private static DruidJsonFilter toBoundDruidFilter(RexNode rexNode, RelDataType rowType,
@@ -150,18 +156,20 @@ abstract class DruidJsonFilter implements DruidJson {
     }
 
     if (RexLiteral.isNullLiteral(rexLiteral)) {
-      // we are not handling is NULL filter here thus we bail out if Literal is null
+      // we are not handling is NULL filter here; thus we bail out if Literal is
+      // null
       return null;
     }
-    final String literalValue = DruidJsonFilter.toDruidLiteral(rexLiteral, rowType, druidQuery);
+    final String literalValue =
+        DruidJsonFilter.toDruidLiteral(rexLiteral, rowType, druidQuery);
     if (literalValue == null) {
-      // can not translate literal better bail out
+      // cannot translate literal; better bail out
       return null;
     }
     final boolean isNumeric = refNode.getType().getFamily() == SqlTypeFamily.NUMERIC
         || rexLiteral.getType().getFamily() == SqlTypeFamily.NUMERIC;
-    final Pair<String, ExtractionFunction> druidColumn = DruidQuery.toDruidColumn(refNode, rowType,
-        druidQuery);
+    final Pair<String, ExtractionFunction> druidColumn =
+        DruidQuery.toDruidColumn(refNode, rowType, druidQuery);
     final String columnName = druidColumn.left;
     final ExtractionFunction extractionFunction = druidColumn.right;
     if (columnName == null) {
@@ -199,11 +207,14 @@ abstract class DruidJsonFilter implements DruidJson {
   }
 
   /**
-   * @param rexNode    rexNode to translate to Druid literal equivalante
-   * @param rowType    rowType associated to rexNode
-   * @param druidQuery druid Query
+   * Converts a {@link RexNode} to a Druid literal.
    *
-   * @return non null string or null if it can not translate to valid Druid equivalent
+   * @param rexNode    RexNode to translate to Druid literal equivalant
+   * @param rowType    Row type associated to rexNode
+   * @param druidQuery Druid query
+   *
+   * @return non null string, or null if it cannot translate to valid Druid
+   * equivalent
    */
   @Nullable
   private static String toDruidLiteral(RexNode rexNode, RelDataType rowType,
@@ -356,11 +367,14 @@ abstract class DruidJsonFilter implements DruidJson {
   }
 
   /**
-   * @param rexNode    rexNode to translate to Druid Filter
-   * @param rowType    rowType of filter input
+   * Converts a {@link RexNode} to a Druid filter.
+   *
+   * @param rexNode    RexNode to translate to Druid Filter
+   * @param rowType    Row type of filter input
    * @param druidQuery Druid query
    *
-   * @return Druid Json Filters or null when can not translate to valid Druid Filters.
+   * @return Druid Json filters, or null when cannot translate to valid Druid
+   * filters
    */
   @Nullable
   static DruidJsonFilter toDruidFilters(final RexNode rexNode, RelDataType rowType,
@@ -411,9 +425,7 @@ abstract class DruidJsonFilter implements DruidJson {
     return expression == null ? null : new JsonExpressionFilter(expression);
   }
 
-  /**
-   * Supported filter types
-   */
+  /** Supported filter types. */
   protected enum Type {
     AND,
     OR,
@@ -613,9 +625,7 @@ abstract class DruidJsonFilter implements DruidJson {
     return new JsonSelector(column, value, extractionFunction);
   }
 
-  /**
-   * Druid Having Filter spec
-   */
+  /** Druid Having Filter spec. */
   protected static class JsonDimHavingFilter implements DruidJson {
 
     private final DruidJsonFilter filter;
diff --git a/druid/src/main/java/org/apache/calcite/adapter/druid/DruidQuery.java b/druid/src/main/java/org/apache/calcite/adapter/druid/DruidQuery.java
index 2fa2e49..509d9e7 100644
--- a/druid/src/main/java/org/apache/calcite/adapter/druid/DruidQuery.java
+++ b/druid/src/main/java/org/apache/calcite/adapter/druid/DruidQuery.java
@@ -186,7 +186,7 @@ public class DruidQuery extends AbstractRelNode implements BindableRel {
     this.intervals = ImmutableList.copyOf(intervals);
     this.rels = ImmutableList.copyOf(rels);
     this.converterOperatorMap = Objects.requireNonNull(converterOperatorMap,
-        "Operator map can not be null");
+        "Operator map cannot be null");
     assert isValid(Litmus.THROW, null);
   }
 
@@ -250,12 +250,15 @@ public class DruidQuery extends AbstractRelNode implements BindableRel {
   }
 
   /**
+   * Converts a {@link RexNode} to a Druid column.
+   *
    * @param rexNode    leaf Input Ref to Druid Column
    * @param rowType    row type
-   * @param druidQuery druid query
+   * @param druidQuery Druid query
    *
-   * @return {@link Pair} of Column name and Extraction Function on the top of the input ref or
-   * {@link Pair of(null, null)} when can not translate to valid Druid column
+   * @return {@link Pair} of Column name and Extraction Function on the top of
+   * the input ref, or {@code Pair.of(null, null)} when cannot translate to a
+   * valid Druid column
    */
   protected static Pair<String, ExtractionFunction> toDruidColumn(RexNode rexNode,
       RelDataType rowType, DruidQuery druidQuery) {
@@ -350,9 +353,12 @@ public class DruidQuery extends AbstractRelNode implements BindableRel {
   }
 
   /**
-   * @param rexNode rexNode
+   * Returns whether a {@link RexNode} is a valid Druid cast operation.
    *
-   * @return true if the operand is an inputRef and it is a valid Druid Cast operation
+   * @param rexNode RexNode
+   *
+   * @return whether the operand is an inputRef and it is a valid Druid Cast
+   * operation
    */
   private static boolean isValidLeafCast(RexNode rexNode) {
     assert rexNode.isA(SqlKind.CAST);
@@ -385,11 +391,11 @@ public class DruidQuery extends AbstractRelNode implements BindableRel {
   }
 
   /**
-   * @param rexNode Druid input ref node
-   * @param rowType rowType
-   * @param query Druid Query
+   * Returns Druid column name or null when it is not possible to translate.
    *
-   * @return Druid column name or null when not possible to translate.
+   * @param rexNode Druid input ref node
+   * @param rowType Row type
+   * @param query Druid query
    */
   @Nullable
   protected static String extractColumnName(RexNode rexNode, RelDataType rowType,
@@ -720,7 +726,7 @@ public class DruidQuery extends AbstractRelNode implements BindableRel {
    * @param filterRel input filter rel
    * @param druidQuery Druid query
    *
-   * @return DruidJson Filter or null if can not translate one of filters
+   * @return DruidJson Filter or null if cannot translate one of filters
    */
   @Nullable
   private static DruidJsonFilter computeFilter(@Nullable Filter filterRel,
@@ -737,15 +743,18 @@ public class DruidQuery extends AbstractRelNode implements BindableRel {
   }
 
   /**
-   * Translates list of projects to Druid Column names and Virtual Columns if any
-   * We can not use {@link Pair#zip(Object[], Object[])}, since size can be different
+   * Translates a list of projects to Druid Column names and Virtual Columns if
+   * any.
+   *
+   * <p>We cannot use {@link Pair#zip(Object[], Object[])}, since size may be
+   * different.
    *
-   * @param projectRel       Project Rel
+   * @param projectRel Project
    *
    * @param druidQuery Druid query
    *
-   * @return Pair of list of Druid Columns and Expression Virtual Columns or null when can not
-   * translate one of the projects.
+   * @return Pair of list of Druid Columns and Expression Virtual Columns, or
+   * null when cannot translate one of the projects
    */
   @Nullable
   protected static Pair<List<String>, List<VirtualColumn>> computeProjectAsScan(
@@ -806,7 +815,7 @@ public class DruidQuery extends AbstractRelNode implements BindableRel {
    * @param projectNode Project under the Aggregates if any
    * @param groupSet Ids of grouping keys as they are listed in {@code projects} list
    * @param inputRowType Input row type under the project
-   * @param druidQuery Druid Query
+   * @param druidQuery Druid query
    *
    * @return A list of {@link DimensionSpec} containing the group by dimensions,
    * and a list of {@link VirtualColumn} containing Druid virtual column
@@ -894,7 +903,7 @@ public class DruidQuery extends AbstractRelNode implements BindableRel {
    * @param project Input project under the aggregate calls,
    *               or null if we have {@link TableScan} immediately under the
    *               {@link Aggregate}
-   * @param druidQuery Druid Query Rel
+   * @param druidQuery Druid query
    *
    * @return List of valid Druid {@link JsonAggregation}s, or null if any of the
    * aggregates is not supported
@@ -1110,22 +1119,23 @@ public class DruidQuery extends AbstractRelNode implements BindableRel {
         virtualColumnList, aggregations, postAggs, limit, havingJsonFilter);
 
     if (groupByQuery == null) {
-      throw new IllegalStateException("Can not plan Druid Query");
+      throw new IllegalStateException("Cannot plan Druid Query");
     }
     return new QuerySpec(QueryType.GROUP_BY, groupByQuery, queryOutputFieldNames);
   }
 
   /**
+   * Converts a sort specification to a {@link JsonLimit} (never null).
+   *
    * @param fetch limit to fetch
    * @param collationIndexes index of fields as listed in query row output
    * @param collationDirections direction of sort
    * @param numericCollationIndexes flag of to determine sort comparator
    * @param queryOutputFieldNames query output fields
-   *
-   * @return always an non null Json Limit object
    */
-  private JsonLimit computeSort(@Nullable Integer fetch, List<Integer> collationIndexes,
-      List<Direction> collationDirections, ImmutableBitSet numericCollationIndexes,
+  private @Nonnull JsonLimit computeSort(@Nullable Integer fetch,
+      List<Integer> collationIndexes, List<Direction> collationDirections,
+      ImmutableBitSet numericCollationIndexes,
       List<String> queryOutputFieldNames) {
     final List<JsonCollation> collations;
     if (collationIndexes != null) {
@@ -1180,7 +1190,8 @@ public class DruidQuery extends AbstractRelNode implements BindableRel {
     if (groupByKeyDims.size() == 1) {
       DimensionSpec dimensionSpec = Iterables.getOnlyElement(groupByKeyDims);
       Granularity granularity = ExtractionDimensionSpec.toQueryGranularity(dimensionSpec);
-      // case we have project expression on the top of the time extract then can not use timeseries
+      // case we have project expression on the top of the time extract then
+      // cannot use timeseries
       boolean hasExpressionOnTopOfTimeExtract = false;
       for (JsonExpressionPostAgg postAgg : postAggregations) {
         if (postAgg instanceof JsonExpressionPostAgg) {
@@ -1191,7 +1202,7 @@ public class DruidQuery extends AbstractRelNode implements BindableRel {
       }
       timeseriesGranularity = hasExpressionOnTopOfTimeExtract ? null : granularity;
       if (timeseriesGranularity == null) {
-        // can not extract granularity bailout
+        // cannot extract granularity bailout
         return null;
       }
     } else {
@@ -1309,9 +1320,7 @@ public class DruidQuery extends AbstractRelNode implements BindableRel {
     return sw.toString();
   }
 
-  /**
-   * Druid Scan Query Body
-   */
+  /** Druid Scan Query body. */
   private static class ScanQuery {
 
     private String dataSource;
@@ -1455,7 +1464,7 @@ public class DruidQuery extends AbstractRelNode implements BindableRel {
       DruidJsonFilter druidFilter = DruidJsonFilter
           .toDruidFilters(filterNode, druidQuery.table.getRowType(), druidQuery);
       if (druidFilter == null) {
-        // can not translate filter
+        // cannot translate filter
         return null;
       }
       return new JsonFilteredAggregation(druidFilter, aggregation);
@@ -1739,7 +1748,7 @@ public class DruidQuery extends AbstractRelNode implements BindableRel {
     }
   }
 
-  /** Aggregation element that contains a filter */
+  /** Aggregation element that contains a filter. */
   private static class JsonFilteredAggregation extends JsonAggregation {
     final DruidJsonFilter filter;
     final JsonAggregation aggregation;
@@ -1761,7 +1770,7 @@ public class DruidQuery extends AbstractRelNode implements BindableRel {
     }
   }
 
-  /** Post-Aggregator Post aggregator abstract writer */
+  /** Post-aggregator abstract writer. */
   protected abstract static class JsonPostAggregation implements DruidJson {
     final String type;
     String name;
@@ -1784,9 +1793,7 @@ public class DruidQuery extends AbstractRelNode implements BindableRel {
 
   }
 
-  /**
-   * @return index of the timestamp ref or -1 if not present
-   */
+  /** Returns the index of the timestamp ref, or -1 if not present. */
   protected int getTimestampFieldIndex() {
     return Iterables.indexOf(this.getRowType().getFieldList(),
         input -> druidTable.timestampFieldName.equals(input.getName()));
diff --git a/druid/src/main/java/org/apache/calcite/adapter/druid/DruidRules.java b/druid/src/main/java/org/apache/calcite/adapter/druid/DruidRules.java
index 1b8fffe..bb7a271 100644
--- a/druid/src/main/java/org/apache/calcite/adapter/druid/DruidRules.java
+++ b/druid/src/main/java/org/apache/calcite/adapter/druid/DruidRules.java
@@ -313,9 +313,7 @@ public class DruidRules {
     }
   }
 
-  /**
-   * Rule to Push a Having {@link Filter} into a {@link DruidQuery}
-   */
+  /** Rule to Push a Having {@link Filter} into a {@link DruidQuery}. */
   public static class DruidHavingFilterRule
       extends RelRule<DruidHavingFilterRule.Config> {
 
@@ -596,10 +594,8 @@ public class DruidRules {
       call.transformTo(query2);
     }
 
-    /**
-     * Returns an array of unique filter references from
-     * the given list of {@link org.apache.calcite.rel.core.AggregateCall}
-     * */
+    /** Returns an array of unique filter references from the given list of
+     * {@link org.apache.calcite.rel.core.AggregateCall}s. */
     private Set<Integer> getUniqueFilterRefs(List<AggregateCall> calls) {
       Set<Integer> refs = new HashSet<>();
       for (AggregateCall call : calls) {
diff --git a/druid/src/main/java/org/apache/calcite/adapter/druid/DruidSqlCastConverter.java b/druid/src/main/java/org/apache/calcite/adapter/druid/DruidSqlCastConverter.java
index fda03e9..b38ead5 100644
--- a/druid/src/main/java/org/apache/calcite/adapter/druid/DruidSqlCastConverter.java
+++ b/druid/src/main/java/org/apache/calcite/adapter/druid/DruidSqlCastConverter.java
@@ -31,7 +31,8 @@ import org.joda.time.Period;
 import java.util.TimeZone;
 
 /**
- * Druid cast converter operator used to translates calcite casts to Druid expression cast
+ * Druid cast converter operator; used to translates Calcite casts to Druid
+ * expression casts.
  */
 public class DruidSqlCastConverter implements DruidSqlOperatorConverter {
 
diff --git a/druid/src/main/java/org/apache/calcite/adapter/druid/DruidSqlOperatorConverter.java b/druid/src/main/java/org/apache/calcite/adapter/druid/DruidSqlOperatorConverter.java
index 71396e5..18c9048 100644
--- a/druid/src/main/java/org/apache/calcite/adapter/druid/DruidSqlOperatorConverter.java
+++ b/druid/src/main/java/org/apache/calcite/adapter/druid/DruidSqlOperatorConverter.java
@@ -23,7 +23,8 @@ import org.apache.calcite.sql.SqlOperator;
 import javax.annotation.Nullable;
 
 /**
- * Defines how to convert RexNode with a given calcite SQL operator to Druid expressions
+ * Defines how to convert a {@link RexNode} with a given Calcite SQL operator to
+ * a Druid expression.
  */
 public interface DruidSqlOperatorConverter {
 
diff --git a/druid/src/main/java/org/apache/calcite/adapter/druid/DruidTable.java b/druid/src/main/java/org/apache/calcite/adapter/druid/DruidTable.java
index 9c9f43c..307d154 100644
--- a/druid/src/main/java/org/apache/calcite/adapter/druid/DruidTable.java
+++ b/druid/src/main/java/org/apache/calcite/adapter/druid/DruidTable.java
@@ -213,20 +213,17 @@ public class DruidTable extends AbstractTable implements TranslatableTable {
             || kind == SqlKind.TIMES;
   }
 
-  /**
-   * Returns the list of {@link ComplexMetric} that match the given <code>alias</code> if it exists,
-   * otherwise returns an empty list, never <code>null</code>
-   * */
+  /** Returns the list of {@link ComplexMetric} that match the given
+   * <code>alias</code> if it exists, otherwise returns an empty list, never
+   * <code>null</code>. */
   public List<ComplexMetric> getComplexMetricsFrom(String alias) {
     return complexMetrics.containsKey(alias)
             ? complexMetrics.get(alias)
             : new ArrayList<>();
   }
 
-  /**
-   * Returns true if and only if the given <code>alias</code> is a reference to a registered
-   * {@link ComplexMetric}
-   * */
+  /** Returns whether the given <code>alias</code> is a reference to a
+   * registered {@link ComplexMetric}. */
   public boolean isComplexMetric(String alias) {
     return complexMetrics.get(alias) != null;
   }
diff --git a/druid/src/main/java/org/apache/calcite/adapter/druid/DruidType.java b/druid/src/main/java/org/apache/calcite/adapter/druid/DruidType.java
index d866381..2e178c9 100644
--- a/druid/src/main/java/org/apache/calcite/adapter/druid/DruidType.java
+++ b/druid/src/main/java/org/apache/calcite/adapter/druid/DruidType.java
@@ -35,16 +35,13 @@ public enum DruidType {
     this.sqlType = sqlType;
   }
 
-  /**
-   * Returns true if and only if this enum should be used inside of a {@link ComplexMetric}
-   * */
+  /** Returns whether this type should be used inside a
+   * {@link ComplexMetric}. */
   public boolean isComplex() {
     return this == THETA_SKETCH || this == HYPER_UNIQUE || this == COMPLEX;
   }
 
-  /**
-   * Returns a DruidType matching the given String type from a Druid metric
-   * */
+  /** Returns a DruidType matching the given String type from a Druid metric. */
   protected static DruidType getTypeFromMetric(String type) {
     assert type != null;
     if (type.equals("hyperUnique")) {
@@ -61,9 +58,7 @@ public enum DruidType {
     throw new AssertionError("Unknown type: " + type);
   }
 
-  /**
-   * Returns a DruidType matching the String from a meta data query
-   * */
+  /** Returns a DruidType matching the String from a meta data query. */
   protected static DruidType getTypeFromMetaData(String type) {
     assert type != null;
     switch (type) {
diff --git a/druid/src/main/java/org/apache/calcite/adapter/druid/ExtractOperatorConversion.java b/druid/src/main/java/org/apache/calcite/adapter/druid/ExtractOperatorConversion.java
index d6b7074..c0750a2 100644
--- a/druid/src/main/java/org/apache/calcite/adapter/druid/ExtractOperatorConversion.java
+++ b/druid/src/main/java/org/apache/calcite/adapter/druid/ExtractOperatorConversion.java
@@ -32,11 +32,13 @@ import java.util.Map;
 import java.util.TimeZone;
 
 /**
- * Time extract operator conversion for expressions like EXTRACT(timeUnit FROM arg)
- * Unit can be SECOND, MINUTE, HOUR, DAY (day of month),
- * DOW (day of week), DOY (day of year), WEEK (week of week year),
- * MONTH (1 through 12), QUARTER (1 through 4), or YEAR
- **/
+ * Time extract operator conversion for expressions like
+ * {@code EXTRACT(timeUnit FROM arg)}.
+ *
+ * <p>Unit can be SECOND, MINUTE, HOUR, DAY (day of month), DOW (day of week),
+ * DOY (day of year), WEEK (week of week year), MONTH (1 through 12), QUARTER (1
+ * through 4), or YEAR.
+ */
 public class ExtractOperatorConversion implements DruidSqlOperatorConverter {
   private static final Map<TimeUnitRange, String> EXTRACT_UNIT_MAP =
       ImmutableMap.<TimeUnitRange, String>builder()
diff --git a/druid/src/main/java/org/apache/calcite/adapter/druid/ExtractionDimensionSpec.java b/druid/src/main/java/org/apache/calcite/adapter/druid/ExtractionDimensionSpec.java
index 778c12d..e5f2e51 100644
--- a/druid/src/main/java/org/apache/calcite/adapter/druid/ExtractionDimensionSpec.java
+++ b/druid/src/main/java/org/apache/calcite/adapter/druid/ExtractionDimensionSpec.java
@@ -76,10 +76,10 @@ public class ExtractionDimensionSpec implements DimensionSpec {
     generator.writeEndObject();
   }
 
-  /**
-   * @param dimensionSpec Druid Dimesion spec object
+  /** Returns a valid {@link Granularity} of floor extract, or null when not
+   * possible.
    *
-   * @return valid {@link Granularity} of floor extract or null when not possible.
... 2530 lines suppressed ...


[calcite] 02/09: [CALCITE-4079] Dialect constants in SqlDialect can cause class initialization deadlock

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

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

commit 8151c6f0e6b49660f32232d860c37a15c2d14f79
Author: Julian Hyde <jh...@apache.org>
AuthorDate: Thu Jul 9 14:01:04 2020 -0700

    [CALCITE-4079] Dialect constants in SqlDialect can cause class initialization deadlock
    
    Remove constants SqlDialect.DUMMY and SqlDialect.CALCITE. We gave fair
    warning: before release 1.24, they were marked 'deprecated to be removed
    before 1.25'.
    
    Remove RelNode methods deprecated in [CALCITE-3786],
    class SqlExecutableStatement, and a few other methods:
    RelOptUtil.appendRelDescription, RexShuttle.apply,
    RelBuilder.shouldMergeProject.
    
    Add Sort.getSortExps() to replace removed RelNode.getChildExps().
    
    Close apache/calcite#2040
---
 .../apache/calcite/jdbc/ContextSqlValidator.java   |   5 +-
 .../java/org/apache/calcite/plan/RelOptUtil.java   |  14 ---
 .../org/apache/calcite/rel/AbstractRelNode.java    |  52 ----------
 .../main/java/org/apache/calcite/rel/RelNode.java  | 110 +--------------------
 .../java/org/apache/calcite/rel/core/Filter.java   |   6 --
 .../java/org/apache/calcite/rel/core/Join.java     |  19 +---
 .../java/org/apache/calcite/rel/core/Project.java  |   4 -
 .../java/org/apache/calcite/rel/core/Snapshot.java |   6 --
 .../java/org/apache/calcite/rel/core/Sort.java     |  34 +++----
 .../apache/calcite/rel/core/TableFunctionScan.java |   4 -
 .../org/apache/calcite/rel/core/TableScan.java     |   6 --
 .../org/apache/calcite/rel/rules/MultiJoin.java    |   4 -
 .../java/org/apache/calcite/rex/RexShuttle.java    |   8 --
 .../java/org/apache/calcite/sql/SqlDialect.java    |  12 ---
 .../apache/calcite/sql/SqlExecutableStatement.java |  32 ------
 .../java/org/apache/calcite/tools/RelBuilder.java  |  10 --
 .../org/apache/calcite/test/RelBuilderTest.java    |   2 +
 17 files changed, 20 insertions(+), 308 deletions(-)

diff --git a/core/src/main/java/org/apache/calcite/jdbc/ContextSqlValidator.java b/core/src/main/java/org/apache/calcite/jdbc/ContextSqlValidator.java
index 37e9a09..352ce01 100644
--- a/core/src/main/java/org/apache/calcite/jdbc/ContextSqlValidator.java
+++ b/core/src/main/java/org/apache/calcite/jdbc/ContextSqlValidator.java
@@ -31,8 +31,9 @@ import com.google.common.collect.ImmutableList;
  * Usually we deduce query sql node data type(i.e. the {@code SqlSelect})
  * during the validation phrase. DDL nodes don't have validation,
  * they can be executed directly through
- * {@link org.apache.calcite.sql.SqlExecutableStatement#execute(CalcitePrepare.Context)}.
- * During the execution, {@link org.apache.calcite.sql.SqlDataTypeSpec} uses
+ * {@link org.apache.calcite.server.DdlExecutor}.
+ *
+ * <p>During the execution, {@link org.apache.calcite.sql.SqlDataTypeSpec} uses
  * this validator to derive its type.
  */
 public class ContextSqlValidator extends SqlValidatorImpl {
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 f7c1ca7..13958ef 100644
--- a/core/src/main/java/org/apache/calcite/plan/RelOptUtil.java
+++ b/core/src/main/java/org/apache/calcite/plan/RelOptUtil.java
@@ -2061,14 +2061,6 @@ public abstract class RelOptUtil {
     planner.addRule(CoreRules.FILTER_REDUCE_EXPRESSIONS);
   }
 
-  @Deprecated // to be removed before 1.25
-  public static StringBuilder appendRelDescription(
-      StringBuilder sb, RelNode rel) {
-    sb.append("rel#").append(rel.getId())
-        .append(':').append(rel.getDigest());
-    return sb;
-  }
-
   /**
    * Dumps a plan as a string.
    *
@@ -4231,11 +4223,6 @@ public abstract class RelOptUtil {
    * Visitor which builds a bitmap of the inputs used by an expression.
    */
   public static class InputFinder extends RexVisitorImpl<Void> {
-    /** @deprecated Being replaced by private field {@link #bitBuilder}.
-     * Use {@link #build}. */
-    @Deprecated // to be removed before 1.25
-    public final ImmutableBitSet.Builder inputBitSet;
-
     private final ImmutableBitSet.Builder bitBuilder;
     private final Set<RelDataTypeField> extraFields;
 
@@ -4243,7 +4230,6 @@ public abstract class RelOptUtil {
         ImmutableBitSet.Builder bitBuilder) {
       super(true);
       this.bitBuilder = bitBuilder;
-      this.inputBitSet = bitBuilder; // deprecated field mirrors private field
       this.extraFields = extraFields;
     }
 
diff --git a/core/src/main/java/org/apache/calcite/rel/AbstractRelNode.java b/core/src/main/java/org/apache/calcite/rel/AbstractRelNode.java
index f6c2987..9f104a1 100644
--- a/core/src/main/java/org/apache/calcite/rel/AbstractRelNode.java
+++ b/core/src/main/java/org/apache/calcite/rel/AbstractRelNode.java
@@ -22,7 +22,6 @@ import org.apache.calcite.plan.RelDigest;
 import org.apache.calcite.plan.RelOptCluster;
 import org.apache.calcite.plan.RelOptCost;
 import org.apache.calcite.plan.RelOptPlanner;
-import org.apache.calcite.plan.RelOptQuery;
 import org.apache.calcite.plan.RelOptTable;
 import org.apache.calcite.plan.RelOptUtil;
 import org.apache.calcite.plan.RelTraitSet;
@@ -33,16 +32,13 @@ import org.apache.calcite.rel.metadata.Metadata;
 import org.apache.calcite.rel.metadata.MetadataFactory;
 import org.apache.calcite.rel.metadata.RelMetadataQuery;
 import org.apache.calcite.rel.type.RelDataType;
-import org.apache.calcite.rex.RexNode;
 import org.apache.calcite.rex.RexShuttle;
 import org.apache.calcite.sql.SqlExplainLevel;
-import org.apache.calcite.util.ImmutableBitSet;
 import org.apache.calcite.util.Litmus;
 import org.apache.calcite.util.Pair;
 import org.apache.calcite.util.Util;
 import org.apache.calcite.util.trace.CalciteTrace;
 
-import com.google.common.collect.ImmutableList;
 import com.google.common.collect.ImmutableSet;
 
 import org.apiguardian.api.API;
@@ -126,11 +122,6 @@ public abstract class AbstractRelNode implements RelNode {
     return collection.get(0);
   }
 
-  @SuppressWarnings("deprecation")
-  public List<RexNode> getChildExps() {
-    return ImmutableList.of();
-  }
-
   public final RelOptCluster getCluster() {
     return cluster;
   }
@@ -147,18 +138,6 @@ public abstract class AbstractRelNode implements RelNode {
     return null;
   }
 
-  @Deprecated // to be removed before 1.25
-  @Override public boolean isDistinct() {
-    final RelMetadataQuery mq = cluster.getMetadataQuery();
-    return Boolean.TRUE.equals(mq.areRowsUnique(this));
-  }
-
-  @Deprecated // to be removed before 1.25
-  @Override public boolean isKey(ImmutableBitSet columns) {
-    final RelMetadataQuery mq = cluster.getMetadataQuery();
-    return Boolean.TRUE.equals(mq.areColumnsUnique(this, columns));
-  }
-
   public int getId() {
     return id;
   }
@@ -168,11 +147,6 @@ public abstract class AbstractRelNode implements RelNode {
     return inputs.get(i);
   }
 
-  @Deprecated // to be removed before 1.25
-  @Override public final RelOptQuery getQuery() {
-    return getCluster().getQuery();
-  }
-
   public void register(RelOptPlanner planner) {
     Util.discard(planner);
   }
@@ -193,17 +167,6 @@ public abstract class AbstractRelNode implements RelNode {
     return litmus.succeed();
   }
 
-  @Deprecated // to be removed before 1.25
-  @Override public boolean isValid(boolean fail) {
-    return isValid(Litmus.THROW, null);
-  }
-
-  /** @deprecated Use {@link RelMetadataQuery#collations(RelNode)} */
-  @Deprecated // to be removed before 2.0
-  public List<RelCollation> getCollationList() {
-    return ImmutableList.of();
-  }
-
   public final RelDataType getRowType() {
     if (rowType == null) {
       rowType = deriveRowType();
@@ -226,20 +189,10 @@ public abstract class AbstractRelNode implements RelNode {
     return Collections.emptyList();
   }
 
-  @Deprecated // to be removed before 1.25
-  @Override public final double getRows() {
-    return estimateRowCount(cluster.getMetadataQuery());
-  }
-
   public double estimateRowCount(RelMetadataQuery mq) {
     return 1.0;
   }
 
-  @Deprecated // to be removed before 1.25
-  @Override public final Set<String> getVariablesStopped() {
-    return CorrelationId.names(getVariablesSet());
-  }
-
   public Set<CorrelationId> getVariablesSet() {
     return ImmutableSet.of();
   }
@@ -272,11 +225,6 @@ public abstract class AbstractRelNode implements RelNode {
     return this;
   }
 
-  @Deprecated // to be removed before 1.25
-  @Override public final RelOptCost computeSelfCost(RelOptPlanner planner) {
-    return computeSelfCost(planner, cluster.getMetadataQuery());
-  }
-
   public RelOptCost computeSelfCost(RelOptPlanner planner,
       RelMetadataQuery mq) {
     // by default, assume cost is proportional to number of rows
diff --git a/core/src/main/java/org/apache/calcite/rel/RelNode.java b/core/src/main/java/org/apache/calcite/rel/RelNode.java
index 69db149..3a25bde 100644
--- a/core/src/main/java/org/apache/calcite/rel/RelNode.java
+++ b/core/src/main/java/org/apache/calcite/rel/RelNode.java
@@ -21,7 +21,6 @@ import org.apache.calcite.plan.RelDigest;
 import org.apache.calcite.plan.RelOptCost;
 import org.apache.calcite.plan.RelOptNode;
 import org.apache.calcite.plan.RelOptPlanner;
-import org.apache.calcite.plan.RelOptQuery;
 import org.apache.calcite.plan.RelOptTable;
 import org.apache.calcite.plan.RelTraitSet;
 import org.apache.calcite.rel.core.CorrelationId;
@@ -30,7 +29,6 @@ import org.apache.calcite.rel.metadata.RelMetadataQuery;
 import org.apache.calcite.rel.type.RelDataType;
 import org.apache.calcite.rex.RexNode;
 import org.apache.calcite.rex.RexShuttle;
-import org.apache.calcite.util.ImmutableBitSet;
 import org.apache.calcite.util.Litmus;
 
 import org.apiguardian.api.API;
@@ -83,23 +81,6 @@ public interface RelNode extends RelOptNode, Cloneable {
   //~ Methods ----------------------------------------------------------------
 
   /**
-   * Returns a list of this relational expression's child expressions.
-   * (These are scalar expressions, and so do not include the relational
-   * inputs that are returned by {@link #getInputs}.
-   *
-   * <p>The caller should treat the list as unmodifiable; typical
-   * implementations will return an immutable list. If there are no
-   * child expressions, returns an empty list, not <code>null</code>.
-   *
-   * @deprecated use {@link }#accept(org.apache.calcite.rex.RexShuttle)}
-   *
-   * @return List of this relational expression's child expressions
-   * @see #accept(org.apache.calcite.rex.RexShuttle)
-   */
-  @Deprecated // to be removed before 1.25
-  List<RexNode> getChildExps();
-
-  /**
    * Return the CallingConvention trait from this RelNode's
    * {@link #getTraitSet() trait set}.
    *
@@ -117,17 +98,6 @@ public interface RelNode extends RelOptNode, Cloneable {
   String getCorrelVariable();
 
   /**
-   * Returns whether the same value will not come out twice. Default value is
-   * <code>false</code>, derived classes should override.
-   *
-   * @return Whether the same value will not come out twice
-   *
-   * @deprecated Use {@link RelMetadataQuery#areRowsUnique(RelNode)}
-   */
-  @Deprecated // to be removed before 1.25
-  boolean isDistinct();
-
-  /**
    * Returns the <code>i</code><sup>th</sup> input relational expression.
    *
    * @param i Ordinal of input
@@ -136,16 +106,6 @@ public interface RelNode extends RelOptNode, Cloneable {
   RelNode getInput(int i);
 
   /**
-   * Returns the sub-query this relational expression belongs to.
-   *
-   * @deprecated With no replacement
-   *
-   * @return Sub-query
-   */
-  @Deprecated // to be removed before 1.25
-  RelOptQuery getQuery();
-
-  /**
    * Returns the type of the rows returned by this relational expression.
    */
   RelDataType getRowType();
@@ -183,35 +143,6 @@ public interface RelNode extends RelOptNode, Cloneable {
   double estimateRowCount(RelMetadataQuery mq);
 
   /**
-   * @deprecated Call {@link RelMetadataQuery#getRowCount(RelNode)};
-   * if you wish to override the default row count formula, override the
-   * {@link #estimateRowCount(RelMetadataQuery)} method.
-   */
-  @Deprecated // to be removed before 1.25
-  double getRows();
-
-  /**
-   * Returns the names of variables that are set in this relational
-   * expression but also used and therefore not available to parents of this
-   * relational expression.
-   *
-   * <p>Note: only {@link org.apache.calcite.rel.core.Correlate} should set
-   * variables.
-   *
-   * <p>Note: {@link #getVariablesSet()} is equivalent but returns
-   * {@link CorrelationId} rather than their names. It is preferable except for
-   * calling old methods that require a set of strings.
-   *
-   * @return Names of variables which are set in this relational
-   *   expression
-   *
-   * @deprecated Use {@link #getVariablesSet()}
-   * and {@link CorrelationId#names(Set)}
-   */
-  @Deprecated // to be removed before 1.25
-  Set<String> getVariablesStopped();
-
-  /**
    * Returns the variables that are set in this relational
    * expression but also used and therefore not available to parents of this
    * relational expression.
@@ -267,14 +198,6 @@ public interface RelNode extends RelOptNode, Cloneable {
   RelOptCost computeSelfCost(RelOptPlanner planner, RelMetadataQuery mq);
 
   /**
-   * @deprecated Call {@link RelMetadataQuery#getNonCumulativeCost(RelNode)};
-   * if you wish to override the default cost formula, override the
-   * {@link #computeSelfCost(RelOptPlanner, RelMetadataQuery)} method.
-   */
-  @Deprecated // to be removed before 1.25
-  RelOptCost computeSelfCost(RelOptPlanner planner);
-
-  /**
    * Returns a metadata interface.
    *
    * @param <M> Type of metadata being requested
@@ -408,27 +331,12 @@ public interface RelNode extends RelOptNode, Cloneable {
    */
   boolean isValid(Litmus litmus, Context context);
 
-  @Deprecated // to be removed before 1.25
-  boolean isValid(boolean fail);
-
-  /**
-   * Returns a description of the physical ordering (or orderings) of this
-   * relational expression. Never null.
-   *
-   * @return Description of the physical ordering (or orderings) of this
-   *   relational expression. Never null
-   *
-   * @deprecated Use {@link RelMetadataQuery#distribution(RelNode)}
-   */
-  @Deprecated // to be removed before 1.25
-  List<RelCollation> getCollationList();
-
   /**
    * Creates a copy of this relational expression, perhaps changing traits and
    * inputs.
    *
    * <p>Sub-classes with other important attributes are encouraged to create
-   * variants of this method with more parameters.</p>
+   * variants of this method with more parameters.
    *
    * @param traitSet Trait set
    * @param inputs   Inputs
@@ -465,22 +373,6 @@ public interface RelNode extends RelOptNode, Cloneable {
   }
 
   /**
-   * Returns whether the result of this relational expression is uniquely
-   * identified by this columns with the given ordinals.
-   *
-   * <p>For example, if this relational expression is a LogicalTableScan to
-   * T(A, B, C, D) whose key is (A, B), then isKey([0, 1]) yields true,
-   * and isKey([0]) and isKey([0, 2]) yields false.</p>
-   *
-   * @param columns Ordinals of key columns
-   * @return Whether the given columns are a key or a superset of a key
-   *
-   * @deprecated Use {@link RelMetadataQuery#areColumnsUnique(RelNode, ImmutableBitSet)}
-   */
-  @Deprecated // to be removed before 1.25
-  boolean isKey(ImmutableBitSet columns);
-
-  /**
    * Accepts a visit from a shuttle.
    *
    * @param shuttle Shuttle
diff --git a/core/src/main/java/org/apache/calcite/rel/core/Filter.java b/core/src/main/java/org/apache/calcite/rel/core/Filter.java
index 91c435e..e3ba20a 100644
--- a/core/src/main/java/org/apache/calcite/rel/core/Filter.java
+++ b/core/src/main/java/org/apache/calcite/rel/core/Filter.java
@@ -35,8 +35,6 @@ import org.apache.calcite.rex.RexShuttle;
 import org.apache.calcite.rex.RexUtil;
 import org.apache.calcite.util.Litmus;
 
-import com.google.common.collect.ImmutableList;
-
 import org.apiguardian.api.API;
 
 import java.util.List;
@@ -99,10 +97,6 @@ public abstract class Filter extends SingleRel {
   public abstract Filter copy(RelTraitSet traitSet, RelNode input,
       RexNode condition);
 
-  @Override public List<RexNode> getChildExps() {
-    return ImmutableList.of(condition);
-  }
-
   public RelNode accept(RexShuttle shuttle) {
     RexNode condition = shuttle.apply(this.condition);
     if (this.condition == condition) {
diff --git a/core/src/main/java/org/apache/calcite/rel/core/Join.java b/core/src/main/java/org/apache/calcite/rel/core/Join.java
index c686b37..4f38695 100644
--- a/core/src/main/java/org/apache/calcite/rel/core/Join.java
+++ b/core/src/main/java/org/apache/calcite/rel/core/Join.java
@@ -73,22 +73,9 @@ public abstract class Join extends BiRel implements Hintable {
 
   //~ Constructors -----------------------------------------------------------
 
-  // Next time we need to change the constructor of Join, let's change the
-  // "Set<String> variablesStopped" parameter to
-  // "Set<CorrelationId> variablesSet". At that point we would deprecate
-  // RelNode.getVariablesStopped().
-
   /**
    * Creates a Join.
    *
-   * <p>Note: We plan to change the {@code variablesStopped} parameter to
-   * {@code Set&lt;CorrelationId&gt; variablesSet}
-   * {@link org.apache.calcite.util.Bug#upgrade(String) before version 2.0},
-   * because {@link #getVariablesSet()}
-   * is preferred over {@link #getVariablesStopped()}.
-   * This constructor is not deprecated, for now, because maintaining overloaded
-   * constructors in multiple sub-classes would be onerous.
-   *
    * @param cluster          Cluster
    * @param traitSet         Trait set
    * @param hints            Hints
@@ -96,7 +83,7 @@ public abstract class Join extends BiRel implements Hintable {
    * @param right            Right input
    * @param condition        Join condition
    * @param joinType         Join type
-   * @param variablesSet     Set variables that are set by the
+   * @param variablesSet     variables that are set by the
    *                         LHS and used by the RHS and are not available to
    *                         nodes above this Join in the tree
    */
@@ -141,10 +128,6 @@ public abstract class Join extends BiRel implements Hintable {
 
   //~ Methods ----------------------------------------------------------------
 
-  @Override public List<RexNode> getChildExps() {
-    return ImmutableList.of(condition);
-  }
-
   @Override public RelNode accept(RexShuttle shuttle) {
     RexNode condition = shuttle.apply(this.condition);
     if (this.condition == condition) {
diff --git a/core/src/main/java/org/apache/calcite/rel/core/Project.java b/core/src/main/java/org/apache/calcite/rel/core/Project.java
index 92c7d4f..47ca1c4 100644
--- a/core/src/main/java/org/apache/calcite/rel/core/Project.java
+++ b/core/src/main/java/org/apache/calcite/rel/core/Project.java
@@ -154,10 +154,6 @@ public abstract class Project extends SingleRel implements Hintable {
     return true;
   }
 
-  @Override public List<RexNode> getChildExps() {
-    return exps;
-  }
-
   public RelNode accept(RexShuttle shuttle) {
     List<RexNode> exps = shuttle.apply(this.exps);
     if (this.exps == exps) {
diff --git a/core/src/main/java/org/apache/calcite/rel/core/Snapshot.java b/core/src/main/java/org/apache/calcite/rel/core/Snapshot.java
index 2f78973..eaba15d 100644
--- a/core/src/main/java/org/apache/calcite/rel/core/Snapshot.java
+++ b/core/src/main/java/org/apache/calcite/rel/core/Snapshot.java
@@ -28,8 +28,6 @@ import org.apache.calcite.rex.RexShuttle;
 import org.apache.calcite.sql.type.SqlTypeName;
 import org.apache.calcite.util.Litmus;
 
-import com.google.common.collect.ImmutableList;
-
 import java.util.List;
 import java.util.Objects;
 
@@ -76,10 +74,6 @@ public abstract class Snapshot extends SingleRel  {
 
   public abstract Snapshot copy(RelTraitSet traitSet, RelNode input, RexNode period);
 
-  @Override public List<RexNode> getChildExps() {
-    return ImmutableList.of(period);
-  }
-
   public RelNode accept(RexShuttle shuttle) {
     RexNode condition = shuttle.apply(this.period);
     if (this.period == condition) {
diff --git a/core/src/main/java/org/apache/calcite/rel/core/Sort.java b/core/src/main/java/org/apache/calcite/rel/core/Sort.java
index 4bbcdd8..de12dd7 100644
--- a/core/src/main/java/org/apache/calcite/rel/core/Sort.java
+++ b/core/src/main/java/org/apache/calcite/rel/core/Sort.java
@@ -33,10 +33,10 @@ import org.apache.calcite.rex.RexNode;
 import org.apache.calcite.rex.RexShuttle;
 import org.apache.calcite.util.Util;
 
-import com.google.common.collect.ImmutableList;
+import com.google.common.collect.Lists;
 
-import java.util.Collections;
 import java.util.List;
+import java.util.Objects;
 
 /**
  * Relational expression that imposes a particular sort order on its input
@@ -46,7 +46,6 @@ public abstract class Sort extends SingleRel {
   //~ Instance fields --------------------------------------------------------
 
   public final RelCollation collation;
-  protected final ImmutableList<RexNode> fieldExps;
   public final RexNode offset;
   public final RexNode fetch;
 
@@ -97,12 +96,6 @@ public abstract class Sort extends SingleRel {
         && offset == null
         && collation.getFieldCollations().isEmpty())
         : "trivial sort";
-    ImmutableList.Builder<RexNode> builder = ImmutableList.builder();
-    for (RelFieldCollation field : collation.getFieldCollations()) {
-      int index = field.getFieldIndex();
-      builder.add(cluster.getRexBuilder().makeInputRef(child, index));
-    }
-    fieldExps = builder.build();
   }
 
   /**
@@ -139,17 +132,14 @@ public abstract class Sort extends SingleRel {
     return planner.getCostFactory().makeCost(rowCount, cpu, 0);
   }
 
-  @Override public List<RexNode> getChildExps() {
-    return fieldExps;
-  }
-
   public RelNode accept(RexShuttle shuttle) {
     RexNode offset = shuttle.apply(this.offset);
     RexNode fetch = shuttle.apply(this.fetch);
-    List<RexNode> fieldExps = shuttle.apply(this.fieldExps);
-    assert fieldExps == this.fieldExps
+    List<RexNode> originalSortExps = getSortExps();
+    List<RexNode> sortExps = shuttle.apply(originalSortExps);
+    assert sortExps == originalSortExps
         : "Sort node does not support modification of input field expressions."
-          + " Old expressions: " + this.fieldExps + ", new ones: " + fieldExps;
+          + " Old expressions: " + originalSortExps + ", new ones: " + sortExps;
     if (offset == this.offset
         && fetch == this.fetch) {
       return this;
@@ -178,18 +168,20 @@ public abstract class Sort extends SingleRel {
     return collation;
   }
 
-  @SuppressWarnings("deprecation")
-  @Override public List<RelCollation> getCollationList() {
-    return Collections.singletonList(getCollation());
+  /** Returns the sort expressions. */
+  public List<RexNode> getSortExps() {
+    //noinspection StaticPseudoFunctionalStyleMethod
+    return Lists.transform(collation.getFieldCollations(), field ->
+        getCluster().getRexBuilder().makeInputRef(input,
+            Objects.requireNonNull(field).getFieldIndex()));
   }
 
   public RelWriter explainTerms(RelWriter pw) {
     super.explainTerms(pw);
-    assert fieldExps.size() == collation.getFieldCollations().size();
     if (pw.nest()) {
       pw.item("collation", collation);
     } else {
-      for (Ord<RexNode> ord : Ord.zip(fieldExps)) {
+      for (Ord<RexNode> ord : Ord.zip(getSortExps())) {
         pw.item("sort" + ord.i, ord.e);
       }
       for (Ord<RelFieldCollation> ord
diff --git a/core/src/main/java/org/apache/calcite/rel/core/TableFunctionScan.java b/core/src/main/java/org/apache/calcite/rel/core/TableFunctionScan.java
index 5a39071..c79e0a5 100644
--- a/core/src/main/java/org/apache/calcite/rel/core/TableFunctionScan.java
+++ b/core/src/main/java/org/apache/calcite/rel/core/TableFunctionScan.java
@@ -133,10 +133,6 @@ public abstract class TableFunctionScan extends AbstractRelNode {
     return inputs;
   }
 
-  @Override public List<RexNode> getChildExps() {
-    return ImmutableList.of(rexCall);
-  }
-
   public RelNode accept(RexShuttle shuttle) {
     RexNode rexCall = shuttle.apply(this.rexCall);
     if (rexCall == this.rexCall) {
diff --git a/core/src/main/java/org/apache/calcite/rel/core/TableScan.java b/core/src/main/java/org/apache/calcite/rel/core/TableScan.java
index c826f3c..7f6dd35 100644
--- a/core/src/main/java/org/apache/calcite/rel/core/TableScan.java
+++ b/core/src/main/java/org/apache/calcite/rel/core/TableScan.java
@@ -22,7 +22,6 @@ import org.apache.calcite.plan.RelOptPlanner;
 import org.apache.calcite.plan.RelOptTable;
 import org.apache.calcite.plan.RelTraitSet;
 import org.apache.calcite.rel.AbstractRelNode;
-import org.apache.calcite.rel.RelCollation;
 import org.apache.calcite.rel.RelInput;
 import org.apache.calcite.rel.RelNode;
 import org.apache.calcite.rel.RelShuttle;
@@ -96,11 +95,6 @@ public abstract class TableScan
     return table;
   }
 
-  @SuppressWarnings("deprecation")
-  @Override public List<RelCollation> getCollationList() {
-    return table.getCollationList();
-  }
-
   @Override public RelOptCost computeSelfCost(RelOptPlanner planner,
       RelMetadataQuery mq) {
     double dRows = table.getRowCount();
diff --git a/core/src/main/java/org/apache/calcite/rel/rules/MultiJoin.java b/core/src/main/java/org/apache/calcite/rel/rules/MultiJoin.java
index ebf669b..9fb75d9 100644
--- a/core/src/main/java/org/apache/calcite/rel/rules/MultiJoin.java
+++ b/core/src/main/java/org/apache/calcite/rel/rules/MultiJoin.java
@@ -180,10 +180,6 @@ public final class MultiJoin extends AbstractRelNode {
     return inputs;
   }
 
-  @Override public List<RexNode> getChildExps() {
-    return ImmutableList.of(joinFilter);
-  }
-
   public RelNode accept(RexShuttle shuttle) {
     RexNode joinFilter = shuttle.apply(this.joinFilter);
     List<RexNode> outerJoinConditions = shuttle.apply(this.outerJoinConditions);
diff --git a/core/src/main/java/org/apache/calcite/rex/RexShuttle.java b/core/src/main/java/org/apache/calcite/rex/RexShuttle.java
index 2aaabad..740c995 100644
--- a/core/src/main/java/org/apache/calcite/rex/RexShuttle.java
+++ b/core/src/main/java/org/apache/calcite/rex/RexShuttle.java
@@ -17,7 +17,6 @@
 package org.apache.calcite.rex;
 
 import com.google.common.collect.ImmutableList;
-import com.google.common.collect.Iterables;
 
 import java.util.ArrayList;
 import java.util.List;
@@ -261,13 +260,6 @@ public class RexShuttle implements RexVisitor<RexNode> {
     }
   }
 
-  /** @deprecated Use {@link RexVisitor#visitList(Iterable)} if possible. */
-  @Deprecated // to be removed before 1.25
-  public final Iterable<RexNode> apply(Iterable<? extends RexNode> iterable) {
-    return Iterables.transform(iterable,
-        t -> t == null ? null : t.accept(RexShuttle.this));
-  }
-
   /**
    * Applies this shuttle to an expression, or returns null if the expression
    * is null.
diff --git a/core/src/main/java/org/apache/calcite/sql/SqlDialect.java b/core/src/main/java/org/apache/calcite/sql/SqlDialect.java
index f9434f2..6c19c1a 100644
--- a/core/src/main/java/org/apache/calcite/sql/SqlDialect.java
+++ b/core/src/main/java/org/apache/calcite/sql/SqlDialect.java
@@ -27,8 +27,6 @@ import org.apache.calcite.rel.type.RelDataTypeSystem;
 import org.apache.calcite.rel.type.RelDataTypeSystemImpl;
 import org.apache.calcite.rex.RexCall;
 import org.apache.calcite.rex.RexNode;
-import org.apache.calcite.sql.dialect.AnsiSqlDialect;
-import org.apache.calcite.sql.dialect.CalciteSqlDialect;
 import org.apache.calcite.sql.dialect.JethroDataSqlDialect;
 import org.apache.calcite.sql.fun.SqlStdOperatorTable;
 import org.apache.calcite.sql.parser.SqlParser;
@@ -83,16 +81,6 @@ public class SqlDialect {
   /** Empty context. */
   public static final Context EMPTY_CONTEXT = emptyContext();
 
-  /** @deprecated Use {@link AnsiSqlDialect#DEFAULT} instead. */
-  @Deprecated // to be removed before 1.25
-  public static final SqlDialect DUMMY =
-      AnsiSqlDialect.DEFAULT;
-
-  /** @deprecated Use {@link CalciteSqlDialect#DEFAULT} instead. */
-  @Deprecated // to be removed before 1.25
-  public static final SqlDialect CALCITE =
-      CalciteSqlDialect.DEFAULT;
-
   /** Built-in scalar functions and operators common for every dialect. */
   protected static final Set<SqlOperator> BUILT_IN_OPERATORS_LIST =
       ImmutableSet.<SqlOperator>builder()
diff --git a/core/src/main/java/org/apache/calcite/sql/SqlExecutableStatement.java b/core/src/main/java/org/apache/calcite/sql/SqlExecutableStatement.java
deleted file mode 100644
index 56cf8d9..0000000
--- a/core/src/main/java/org/apache/calcite/sql/SqlExecutableStatement.java
+++ /dev/null
@@ -1,32 +0,0 @@
-/*
- * 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.sql;
-
-import org.apache.calcite.jdbc.CalcitePrepare;
-import org.apache.calcite.linq4j.function.Experimental;
-
-/**
- * Mix-in interface for {@link SqlNode} that allows DDL commands to be
- * executed directly.
- *
- * <p>NOTE: Subject to change without notice.
- */
-@Experimental
-@Deprecated // to be removed before 1.25
-public interface SqlExecutableStatement {
-  void execute(CalcitePrepare.Context context);
-}
diff --git a/core/src/main/java/org/apache/calcite/tools/RelBuilder.java b/core/src/main/java/org/apache/calcite/tools/RelBuilder.java
index 3362a24..f3b8188 100644
--- a/core/src/main/java/org/apache/calcite/tools/RelBuilder.java
+++ b/core/src/main/java/org/apache/calcite/tools/RelBuilder.java
@@ -1478,16 +1478,6 @@ public class RelBuilder {
     return this;
   }
 
-  /** Whether to attempt to merge consecutive {@link Project} operators.
-   *
-   * <p>The default implementation returns {@code true};
-   * sub-classes may disable merge by overriding to return {@code false}. */
-  @Experimental
-  @Deprecated // to be removed before 1.25
-  protected boolean shouldMergeProject() {
-    return true;
-  }
-
   /** Creates a {@link Project} of the given
    * expressions and field names, and optionally optimizing.
    *
diff --git a/core/src/test/java/org/apache/calcite/test/RelBuilderTest.java b/core/src/test/java/org/apache/calcite/test/RelBuilderTest.java
index dd1463d..8c213f7 100644
--- a/core/src/test/java/org/apache/calcite/test/RelBuilderTest.java
+++ b/core/src/test/java/org/apache/calcite/test/RelBuilderTest.java
@@ -31,6 +31,7 @@ import org.apache.calcite.rel.core.CorrelationId;
 import org.apache.calcite.rel.core.Exchange;
 import org.apache.calcite.rel.core.JoinRelType;
 import org.apache.calcite.rel.core.Project;
+import org.apache.calcite.rel.core.Sort;
 import org.apache.calcite.rel.core.TableFunctionScan;
 import org.apache.calcite.rel.core.TableModify;
 import org.apache.calcite.rel.core.Window;
@@ -2793,6 +2794,7 @@ public class RelBuilderTest {
         "LogicalSort(sort0=[$2], sort1=[$0], dir0=[ASC], dir1=[DESC])\n"
             + "  LogicalTableScan(table=[[scott, EMP]])\n";
     assertThat(root, hasTree(expected));
+    assertThat(((Sort) root).getSortExps().toString(), is("[$2, $0]"));
 
     // same result using ordinals
     final RelNode root2 =


[calcite] 06/09: [CALCITE-4134] Interval expressions

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

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

commit 03c76a7d2b896042ab417ddc36f1849f874ad3dd
Author: Julian Hyde <jh...@apache.org>
AuthorDate: Mon Jul 20 13:33:06 2020 -0700

    [CALCITE-4134] Interval expressions
---
 core/src/main/codegen/templates/Parser.jj          | 159 +++++++++++++++++----
 .../java/org/apache/calcite/sql/SqlDialect.java    |   2 +-
 .../main/java/org/apache/calcite/sql/SqlKind.java  |   5 +-
 .../java/org/apache/calcite/sql/SqlLiteral.java    |  18 +++
 .../main/java/org/apache/calcite/sql/SqlNode.java  |  11 ++
 .../calcite/sql/dialect/BigQuerySqlDialect.java    |   6 +-
 .../apache/calcite/sql/dialect/Db2SqlDialect.java  |   2 +-
 .../calcite/sql/dialect/MssqlSqlDialect.java       |   2 +-
 .../calcite/sql/fun/SqlIntervalOperator.java       |  83 +++++++++++
 .../calcite/sql/fun/SqlStdOperatorTable.java       |   6 +
 .../calcite/sql/type/SqlOperandTypeChecker.java    |   2 +
 .../calcite/sql/type/SqlOperandTypeInference.java  |   2 +
 .../calcite/sql/type/SqlReturnTypeInference.java   |   4 +-
 .../apache/calcite/sql/type/SqlTypeTransform.java  |   2 +
 .../calcite/sql/validate/SqlValidatorImpl.java     |   3 +-
 .../calcite/sql2rel/ReflectiveConvertletTable.java |   2 +-
 .../calcite/sql2rel/SqlNodeToRexConverterImpl.java |   9 +-
 .../calcite/sql2rel/StandardConvertletTable.java   |  20 ++-
 .../apache/calcite/sql/parser/SqlParserTest.java   |  20 +++
 .../apache/calcite/sql/test/AbstractSqlTester.java |   2 +-
 .../apache/calcite/test/SqlToRelConverterTest.java |   4 +
 .../org/apache/calcite/test/SqlValidatorTest.java  | 149 +++++++++++--------
 .../apache/calcite/test/SqlToRelConverterTest.xml  |  12 +-
 core/src/test/resources/sql/misc.iq                |  27 ++++
 24 files changed, 444 insertions(+), 108 deletions(-)

diff --git a/core/src/main/codegen/templates/Parser.jj b/core/src/main/codegen/templates/Parser.jj
index 1d8465e..279fa38 100644
--- a/core/src/main/codegen/templates/Parser.jj
+++ b/core/src/main/codegen/templates/Parser.jj
@@ -3770,7 +3770,7 @@ SqlNode AtomicRowExpression() :
 }
 {
     (
-        e = Literal()
+        e = LiteralOrIntervalExpression()
     |
         e = DynamicParam()
     |
@@ -4007,6 +4007,10 @@ SqlDrop SqlDrop() :
  * Usually returns an SqlLiteral, but a continued string literal
  * is an SqlCall expression, which concatenates 2 or more string
  * literals; the validator reduces this.
+ *
+ * <p>If the context allows both literals and expressions,
+ * use {@link #LiteralOrIntervalExpression}, which requires less
+ * lookahead.
  */
 SqlNode Literal() :
 {
@@ -4014,6 +4018,20 @@ SqlNode Literal() :
 }
 {
     (
+        e = NonIntervalLiteral()
+    |
+        e = IntervalLiteral()
+    )
+    { return e; }
+}
+
+/** Parses a literal that is not an interval literal. */
+SqlNode NonIntervalLiteral() :
+{
+    final SqlNode e;
+}
+{
+    (
         e = NumericLiteral()
     |
         e = StringLiteral()
@@ -4021,8 +4039,6 @@ SqlNode Literal() :
         e = SpecialLiteral()
     |
         e = DateTimeLiteral()
-    |
-        e = IntervalLiteral()
 <#-- additional literal parser methods are included here -->
 <#list parser.literalParserMethods as method>
     |
@@ -4032,8 +4048,25 @@ SqlNode Literal() :
     {
         return e;
     }
+}
 
-
+/** Parses a literal or an interval expression.
+ *
+ * <p>We include them in the same production because it is difficult to
+ * distinguish interval literals from interval expression (both of which
+ * start with the {@code INTERVAL} keyword); this way, we can use less
+ * LOOKAHEAD. */
+SqlNode LiteralOrIntervalExpression() :
+{
+    final SqlNode e;
+}
+{
+    (
+        e = IntervalLiteralOrExpression()
+    |
+        e = NonIntervalLiteral()
+    )
+    { return e; }
 }
 
 /** Parses a unsigned numeric literal */
@@ -4416,6 +4449,53 @@ SqlLiteral IntervalLiteral() :
     }
 }
 
+/** Parses an interval literal (e.g. {@code INTERVAL '2:3' HOUR TO MINUTE})
+ * or an interval expression (e.g. {@code INTERVAL emp.empno MINUTE}
+ * or {@code INTERVAL 3 MONTHS}). */
+SqlNode IntervalLiteralOrExpression() :
+{
+    final String p;
+    final SqlIntervalQualifier intervalQualifier;
+    int sign = 1;
+    final Span s;
+    SqlNode e;
+}
+{
+    <INTERVAL> { s = span(); }
+    [
+        <MINUS> { sign = -1; }
+    |
+        <PLUS> { sign = 1; }
+    ]
+    (
+        // literal (with quoted string)
+        <QUOTED_STRING> { p = token.image; }
+        intervalQualifier = IntervalQualifier() {
+            return SqlParserUtil.parseIntervalLiteral(s.end(intervalQualifier),
+                sign, p, intervalQualifier);
+        }
+    |
+        // To keep parsing simple, any expressions besides numeric literal and
+        // identifiers must be enclosed in parentheses.
+        (
+            <LPAREN>
+            e = Expression(ExprContext.ACCEPT_SUB_QUERY)
+            <RPAREN>
+        |
+            e = UnsignedNumericLiteral()
+        |
+            e = CompoundIdentifier()
+        )
+        intervalQualifier = IntervalQualifierStart() {
+            if (sign == -1) {
+                e = SqlStdOperatorTable.UNARY_MINUS.createCall(e.getParserPosition(), e);
+            }
+            return SqlStdOperatorTable.INTERVAL.createCall(s.end(this), e,
+                intervalQualifier);
+        }
+    )
+}
+
 TimeUnit Year() :
 {
 }
@@ -4472,6 +4552,7 @@ TimeUnit Second() :
 
 SqlIntervalQualifier IntervalQualifier() :
 {
+    final Span s;
     final TimeUnit start;
     TimeUnit end = null;
     int startPrec = RelDataType.PRECISION_NOT_SPECIFIED;
@@ -4479,27 +4560,28 @@ SqlIntervalQualifier IntervalQualifier() :
 }
 {
     (
-        start = Year() [ <LPAREN> startPrec = UnsignedIntLiteral() <RPAREN> ]
+        start = Year() { s = span(); } startPrec = PrecisionOpt()
         [
             LOOKAHEAD(2) <TO> end = Month()
         ]
     |
-        start = Month() [ <LPAREN> startPrec = UnsignedIntLiteral() <RPAREN> ]
+        start = Month() { s = span(); } startPrec = PrecisionOpt()
     |
-        start = Day() [ <LPAREN> startPrec = UnsignedIntLiteral() <RPAREN> ]
-        [ LOOKAHEAD(2) <TO>
+        start = Day() { s = span(); } startPrec = PrecisionOpt()
+        [
+            LOOKAHEAD(2) <TO>
             (
                 end = Hour()
             |
                 end = Minute()
             |
-                end = Second()
-                [ <LPAREN> secondFracPrec = UnsignedIntLiteral() <RPAREN> ]
+                end = Second() secondFracPrec = PrecisionOpt()
             )
         ]
     |
-        start = Hour() [ <LPAREN> startPrec = UnsignedIntLiteral() <RPAREN> ]
-        [ LOOKAHEAD(2) <TO>
+        start = Hour() { s = span(); } startPrec = PrecisionOpt()
+        [
+            LOOKAHEAD(2) <TO>
             (
                 end = Minute()
             |
@@ -4508,26 +4590,54 @@ SqlIntervalQualifier IntervalQualifier() :
             )
         ]
     |
-        start = Minute() [ <LPAREN> startPrec = UnsignedIntLiteral() <RPAREN> ]
-        [ LOOKAHEAD(2) <TO>
-            (
-                end = Second()
-                [ <LPAREN> secondFracPrec = UnsignedIntLiteral() <RPAREN> ]
-            )
+        start = Minute() { s = span(); } startPrec = PrecisionOpt()
+        [
+            LOOKAHEAD(2) <TO> end = Second()
+            [ <LPAREN> secondFracPrec = UnsignedIntLiteral() <RPAREN> ]
         ]
     |
-        start = Second()
+        start = Second() { s = span(); }
+        [
+            <LPAREN> startPrec = UnsignedIntLiteral()
+            [ <COMMA> secondFracPrec = UnsignedIntLiteral() ]
+            <RPAREN>
+        ]
+    )
+    {
+        return new SqlIntervalQualifier(start, startPrec, end, secondFracPrec,
+            s.end(this));
+    }
+}
+
+/** Interval qualifier without 'TO unit'. */
+SqlIntervalQualifier IntervalQualifierStart() :
+{
+    final Span s;
+    final TimeUnit start;
+    int startPrec = RelDataType.PRECISION_NOT_SPECIFIED;
+    int secondFracPrec = RelDataType.PRECISION_NOT_SPECIFIED;
+}
+{
+    (
+        (
+            start = Year()
+        |   start = Month()
+        |   start = Day()
+        |   start = Hour()
+        |   start = Minute()
+        )
+        { s = span(); }
+        startPrec = PrecisionOpt()
+    |
+        start = Second() { s = span(); }
         [   <LPAREN> startPrec = UnsignedIntLiteral()
             [ <COMMA> secondFracPrec = UnsignedIntLiteral() ]
             <RPAREN>
         ]
     )
     {
-        return new SqlIntervalQualifier(start,
-            startPrec,
-            end,
-            secondFracPrec,
-            getPos());
+        return new SqlIntervalQualifier(start, startPrec, null, secondFracPrec,
+            s.end(this));
     }
 }
 
@@ -5260,7 +5370,6 @@ int PrecisionOpt() :
     int precision = -1;
 }
 {
-    LOOKAHEAD(2)
     <LPAREN>
     precision = UnsignedIntLiteral()
     <RPAREN>
diff --git a/core/src/main/java/org/apache/calcite/sql/SqlDialect.java b/core/src/main/java/org/apache/calcite/sql/SqlDialect.java
index 6c19c1a..5dedfc8 100644
--- a/core/src/main/java/org/apache/calcite/sql/SqlDialect.java
+++ b/core/src/main/java/org/apache/calcite/sql/SqlDialect.java
@@ -526,7 +526,7 @@ public class SqlDialect {
   public void unparseSqlIntervalLiteral(SqlWriter writer,
       SqlIntervalLiteral literal, int leftPrec, int rightPrec) {
     SqlIntervalLiteral.IntervalValue interval =
-        (SqlIntervalLiteral.IntervalValue) literal.getValue();
+        literal.getValueAs(SqlIntervalLiteral.IntervalValue.class);
     writer.keyword("INTERVAL");
     if (interval.getSign() == -1) {
       writer.print("-");
diff --git a/core/src/main/java/org/apache/calcite/sql/SqlKind.java b/core/src/main/java/org/apache/calcite/sql/SqlKind.java
index adb3223..1fee9bf 100644
--- a/core/src/main/java/org/apache/calcite/sql/SqlKind.java
+++ b/core/src/main/java/org/apache/calcite/sql/SqlKind.java
@@ -349,6 +349,9 @@ public enum SqlKind {
   /** {@code CASE} expression. */
   CASE,
 
+  /** {@code INTERVAL} expression. */
+  INTERVAL,
+
   /** {@code NULLIF} operator. */
   NULLIF,
 
@@ -1066,7 +1069,7 @@ public enum SqlKind {
                   FILTER, WITHIN_GROUP, IGNORE_NULLS, RESPECT_NULLS,
                   DESCENDING, CUBE, ROLLUP, GROUPING_SETS, EXTEND, LATERAL,
                   SELECT, JOIN, OTHER_FUNCTION, POSITION, CAST, TRIM, FLOOR, CEIL,
-                  TIMESTAMP_ADD, TIMESTAMP_DIFF, EXTRACT,
+                  TIMESTAMP_ADD, TIMESTAMP_DIFF, EXTRACT, INTERVAL,
                   LITERAL_CHAIN, JDBC_FN, PRECEDING, FOLLOWING, ORDER_BY,
                   NULLS_FIRST, NULLS_LAST, COLLECTION_TABLE, TABLESAMPLE,
                   VALUES, WITH, WITH_ITEM, ITEM, SKIP_TO_FIRST, SKIP_TO_LAST,
diff --git a/core/src/main/java/org/apache/calcite/sql/SqlLiteral.java b/core/src/main/java/org/apache/calcite/sql/SqlLiteral.java
index bd80488..7ce5fa1 100644
--- a/core/src/main/java/org/apache/calcite/sql/SqlLiteral.java
+++ b/core/src/main/java/org/apache/calcite/sql/SqlLiteral.java
@@ -252,6 +252,20 @@ public class SqlLiteral extends SqlNode {
     return value;
   }
 
+  /**
+   * Returns the value of this literal as a particular type.
+   *
+   * <p>The type might be the internal type, or other convenient types.
+   * For example, numeric literals' values are stored internally as
+   * {@link BigDecimal}, but other numeric types such as {@link Long} and
+   * {@link Double} are also allowed.
+   *
+   * @param clazz Desired value type
+   * @param <T> Value type
+   * @return Value of the literal
+   *
+   * @throws AssertionError if the value type is not supported
+   */
   public <T> T getValueAs(Class<T> clazz) {
     if (clazz.isInstance(value)) {
       return clazz.cast(value);
@@ -320,6 +334,8 @@ public class SqlLiteral extends SqlNode {
         return clazz.cast(BigDecimal.valueOf(getValueAs(Long.class)));
       } else if (clazz == TimeUnitRange.class) {
         return clazz.cast(valMonth.getIntervalQualifier().timeUnitRange);
+      } else if (clazz == SqlIntervalQualifier.class) {
+        return clazz.cast(valMonth.getIntervalQualifier());
       }
       break;
     case INTERVAL_DAY:
@@ -341,6 +357,8 @@ public class SqlLiteral extends SqlNode {
         return clazz.cast(BigDecimal.valueOf(getValueAs(Long.class)));
       } else if (clazz == TimeUnitRange.class) {
         return clazz.cast(valTime.getIntervalQualifier().timeUnitRange);
+      } else if (clazz == SqlIntervalQualifier.class) {
+        return clazz.cast(valTime.getIntervalQualifier());
       }
       break;
     }
diff --git a/core/src/main/java/org/apache/calcite/sql/SqlNode.java b/core/src/main/java/org/apache/calcite/sql/SqlNode.java
index ec69cae..82bad71 100644
--- a/core/src/main/java/org/apache/calcite/sql/SqlNode.java
+++ b/core/src/main/java/org/apache/calcite/sql/SqlNode.java
@@ -211,6 +211,17 @@ public abstract class SqlNode implements Cloneable {
       int leftPrec,
       int rightPrec);
 
+  public void unparseWithParentheses(SqlWriter writer, int leftPrec,
+      int rightPrec, boolean parentheses) {
+    if (parentheses) {
+      final SqlWriter.Frame frame = writer.startList("(", ")");
+      unparse(writer, 0, 0);
+      writer.endList(frame);
+    } else {
+      unparse(writer, leftPrec, rightPrec);
+    }
+  }
+
   public SqlParserPos getParserPosition() {
     return pos;
   }
diff --git a/core/src/main/java/org/apache/calcite/sql/dialect/BigQuerySqlDialect.java b/core/src/main/java/org/apache/calcite/sql/dialect/BigQuerySqlDialect.java
index af2e855..19276b1 100644
--- a/core/src/main/java/org/apache/calcite/sql/dialect/BigQuerySqlDialect.java
+++ b/core/src/main/java/org/apache/calcite/sql/dialect/BigQuerySqlDialect.java
@@ -164,10 +164,10 @@ public class BigQuerySqlDialect extends SqlDialect {
   }
 
   /** BigQuery interval syntax: INTERVAL int64 time_unit. */
-  @Override public void unparseSqlIntervalLiteral(
-          SqlWriter writer, SqlIntervalLiteral literal, int leftPrec, int rightPrec) {
+  @Override public void unparseSqlIntervalLiteral(SqlWriter writer,
+      SqlIntervalLiteral literal, int leftPrec, int rightPrec) {
     SqlIntervalLiteral.IntervalValue interval =
-            (SqlIntervalLiteral.IntervalValue) literal.getValue();
+        literal.getValueAs(SqlIntervalLiteral.IntervalValue.class);
     writer.keyword("INTERVAL");
     if (interval.getSign() == -1) {
       writer.print("-");
diff --git a/core/src/main/java/org/apache/calcite/sql/dialect/Db2SqlDialect.java b/core/src/main/java/org/apache/calcite/sql/dialect/Db2SqlDialect.java
index 27994af..9dbbda0 100644
--- a/core/src/main/java/org/apache/calcite/sql/dialect/Db2SqlDialect.java
+++ b/core/src/main/java/org/apache/calcite/sql/dialect/Db2SqlDialect.java
@@ -84,7 +84,7 @@ public class Db2SqlDialect extends SqlDialect {
     // If one operand is a timestamp, the other operand can be any of teh duration.
 
     SqlIntervalLiteral.IntervalValue interval =
-        (SqlIntervalLiteral.IntervalValue) literal.getValue();
+        literal.getValueAs(SqlIntervalLiteral.IntervalValue.class);
     if (interval.getSign() == -1) {
       writer.print("-");
     }
diff --git a/core/src/main/java/org/apache/calcite/sql/dialect/MssqlSqlDialect.java b/core/src/main/java/org/apache/calcite/sql/dialect/MssqlSqlDialect.java
index 547f9a9..f005175 100644
--- a/core/src/main/java/org/apache/calcite/sql/dialect/MssqlSqlDialect.java
+++ b/core/src/main/java/org/apache/calcite/sql/dialect/MssqlSqlDialect.java
@@ -270,7 +270,7 @@ public class MssqlSqlDialect extends SqlDialect {
   private void unparseSqlIntervalLiteralMssql(
       SqlWriter writer, SqlIntervalLiteral literal, int sign) {
     final SqlIntervalLiteral.IntervalValue interval =
-        (SqlIntervalLiteral.IntervalValue) literal.getValue();
+        literal.getValueAs(SqlIntervalLiteral.IntervalValue.class);
     unparseSqlIntervalQualifier(writer, interval.getIntervalQualifier(),
         RelDataTypeSystem.DEFAULT);
     writer.sep(",", true);
diff --git a/core/src/main/java/org/apache/calcite/sql/fun/SqlIntervalOperator.java b/core/src/main/java/org/apache/calcite/sql/fun/SqlIntervalOperator.java
new file mode 100644
index 0000000..e959b40
--- /dev/null
+++ b/core/src/main/java/org/apache/calcite/sql/fun/SqlIntervalOperator.java
@@ -0,0 +1,83 @@
+/*
+ * 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.sql.fun;
+
+import org.apache.calcite.rel.type.RelDataType;
+import org.apache.calcite.sql.SqlCall;
+import org.apache.calcite.sql.SqlIdentifier;
+import org.apache.calcite.sql.SqlInternalOperator;
+import org.apache.calcite.sql.SqlIntervalQualifier;
+import org.apache.calcite.sql.SqlKind;
+import org.apache.calcite.sql.SqlLiteral;
+import org.apache.calcite.sql.SqlNode;
+import org.apache.calcite.sql.SqlOperatorBinding;
+import org.apache.calcite.sql.SqlWriter;
+import org.apache.calcite.sql.type.InferTypes;
+import org.apache.calcite.sql.type.OperandTypes;
+import org.apache.calcite.sql.type.SqlReturnTypeInference;
+import org.apache.calcite.sql.type.SqlTypeTransforms;
+
+/** Interval expression.
+ *
+ * <p>Syntax:
+ *
+ * <blockquote><pre>INTERVAL numericExpression timeUnit
+ *
+ * timeUnit: YEAR | MONTH | DAY | HOUR | MINUTE | SECOND</pre></blockquote>
+ *
+ * <p>Compare with interval literal, whose syntax is
+ * {@code INTERVAL characterLiteral timeUnit [ TO timeUnit ]}.
+ */
+public class SqlIntervalOperator extends SqlInternalOperator {
+  private static final SqlReturnTypeInference RETURN_TYPE =
+      ((SqlReturnTypeInference) SqlIntervalOperator::returnType)
+          .andThen(SqlTypeTransforms.TO_NULLABLE);
+
+  SqlIntervalOperator() {
+    super("INTERVAL", SqlKind.INTERVAL, 0, true, RETURN_TYPE,
+        InferTypes.ANY_NULLABLE, OperandTypes.NUMERIC_INTERVAL);
+  }
+
+  private static RelDataType returnType(SqlOperatorBinding opBinding) {
+    final SqlIntervalQualifier intervalQualifier =
+        opBinding.getOperandLiteralValue(1, SqlIntervalQualifier.class);
+    return opBinding.getTypeFactory().createSqlIntervalType(intervalQualifier);
+  }
+
+  @Override public void unparse(SqlWriter writer, SqlCall call, int leftPrec,
+      int rightPrec) {
+    writer.keyword("INTERVAL");
+    final SqlNode expression = call.operand(0);
+    final SqlIntervalQualifier intervalQualifier = call.operand(1);
+    expression.unparseWithParentheses(writer, leftPrec, rightPrec,
+        !(expression instanceof SqlLiteral
+            || expression instanceof SqlIdentifier
+            || expression.getKind() == SqlKind.MINUS_PREFIX
+            || writer.isAlwaysUseParentheses()));
+    assert intervalQualifier.timeUnitRange.endUnit == null;
+    intervalQualifier.unparse(writer, 0, 0);
+  }
+
+  @Override public String getSignatureTemplate(int operandsCount) {
+    switch (operandsCount) {
+    case 2:
+      return "{0} {1} {2}"; // e.g. "INTERVAL <INTEGER> <INTERVAL HOUR>"
+    default:
+      throw new AssertionError();
+    }
+  }
+}
diff --git a/core/src/main/java/org/apache/calcite/sql/fun/SqlStdOperatorTable.java b/core/src/main/java/org/apache/calcite/sql/fun/SqlStdOperatorTable.java
index 46f2208..77cff50 100644
--- a/core/src/main/java/org/apache/calcite/sql/fun/SqlStdOperatorTable.java
+++ b/core/src/main/java/org/apache/calcite/sql/fun/SqlStdOperatorTable.java
@@ -548,6 +548,12 @@ public class SqlStdOperatorTable extends ReflectiveSqlOperatorTable {
       new SqlDatetimePlusOperator();
 
   /**
+   * Interval expression, '<code>INTERVAL n timeUnit</code>'.
+   */
+  public static final SqlSpecialOperator INTERVAL =
+      new SqlIntervalOperator();
+
+  /**
    * Multiset {@code MEMBER OF}, which returns whether a element belongs to a
    * multiset.
    *
diff --git a/core/src/main/java/org/apache/calcite/sql/type/SqlOperandTypeChecker.java b/core/src/main/java/org/apache/calcite/sql/type/SqlOperandTypeChecker.java
index 6e3452c..7f15371 100644
--- a/core/src/main/java/org/apache/calcite/sql/type/SqlOperandTypeChecker.java
+++ b/core/src/main/java/org/apache/calcite/sql/type/SqlOperandTypeChecker.java
@@ -25,6 +25,8 @@ import org.apache.calcite.sql.SqlOperator;
  *
  * <p>This interface is an example of the
  * {@link org.apache.calcite.util.Glossary#STRATEGY_PATTERN strategy pattern}.
+ *
+ * @see OperandTypes
  */
 public interface SqlOperandTypeChecker {
   //~ Methods ----------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/sql/type/SqlOperandTypeInference.java b/core/src/main/java/org/apache/calcite/sql/type/SqlOperandTypeInference.java
index 7091ebb..e27524c 100644
--- a/core/src/main/java/org/apache/calcite/sql/type/SqlOperandTypeInference.java
+++ b/core/src/main/java/org/apache/calcite/sql/type/SqlOperandTypeInference.java
@@ -21,6 +21,8 @@ import org.apache.calcite.sql.SqlCallBinding;
 
 /**
  * Strategy to infer unknown types of the operands of an operator call.
+ *
+ * @see InferTypes
  */
 public interface SqlOperandTypeInference {
   //~ Methods ----------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/sql/type/SqlReturnTypeInference.java b/core/src/main/java/org/apache/calcite/sql/type/SqlReturnTypeInference.java
index c9e31d2..16a80ea 100644
--- a/core/src/main/java/org/apache/calcite/sql/type/SqlReturnTypeInference.java
+++ b/core/src/main/java/org/apache/calcite/sql/type/SqlReturnTypeInference.java
@@ -28,7 +28,9 @@ import org.apache.calcite.sql.SqlOperatorBinding;
  * {@link org.apache.calcite.util.Glossary#STRATEGY_PATTERN strategy pattern}.
  * This makes
  * sense because many operators have similar, straightforward strategies, such
- * as to take the type of the first operand.</p>
+ * as to take the type of the first operand.
+ *
+ * @see ReturnTypes
  */
 @FunctionalInterface
 public interface SqlReturnTypeInference {
diff --git a/core/src/main/java/org/apache/calcite/sql/type/SqlTypeTransform.java b/core/src/main/java/org/apache/calcite/sql/type/SqlTypeTransform.java
index 6bf80b2..ffdd968 100644
--- a/core/src/main/java/org/apache/calcite/sql/type/SqlTypeTransform.java
+++ b/core/src/main/java/org/apache/calcite/sql/type/SqlTypeTransform.java
@@ -27,6 +27,8 @@ import org.apache.calcite.sql.SqlOperatorBinding;
  *
  * <p>This class is an example of the
  * {@link org.apache.calcite.util.Glossary#STRATEGY_PATTERN strategy pattern}.
+ *
+ * @see SqlTypeTransforms
  */
 public interface SqlTypeTransform {
   //~ Methods ----------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/sql/validate/SqlValidatorImpl.java b/core/src/main/java/org/apache/calcite/sql/validate/SqlValidatorImpl.java
index bf59a4f..a81f359 100644
--- a/core/src/main/java/org/apache/calcite/sql/validate/SqlValidatorImpl.java
+++ b/core/src/main/java/org/apache/calcite/sql/validate/SqlValidatorImpl.java
@@ -3088,8 +3088,7 @@ public class SqlValidatorImpl implements SqlValidatorWithHints {
     case INTERVAL_SECOND:
       if (literal instanceof SqlIntervalLiteral) {
         SqlIntervalLiteral.IntervalValue interval =
-            (SqlIntervalLiteral.IntervalValue)
-                literal.getValue();
+            literal.getValueAs(SqlIntervalLiteral.IntervalValue.class);
         SqlIntervalQualifier intervalQualifier =
             interval.getIntervalQualifier();
 
diff --git a/core/src/main/java/org/apache/calcite/sql2rel/ReflectiveConvertletTable.java b/core/src/main/java/org/apache/calcite/sql2rel/ReflectiveConvertletTable.java
index d8243c8..ffa05f0 100644
--- a/core/src/main/java/org/apache/calcite/sql2rel/ReflectiveConvertletTable.java
+++ b/core/src/main/java/org/apache/calcite/sql2rel/ReflectiveConvertletTable.java
@@ -134,7 +134,7 @@ public class ReflectiveConvertletTable implements SqlRexConvertletTable {
     final SqlOperator op = call.getOperator();
 
     // Is there a convertlet for this operator
-    // (e.g. SqlStdOperatorTable.plusOperator)?
+    // (e.g. SqlStdOperatorTable.PLUS)?
     convertlet = (SqlRexConvertlet) map.get(op);
     if (convertlet != null) {
       return convertlet;
diff --git a/core/src/main/java/org/apache/calcite/sql2rel/SqlNodeToRexConverterImpl.java b/core/src/main/java/org/apache/calcite/sql2rel/SqlNodeToRexConverterImpl.java
index a6b124a..c6104c4 100644
--- a/core/src/main/java/org/apache/calcite/sql2rel/SqlNodeToRexConverterImpl.java
+++ b/core/src/main/java/org/apache/calcite/sql2rel/SqlNodeToRexConverterImpl.java
@@ -23,7 +23,6 @@ import org.apache.calcite.rex.RexBuilder;
 import org.apache.calcite.rex.RexLiteral;
 import org.apache.calcite.rex.RexNode;
 import org.apache.calcite.sql.SqlCall;
-import org.apache.calcite.sql.SqlIntervalLiteral;
 import org.apache.calcite.sql.SqlIntervalQualifier;
 import org.apache.calcite.sql.SqlLiteral;
 import org.apache.calcite.sql.SqlTimeLiteral;
@@ -95,10 +94,7 @@ public class SqlNodeToRexConverterImpl implements SqlNodeToRexConverter {
       return rexBuilder.makeNullLiteral(type);
     }
 
-    BitString bitString;
-    SqlIntervalLiteral.IntervalValue intervalValue;
-    long l;
-
+    final BitString bitString;
     switch (literal.getTypeName()) {
     case DECIMAL:
       // exact number
@@ -152,8 +148,7 @@ public class SqlNodeToRexConverterImpl implements SqlNodeToRexConverter {
     case INTERVAL_MINUTE_SECOND:
     case INTERVAL_SECOND:
       SqlIntervalQualifier sqlIntervalQualifier =
-          literal.getValueAs(SqlIntervalLiteral.IntervalValue.class)
-              .getIntervalQualifier();
+          literal.getValueAs(SqlIntervalQualifier.class);
       return rexBuilder.makeIntervalLiteral(
           literal.getValueAs(BigDecimal.class),
           sqlIntervalQualifier);
diff --git a/core/src/main/java/org/apache/calcite/sql2rel/StandardConvertletTable.java b/core/src/main/java/org/apache/calcite/sql2rel/StandardConvertletTable.java
index aaae32e..d1c9bd9 100644
--- a/core/src/main/java/org/apache/calcite/sql2rel/StandardConvertletTable.java
+++ b/core/src/main/java/org/apache/calcite/sql2rel/StandardConvertletTable.java
@@ -259,6 +259,9 @@ public class StandardConvertletTable extends ReflectiveConvertletTable {
     registerOp(SqlStdOperatorTable.TIMESTAMP_DIFF,
         new TimestampDiffConvertlet());
 
+    registerOp(SqlStdOperatorTable.INTERVAL,
+        StandardConvertletTable::convertInterval);
+
     // Convert "element(<expr>)" to "$element_slice(<expr>)", if the
     // expression is a multiset of scalars.
     if (false) {
@@ -294,6 +297,21 @@ public class StandardConvertletTable extends ReflectiveConvertletTable {
     }
   }
 
+  /** Converts an interval expression to a numeric multiplied by an interval
+   * literal. */
+  private static RexNode convertInterval(SqlRexContext cx, SqlCall call) {
+    // "INTERVAL n HOUR" becomes "n * INTERVAL '1' HOUR"
+    final SqlNode n = call.operand(0);
+    final SqlIntervalQualifier intervalQualifier = call.operand(1);
+    final SqlIntervalLiteral literal =
+        SqlLiteral.createInterval(1, "1", intervalQualifier,
+            call.getParserPosition());
+    final SqlCall multiply =
+        SqlStdOperatorTable.MULTIPLY.createCall(call.getParserPosition(), n,
+            literal);
+    return cx.convertExpression(multiply);
+  }
+
   //~ Methods ----------------------------------------------------------------
 
   private RexNode or(RexBuilder rexBuilder, RexNode a0, RexNode a1) {
@@ -546,7 +564,7 @@ public class StandardConvertletTable extends ReflectiveConvertletTable {
         && call.operand(0) instanceof SqlIntervalLiteral) {
       final SqlIntervalLiteral literal = call.operand(0);
       SqlIntervalLiteral.IntervalValue interval =
-          (SqlIntervalLiteral.IntervalValue) literal.getValue();
+          literal.getValueAs(SqlIntervalLiteral.IntervalValue.class);
       BigDecimal val =
           interval.getIntervalQualifier().getStartUnit().multiplier;
       RexNode rexInterval = cx.convertExpression(literal);
diff --git a/core/src/test/java/org/apache/calcite/sql/parser/SqlParserTest.java b/core/src/test/java/org/apache/calcite/sql/parser/SqlParserTest.java
index 4fa3451..3c4bf76 100644
--- a/core/src/test/java/org/apache/calcite/sql/parser/SqlParserTest.java
+++ b/core/src/test/java/org/apache/calcite/sql/parser/SqlParserTest.java
@@ -7096,6 +7096,26 @@ public class SqlParserTest {
         .ok("INTERVAL '1:x:2' HOUR TO SECOND");
   }
 
+  @Test void testIntervalExpression() {
+    expr("interval 0 day").ok("INTERVAL 0 DAY");
+    expr("interval 0 days").ok("INTERVAL 0 DAY");
+    expr("interval -10 days").ok("INTERVAL (- 10) DAY");
+    expr("interval -10 days").ok("INTERVAL (- 10) DAY");
+    // parser requires parentheses for expressions other than numeric
+    // literal or identifier
+    expr("interval 1 ^+^ x.y days")
+        .fails("(?s)Encountered \"\\+\" at .*");
+    expr("interval (1 + x.y) days")
+        .ok("INTERVAL (1 + `X`.`Y`) DAY");
+    expr("interval -x second(3)")
+        .ok("INTERVAL (- `X`) SECOND(3)");
+    expr("interval -x.y second(3)")
+        .ok("INTERVAL (- `X`.`Y`) SECOND(3)");
+    expr("interval 1 day ^to^ hour")
+        .fails("(?s)Encountered \"to\" at .*");
+    expr("interval '1 1' day to hour").ok("INTERVAL '1 1' DAY TO HOUR");
+  }
+
   @Test void testIntervalOperators() {
     expr("-interval '1' day")
         .ok("(- INTERVAL '1' DAY)");
diff --git a/core/src/test/java/org/apache/calcite/sql/test/AbstractSqlTester.java b/core/src/test/java/org/apache/calcite/sql/test/AbstractSqlTester.java
index a858170..4a6bbf1 100644
--- a/core/src/test/java/org/apache/calcite/sql/test/AbstractSqlTester.java
+++ b/core/src/test/java/org/apache/calcite/sql/test/AbstractSqlTester.java
@@ -234,7 +234,7 @@ public abstract class AbstractSqlTester implements SqlTester, AutoCloseable {
     assertNotNull(node);
     SqlIntervalLiteral intervalLiteral = (SqlIntervalLiteral) node;
     SqlIntervalLiteral.IntervalValue interval =
-        (SqlIntervalLiteral.IntervalValue) intervalLiteral.getValue();
+        intervalLiteral.getValueAs(SqlIntervalLiteral.IntervalValue.class);
     long l =
         interval.getIntervalQualifier().isYearMonth()
             ? SqlParserUtil.intervalToMonths(interval)
diff --git a/core/src/test/java/org/apache/calcite/test/SqlToRelConverterTest.java b/core/src/test/java/org/apache/calcite/test/SqlToRelConverterTest.java
index e05ef07..4bb6ed0 100644
--- a/core/src/test/java/org/apache/calcite/test/SqlToRelConverterTest.java
+++ b/core/src/test/java/org/apache/calcite/test/SqlToRelConverterTest.java
@@ -119,6 +119,10 @@ class SqlToRelConverterTest extends SqlToRelTestBase {
     sql(sql).ok();
   }
 
+  @Test void testIntervalExpression() {
+    sql("select interval mgr hour as h from emp").ok();
+  }
+
   @Test void testAliasList() {
     final String sql = "select a + b from (\n"
         + "  select deptno, 1 as uno, name from dept\n"
diff --git a/core/src/test/java/org/apache/calcite/test/SqlValidatorTest.java b/core/src/test/java/org/apache/calcite/test/SqlValidatorTest.java
index 48c2958..12cdca6 100644
--- a/core/src/test/java/org/apache/calcite/test/SqlValidatorTest.java
+++ b/core/src/test/java/org/apache/calcite/test/SqlValidatorTest.java
@@ -92,7 +92,7 @@ import static org.junit.jupiter.api.Assumptions.assumeTrue;
  * {@link org.apache.calcite.sql.test.SqlTester}.
  */
 @LocaleEnUs
-class SqlValidatorTest extends SqlValidatorTestCase {
+public class SqlValidatorTest extends SqlValidatorTestCase {
   //~ Static fields/initializers ---------------------------------------------
 
   // CHECKSTYLE: IGNORE 1
@@ -1550,7 +1550,7 @@ class SqlValidatorTest extends SqlValidatorTestCase {
         .fails("(?s).*Function '.fn HAHAHA.' is not defined.*");
   }
 
-  @Test void testQuotedFunction() {
+  @Test public void testQuotedFunction() {
     if (false) {
       // REVIEW jvs 2-Feb-2005:  I am disabling this test because I
       // removed the corresponding support from the parser.  Where in the
@@ -1822,7 +1822,7 @@ class SqlValidatorTest extends SqlValidatorTestCase {
    * Similarly, any changes to tests here should be echoed appropriately to
    * each of the other 12 subTestIntervalXXXPositive() tests.
    */
-  public void subTestIntervalYearPositive() {
+  void subTestIntervalYearPositive() {
     // default precision
     expr("INTERVAL '1' YEAR")
         .columnType("INTERVAL YEAR NOT NULL");
@@ -1873,7 +1873,7 @@ class SqlValidatorTest extends SqlValidatorTestCase {
    * Similarly, any changes to tests here should be echoed appropriately to
    * each of the other 12 subTestIntervalXXXPositive() tests.
    */
-  public void subTestIntervalYearToMonthPositive() {
+  void subTestIntervalYearToMonthPositive() {
     // default precision
     expr("INTERVAL '1-2' YEAR TO MONTH")
         .columnType("INTERVAL YEAR TO MONTH NOT NULL");
@@ -1928,7 +1928,7 @@ class SqlValidatorTest extends SqlValidatorTestCase {
    * Similarly, any changes to tests here should be echoed appropriately to
    * each of the other 12 subTestIntervalXXXPositive() tests.
    */
-  public void subTestIntervalMonthPositive() {
+  void subTestIntervalMonthPositive() {
     // default precision
     expr("INTERVAL '1' MONTH")
         .columnType("INTERVAL MONTH NOT NULL");
@@ -1979,7 +1979,7 @@ class SqlValidatorTest extends SqlValidatorTestCase {
    * Similarly, any changes to tests here should be echoed appropriately to
    * each of the other 12 subTestIntervalXXXPositive() tests.
    */
-  public void subTestIntervalDayPositive() {
+  void subTestIntervalDayPositive() {
     // default precision
     expr("INTERVAL '1' DAY")
         .columnType("INTERVAL DAY NOT NULL");
@@ -2023,7 +2023,7 @@ class SqlValidatorTest extends SqlValidatorTestCase {
         .columnType("INTERVAL DAY NOT NULL");
   }
 
-  public void subTestIntervalDayToHourPositive() {
+  void subTestIntervalDayToHourPositive() {
     // default precision
     expr("INTERVAL '1 2' DAY TO HOUR")
         .columnType("INTERVAL DAY TO HOUR NOT NULL");
@@ -2078,7 +2078,7 @@ class SqlValidatorTest extends SqlValidatorTestCase {
    * Similarly, any changes to tests here should be echoed appropriately to
    * each of the other 12 subTestIntervalXXXPositive() tests.
    */
-  public void subTestIntervalDayToMinutePositive() {
+  void subTestIntervalDayToMinutePositive() {
     // default precision
     expr("INTERVAL '1 2:3' DAY TO MINUTE")
         .columnType("INTERVAL DAY TO MINUTE NOT NULL");
@@ -2133,7 +2133,7 @@ class SqlValidatorTest extends SqlValidatorTestCase {
    * Similarly, any changes to tests here should be echoed appropriately to
    * each of the other 12 subTestIntervalXXXPositive() tests.
    */
-  public void subTestIntervalDayToSecondPositive() {
+  void subTestIntervalDayToSecondPositive() {
     // default precision
     expr("INTERVAL '1 2:3:4' DAY TO SECOND")
         .columnType("INTERVAL DAY TO SECOND NOT NULL");
@@ -2202,7 +2202,7 @@ class SqlValidatorTest extends SqlValidatorTestCase {
    * Similarly, any changes to tests here should be echoed appropriately to
    * each of the other 12 subTestIntervalXXXPositive() tests.
    */
-  public void subTestIntervalHourPositive() {
+  void subTestIntervalHourPositive() {
     // default precision
     expr("INTERVAL '1' HOUR")
         .columnType("INTERVAL HOUR NOT NULL");
@@ -2253,7 +2253,7 @@ class SqlValidatorTest extends SqlValidatorTestCase {
    * Similarly, any changes to tests here should be echoed appropriately to
    * each of the other 12 subTestIntervalXXXPositive() tests.
    */
-  public void subTestIntervalHourToMinutePositive() {
+  void subTestIntervalHourToMinutePositive() {
     // default precision
     expr("INTERVAL '2:3' HOUR TO MINUTE")
         .columnType("INTERVAL HOUR TO MINUTE NOT NULL");
@@ -2308,7 +2308,7 @@ class SqlValidatorTest extends SqlValidatorTestCase {
    * Similarly, any changes to tests here should be echoed appropriately to
    * each of the other 12 subTestIntervalXXXPositive() tests.
    */
-  public void subTestIntervalHourToSecondPositive() {
+  void subTestIntervalHourToSecondPositive() {
     // default precision
     expr("INTERVAL '2:3:4' HOUR TO SECOND")
         .columnType("INTERVAL HOUR TO SECOND NOT NULL");
@@ -2377,7 +2377,7 @@ class SqlValidatorTest extends SqlValidatorTestCase {
    * Similarly, any changes to tests here should be echoed appropriately to
    * each of the other 12 subTestIntervalXXXPositive() tests.
    */
-  public void subTestIntervalMinutePositive() {
+  void subTestIntervalMinutePositive() {
     // default precision
     expr("INTERVAL '1' MINUTE")
         .columnType("INTERVAL MINUTE NOT NULL");
@@ -2428,7 +2428,7 @@ class SqlValidatorTest extends SqlValidatorTestCase {
    * Similarly, any changes to tests here should be echoed appropriately to
    * each of the other 12 subTestIntervalXXXPositive() tests.
    */
-  public void subTestIntervalMinuteToSecondPositive() {
+  void subTestIntervalMinuteToSecondPositive() {
     // default precision
     expr("INTERVAL '2:4' MINUTE TO SECOND")
         .columnType("INTERVAL MINUTE TO SECOND NOT NULL");
@@ -2497,7 +2497,7 @@ class SqlValidatorTest extends SqlValidatorTestCase {
    * Similarly, any changes to tests here should be echoed appropriately to
    * each of the other 12 subTestIntervalXXXPositive() tests.
    */
-  public void subTestIntervalSecondPositive() {
+  void subTestIntervalSecondPositive() {
     // default precision
     expr("INTERVAL '1' SECOND")
         .columnType("INTERVAL SECOND NOT NULL");
@@ -2558,7 +2558,7 @@ class SqlValidatorTest extends SqlValidatorTestCase {
    * Similarly, any changes to tests here should be echoed appropriately to
    * each of the other 12 subTestIntervalXXXNegative() tests.
    */
-  public void subTestIntervalYearNegative() {
+  void subTestIntervalYearNegative() {
     // Qualifier - field mismatches
     wholeExpr("INTERVAL '-' YEAR")
         .fails("Illegal interval literal format '-' for INTERVAL YEAR.*");
@@ -2595,14 +2595,14 @@ class SqlValidatorTest extends SqlValidatorTestCase {
             + "YEAR\\(10\\) field");
 
     // precision > maximum
-    expr("INTERVAL '1' YEAR(11^)^")
+    expr("INTERVAL '1' ^YEAR(11)^")
         .fails("Interval leading field precision '11' out of range for "
             + "INTERVAL YEAR\\(11\\)");
 
     // precision < minimum allowed)
     // note: parser will catch negative values, here we
     // just need to check for 0
-    expr("INTERVAL '0' YEAR(0^)^")
+    expr("INTERVAL '0' ^YEAR(0)^")
         .fails("Interval leading field precision '0' out of range for "
             + "INTERVAL YEAR\\(0\\)");
   }
@@ -2614,7 +2614,7 @@ class SqlValidatorTest extends SqlValidatorTestCase {
    * Similarly, any changes to tests here should be echoed appropriately to
    * each of the other 12 subTestIntervalXXXNegative() tests.
    */
-  public void subTestIntervalYearToMonthNegative() {
+  void subTestIntervalYearToMonthNegative() {
     // Qualifier - field mismatches
     wholeExpr("INTERVAL '-' YEAR TO MONTH")
         .fails("Illegal interval literal format '-' for INTERVAL YEAR TO MONTH");
@@ -2660,14 +2660,14 @@ class SqlValidatorTest extends SqlValidatorTestCase {
         .fails("Illegal interval literal format '1-12' for INTERVAL YEAR TO MONTH.*");
 
     // precision > maximum
-    expr("INTERVAL '1-1' YEAR(11) TO ^MONTH^")
+    expr("INTERVAL '1-1' ^YEAR(11) TO MONTH^")
         .fails("Interval leading field precision '11' out of range for "
             + "INTERVAL YEAR\\(11\\) TO MONTH");
 
     // precision < minimum allowed)
     // note: parser will catch negative values, here we
     // just need to check for 0
-    expr("INTERVAL '0-0' YEAR(0) TO ^MONTH^")
+    expr("INTERVAL '0-0' ^YEAR(0) TO MONTH^")
         .fails("Interval leading field precision '0' out of range for "
             + "INTERVAL YEAR\\(0\\) TO MONTH");
   }
@@ -2679,7 +2679,7 @@ class SqlValidatorTest extends SqlValidatorTestCase {
    * Similarly, any changes to tests here should be echoed appropriately to
    * each of the other 12 subTestIntervalXXXNegative() tests.
    */
-  public void subTestIntervalMonthNegative() {
+  void subTestIntervalMonthNegative() {
     // Qualifier - field mismatches
     wholeExpr("INTERVAL '-' MONTH")
         .fails("Illegal interval literal format '-' for INTERVAL MONTH.*");
@@ -2714,14 +2714,14 @@ class SqlValidatorTest extends SqlValidatorTestCase {
         .fails("Interval field value -2,147,483,648 exceeds precision of MONTH\\(10\\) field.*");
 
     // precision > maximum
-    expr("INTERVAL '1' MONTH(11^)^")
+    expr("INTERVAL '1' ^MONTH(11)^")
         .fails("Interval leading field precision '11' out of range for "
             + "INTERVAL MONTH\\(11\\)");
 
     // precision < minimum allowed)
     // note: parser will catch negative values, here we
     // just need to check for 0
-    expr("INTERVAL '0' MONTH(0^)^")
+    expr("INTERVAL '0' ^MONTH(0)^")
         .fails("Interval leading field precision '0' out of range for "
             + "INTERVAL MONTH\\(0\\)");
   }
@@ -2733,7 +2733,7 @@ class SqlValidatorTest extends SqlValidatorTestCase {
    * Similarly, any changes to tests here should be echoed appropriately to
    * each of the other 12 subTestIntervalXXXNegative() tests.
    */
-  public void subTestIntervalDayNegative() {
+  void subTestIntervalDayNegative() {
     // Qualifier - field mismatches
     wholeExpr("INTERVAL '-' DAY")
         .fails("Illegal interval literal format '-' for INTERVAL DAY.*");
@@ -2772,14 +2772,14 @@ class SqlValidatorTest extends SqlValidatorTestCase {
             + "DAY\\(10\\) field.*");
 
     // precision > maximum
-    expr("INTERVAL '1' DAY(11^)^")
+    expr("INTERVAL '1' ^DAY(11)^")
         .fails("Interval leading field precision '11' out of range for "
             + "INTERVAL DAY\\(11\\)");
 
     // precision < minimum allowed)
     // note: parser will catch negative values, here we
     // just need to check for 0
-    expr("INTERVAL '0' DAY(0^)^")
+    expr("INTERVAL '0' ^DAY(0)^")
         .fails("Interval leading field precision '0' out of range for "
             + "INTERVAL DAY\\(0\\)");
   }
@@ -2791,7 +2791,7 @@ class SqlValidatorTest extends SqlValidatorTestCase {
    * Similarly, any changes to tests here should be echoed appropriately to
    * each of the other 12 subTestIntervalXXXNegative() tests.
    */
-  public void subTestIntervalDayToHourNegative() {
+  void subTestIntervalDayToHourNegative() {
     // Qualifier - field mismatches
     wholeExpr("INTERVAL '-' DAY TO HOUR")
         .fails("Illegal interval literal format '-' for INTERVAL DAY TO HOUR");
@@ -2838,14 +2838,14 @@ class SqlValidatorTest extends SqlValidatorTestCase {
         .fails("Illegal interval literal format '1 24' for INTERVAL DAY TO HOUR.*");
 
     // precision > maximum
-    expr("INTERVAL '1 1' DAY(11) TO ^HOUR^")
+    expr("INTERVAL '1 1' ^DAY(11) TO HOUR^")
         .fails("Interval leading field precision '11' out of range for "
             + "INTERVAL DAY\\(11\\) TO HOUR");
 
     // precision < minimum allowed)
     // note: parser will catch negative values, here we
     // just need to check for 0
-    expr("INTERVAL '0 0' DAY(0) TO ^HOUR^")
+    expr("INTERVAL '0 0' ^DAY(0) TO HOUR^")
         .fails("Interval leading field precision '0' out of range for INTERVAL DAY\\(0\\) TO HOUR");
   }
 
@@ -2856,7 +2856,7 @@ class SqlValidatorTest extends SqlValidatorTestCase {
    * Similarly, any changes to tests here should be echoed appropriately to
    * each of the other 12 subTestIntervalXXXNegative() tests.
    */
-  public void subTestIntervalDayToMinuteNegative() {
+  void subTestIntervalDayToMinuteNegative() {
     // Qualifier - field mismatches
     wholeExpr("INTERVAL ' :' DAY TO MINUTE")
         .fails("Illegal interval literal format ' :' for INTERVAL DAY TO MINUTE");
@@ -2920,14 +2920,14 @@ class SqlValidatorTest extends SqlValidatorTestCase {
         .fails("Illegal interval literal format '1 1:60' for INTERVAL DAY TO MINUTE.*");
 
     // precision > maximum
-    expr("INTERVAL '1 1:1' DAY(11) TO ^MINUTE^")
+    expr("INTERVAL '1 1:1' ^DAY(11) TO MINUTE^")
         .fails("Interval leading field precision '11' out of range for "
             + "INTERVAL DAY\\(11\\) TO MINUTE");
 
     // precision < minimum allowed)
     // note: parser will catch negative values, here we
     // just need to check for 0
-    expr("INTERVAL '0 0' DAY(0) TO ^MINUTE^")
+    expr("INTERVAL '0 0' ^DAY(0) TO MINUTE^")
         .fails("Interval leading field precision '0' out of range for "
             + "INTERVAL DAY\\(0\\) TO MINUTE");
   }
@@ -2939,7 +2939,7 @@ class SqlValidatorTest extends SqlValidatorTestCase {
    * Similarly, any changes to tests here should be echoed appropriately to
    * each of the other 12 subTestIntervalXXXNegative() tests.
    */
-  public void subTestIntervalDayToSecondNegative() {
+  void subTestIntervalDayToSecondNegative() {
     // Qualifier - field mismatches
     wholeExpr("INTERVAL ' ::' DAY TO SECOND")
         .fails("Illegal interval literal format ' ::' for INTERVAL DAY TO SECOND");
@@ -3041,20 +3041,20 @@ class SqlValidatorTest extends SqlValidatorTestCase {
             + "INTERVAL DAY TO SECOND\\(3\\).*");
 
     // precision > maximum
-    expr("INTERVAL '1 1' DAY(11) TO ^SECOND^")
+    expr("INTERVAL '1 1' ^DAY(11) TO SECOND^")
         .fails("Interval leading field precision '11' out of range for "
             + "INTERVAL DAY\\(11\\) TO SECOND");
-    expr("INTERVAL '1 1' DAY TO SECOND(10^)^")
+    expr("INTERVAL '1 1' ^DAY TO SECOND(10)^")
         .fails("Interval fractional second precision '10' out of range for "
             + "INTERVAL DAY TO SECOND\\(10\\)");
 
     // precision < minimum allowed)
     // note: parser will catch negative values, here we
     // just need to check for 0
-    expr("INTERVAL '0 0:0:0' DAY(0) TO ^SECOND^")
+    expr("INTERVAL '0 0:0:0' ^DAY(0) TO SECOND^")
         .fails("Interval leading field precision '0' out of range for "
             + "INTERVAL DAY\\(0\\) TO SECOND");
-    expr("INTERVAL '0 0:0:0' DAY TO SECOND(0^)^")
+    expr("INTERVAL '0 0:0:0' ^DAY TO SECOND(0)^")
         .fails("Interval fractional second precision '0' out of range for "
             + "INTERVAL DAY TO SECOND\\(0\\)");
   }
@@ -3066,7 +3066,7 @@ class SqlValidatorTest extends SqlValidatorTestCase {
    * Similarly, any changes to tests here should be echoed appropriately to
    * each of the other 12 subTestIntervalXXXNegative() tests.
    */
-  public void subTestIntervalHourNegative() {
+  void subTestIntervalHourNegative() {
     // Qualifier - field mismatches
     wholeExpr("INTERVAL '-' HOUR")
         .fails("Illegal interval literal format '-' for INTERVAL HOUR.*");
@@ -3110,14 +3110,14 @@ class SqlValidatorTest extends SqlValidatorTestCase {
             + "HOUR\\(10\\) field.*");
 
     // precision > maximum
-    expr("INTERVAL '1' HOUR(11^)^")
+    expr("INTERVAL '1' ^HOUR(11)^")
         .fails("Interval leading field precision '11' out of range for "
             + "INTERVAL HOUR\\(11\\)");
 
     // precision < minimum allowed)
     // note: parser will catch negative values, here we
     // just need to check for 0
-    expr("INTERVAL '0' HOUR(0^)^")
+    expr("INTERVAL '0' ^HOUR(0)^")
         .fails("Interval leading field precision '0' out of range for "
             + "INTERVAL HOUR\\(0\\)");
   }
@@ -3129,7 +3129,7 @@ class SqlValidatorTest extends SqlValidatorTestCase {
    * Similarly, any changes to tests here should be echoed appropriately to
    * each of the other 12 subTestIntervalXXXNegative() tests.
    */
-  public void subTestIntervalHourToMinuteNegative() {
+  void subTestIntervalHourToMinuteNegative() {
     // Qualifier - field mismatches
     wholeExpr("INTERVAL ':' HOUR TO MINUTE")
         .fails("Illegal interval literal format ':' for INTERVAL HOUR TO MINUTE");
@@ -3175,14 +3175,14 @@ class SqlValidatorTest extends SqlValidatorTestCase {
         .fails("Illegal interval literal format '1:60' for INTERVAL HOUR TO MINUTE.*");
 
     // precision > maximum
-    expr("INTERVAL '1:1' HOUR(11) TO ^MINUTE^")
+    expr("INTERVAL '1:1' ^HOUR(11) TO MINUTE^")
         .fails("Interval leading field precision '11' out of range for "
             + "INTERVAL HOUR\\(11\\) TO MINUTE");
 
     // precision < minimum allowed)
     // note: parser will catch negative values, here we
     // just need to check for 0
-    expr("INTERVAL '0:0' HOUR(0) TO ^MINUTE^")
+    expr("INTERVAL '0:0' ^HOUR(0) TO MINUTE^")
         .fails("Interval leading field precision '0' out of range for "
             + "INTERVAL HOUR\\(0\\) TO MINUTE");
   }
@@ -3194,7 +3194,7 @@ class SqlValidatorTest extends SqlValidatorTestCase {
    * Similarly, any changes to tests here should be echoed appropriately to
    * each of the other 12 subTestIntervalXXXNegative() tests.
    */
-  public void subTestIntervalHourToSecondNegative() {
+  void subTestIntervalHourToSecondNegative() {
     // Qualifier - field mismatches
     wholeExpr("INTERVAL '::' HOUR TO SECOND")
         .fails("Illegal interval literal format '::' for INTERVAL HOUR TO SECOND");
@@ -3270,20 +3270,20 @@ class SqlValidatorTest extends SqlValidatorTestCase {
             + "INTERVAL HOUR TO SECOND\\(3\\).*");
 
     // precision > maximum
-    expr("INTERVAL '1:1:1' HOUR(11) TO ^SECOND^")
+    expr("INTERVAL '1:1:1' ^HOUR(11) TO SECOND^")
         .fails("Interval leading field precision '11' out of range for "
             + "INTERVAL HOUR\\(11\\) TO SECOND");
-    expr("INTERVAL '1:1:1' HOUR TO SECOND(10^)^")
+    expr("INTERVAL '1:1:1' ^HOUR TO SECOND(10)^")
         .fails("Interval fractional second precision '10' out of range for "
             + "INTERVAL HOUR TO SECOND\\(10\\)");
 
     // precision < minimum allowed)
     // note: parser will catch negative values, here we
     // just need to check for 0
-    expr("INTERVAL '0:0:0' HOUR(0) TO ^SECOND^")
+    expr("INTERVAL '0:0:0' ^HOUR(0) TO SECOND^")
         .fails("Interval leading field precision '0' out of range for "
             + "INTERVAL HOUR\\(0\\) TO SECOND");
-    expr("INTERVAL '0:0:0' HOUR TO SECOND(0^)^")
+    expr("INTERVAL '0:0:0' ^HOUR TO SECOND(0)^")
         .fails("Interval fractional second precision '0' out of range for "
             + "INTERVAL HOUR TO SECOND\\(0\\)");
   }
@@ -3295,7 +3295,7 @@ class SqlValidatorTest extends SqlValidatorTestCase {
    * Similarly, any changes to tests here should be echoed appropriately to
    * each of the other 12 subTestIntervalXXXNegative() tests.
    */
-  public void subTestIntervalMinuteNegative() {
+  void subTestIntervalMinuteNegative() {
     // Qualifier - field mismatches
     wholeExpr("INTERVAL '-' MINUTE")
         .fails("Illegal interval literal format '-' for INTERVAL MINUTE.*");
@@ -3332,14 +3332,14 @@ class SqlValidatorTest extends SqlValidatorTestCase {
         .fails("Interval field value -2,147,483,648 exceeds precision of MINUTE\\(10\\) field.*");
 
     // precision > maximum
-    expr("INTERVAL '1' MINUTE(11^)^")
+    expr("INTERVAL '1' ^MINUTE(11)^")
         .fails("Interval leading field precision '11' out of range for "
             + "INTERVAL MINUTE\\(11\\)");
 
     // precision < minimum allowed)
     // note: parser will catch negative values, here we
     // just need to check for 0
-    expr("INTERVAL '0' MINUTE(0^)^")
+    expr("INTERVAL '0' ^MINUTE(0)^")
         .fails("Interval leading field precision '0' out of range for "
             + "INTERVAL MINUTE\\(0\\)");
   }
@@ -3351,7 +3351,7 @@ class SqlValidatorTest extends SqlValidatorTestCase {
    * Similarly, any changes to tests here should be echoed appropriately to
    * each of the other 12 subTestIntervalXXXNegative() tests.
    */
-  public void subTestIntervalMinuteToSecondNegative() {
+  void subTestIntervalMinuteToSecondNegative() {
     // Qualifier - field mismatches
     wholeExpr("INTERVAL ':' MINUTE TO SECOND")
         .fails("Illegal interval literal format ':' for INTERVAL MINUTE TO SECOND");
@@ -3414,20 +3414,20 @@ class SqlValidatorTest extends SqlValidatorTestCase {
             + " INTERVAL MINUTE TO SECOND\\(3\\).*");
 
     // precision > maximum
-    expr("INTERVAL '1:1' MINUTE(11) TO ^SECOND^")
+    expr("INTERVAL '1:1' ^MINUTE(11) TO SECOND^")
         .fails("Interval leading field precision '11' out of range for"
             + " INTERVAL MINUTE\\(11\\) TO SECOND");
-    expr("INTERVAL '1:1' MINUTE TO SECOND(10^)^")
+    expr("INTERVAL '1:1' ^MINUTE TO SECOND(10)^")
         .fails("Interval fractional second precision '10' out of range for"
             + " INTERVAL MINUTE TO SECOND\\(10\\)");
 
     // precision < minimum allowed)
     // note: parser will catch negative values, here we
     // just need to check for 0
-    expr("INTERVAL '0:0' MINUTE(0) TO ^SECOND^")
+    expr("INTERVAL '0:0' ^MINUTE(0) TO SECOND^")
         .fails("Interval leading field precision '0' out of range for"
             + " INTERVAL MINUTE\\(0\\) TO SECOND");
-    expr("INTERVAL '0:0' MINUTE TO SECOND(0^)^")
+    expr("INTERVAL '0:0' ^MINUTE TO SECOND(0)^")
         .fails("Interval fractional second precision '0' out of range for"
             + " INTERVAL MINUTE TO SECOND\\(0\\)");
   }
@@ -3439,7 +3439,7 @@ class SqlValidatorTest extends SqlValidatorTestCase {
    * Similarly, any changes to tests here should be echoed appropriately to
    * each of the other 12 subTestIntervalXXXNegative() tests.
    */
-  public void subTestIntervalSecondNegative() {
+  void subTestIntervalSecondNegative() {
     // Qualifier - field mismatches
     wholeExpr("INTERVAL ':' SECOND")
         .fails("Illegal interval literal format ':' for INTERVAL SECOND.*");
@@ -3491,20 +3491,20 @@ class SqlValidatorTest extends SqlValidatorTestCase {
             + " INTERVAL SECOND\\(2, 9\\).*");
 
     // precision > maximum
-    expr("INTERVAL '1' SECOND(11^)^")
+    expr("INTERVAL '1' ^SECOND(11)^")
         .fails("Interval leading field precision '11' out of range for"
             + " INTERVAL SECOND\\(11\\)");
-    expr("INTERVAL '1.1' SECOND(1, 10^)^")
+    expr("INTERVAL '1.1' ^SECOND(1, 10)^")
         .fails("Interval fractional second precision '10' out of range for"
             + " INTERVAL SECOND\\(1, 10\\)");
 
     // precision < minimum allowed)
     // note: parser will catch negative values, here we
     // just need to check for 0
-    expr("INTERVAL '0' SECOND(0^)^")
+    expr("INTERVAL '0' ^SECOND(0)^")
         .fails("Interval leading field precision '0' out of range for"
             + " INTERVAL SECOND\\(0\\)");
-    expr("INTERVAL '0' SECOND(1, 0^)^")
+    expr("INTERVAL '0' ^SECOND(1, 0)^")
         .fails("Interval fractional second precision '0' out of range for"
             + " INTERVAL SECOND\\(1, 0\\)");
   }
@@ -3583,6 +3583,31 @@ class SqlValidatorTest extends SqlValidatorTestCase {
         .columnType("INTERVAL MONTH(3) NOT NULL");
   }
 
+  @Test void testIntervalExpression() {
+    expr("interval 1 hour").columnType("INTERVAL HOUR NOT NULL");
+    expr("interval (2 + 3) month").columnType("INTERVAL MONTH NOT NULL");
+    expr("interval (cast(null as integer)) year").columnType("INTERVAL YEAR");
+    expr("interval (cast(null as integer)) year(2)")
+        .columnType("INTERVAL YEAR(2)");
+    expr("interval (date '1970-01-01') hour").withWhole(true)
+        .fails("Cannot apply 'INTERVAL' to arguments of type "
+            + "'INTERVAL <DATE> <INTERVAL HOUR>'\\. Supported form\\(s\\): "
+            + "'INTERVAL <NUMERIC> <DATETIME_INTERVAL>'");
+    expr("interval (nullif(true, true)) hour").withWhole(true)
+        .fails("Cannot apply 'INTERVAL' to arguments of type "
+            + "'INTERVAL <BOOLEAN> <INTERVAL HOUR>'\\. Supported form\\(s\\): "
+            + "'INTERVAL <NUMERIC> <DATETIME_INTERVAL>'");
+    expr("interval (interval '1' day) hour").withWhole(true)
+        .fails("Cannot apply 'INTERVAL' to arguments of type "
+            + "'INTERVAL <INTERVAL DAY> <INTERVAL HOUR>'\\. "
+            + "Supported form\\(s\\): "
+            + "'INTERVAL <NUMERIC> <DATETIME_INTERVAL>'");
+    sql("select interval empno hour as h from emp")
+        .columnType("INTERVAL HOUR NOT NULL");
+    sql("select interval emp.mgr hour as h from emp")
+        .columnType("INTERVAL HOUR");
+  }
+
   @Test void testIntervalOperators() {
     expr("interval '1' hour + TIME '8:8:8'")
         .columnType("TIME(0) NOT NULL");
diff --git a/core/src/test/resources/org/apache/calcite/test/SqlToRelConverterTest.xml b/core/src/test/resources/org/apache/calcite/test/SqlToRelConverterTest.xml
index 53ef390..178e2e6 100644
--- a/core/src/test/resources/org/apache/calcite/test/SqlToRelConverterTest.xml
+++ b/core/src/test/resources/org/apache/calcite/test/SqlToRelConverterTest.xml
@@ -2339,6 +2339,17 @@ LogicalProject(EMPNO=[$0], ENAME=[$1], JOB=[$2], MGR=[$3], HIREDATE=[$4], SAL=[$
 ]]>
         </Resource>
     </TestCase>
+    <TestCase name="testIntervalExpression">
+        <Resource name="sql">
+            <![CDATA[select interval mgr hour as h from emp]]>
+        </Resource>
+        <Resource name="plan">
+            <![CDATA[
+LogicalProject(H=[*($3, 3600000:INTERVAL HOUR)])
+  LogicalTableScan(table=[[CATALOG, SALES, EMP]])
+]]>
+        </Resource>
+    </TestCase>
     <TestCase name="testLateralDecorrelate">
         <Resource name="sql">
             <![CDATA[select * from emp,
@@ -3743,7 +3754,6 @@ LogicalProject(DEPTNO=[$7])
 ]]>
         </Resource>
     </TestCase>
-
     <TestCase name="testSubQueryAggregateFunctionFollowedBySimpleOperation">
         <Resource name="sql">
             <![CDATA[select deptno
diff --git a/core/src/test/resources/sql/misc.iq b/core/src/test/resources/sql/misc.iq
index d30fd56..0980e04 100644
--- a/core/src/test/resources/sql/misc.iq
+++ b/core/src/test/resources/sql/misc.iq
@@ -1394,6 +1394,33 @@ from "scott".dept;
 
 !ok
 
+# [CALCITE-4091] Interval expressions
+select empno, mgr, date '1970-01-01' + interval empno day as d,
+  timestamp '1970-01-01 00:00:00' + interval (mgr / 100) minute as ts
+from "scott".emp
+order by empno;
++-------+------+------------+---------------------+
+| EMPNO | MGR  | D          | TS                  |
++-------+------+------------+---------------------+
+|  7369 | 7902 | 1990-03-06 | 1970-01-01 01:19:00 |
+|  7499 | 7698 | 1990-07-14 | 1970-01-01 01:16:00 |
+|  7521 | 7698 | 1990-08-05 | 1970-01-01 01:16:00 |
+|  7566 | 7839 | 1990-09-19 | 1970-01-01 01:18:00 |
+|  7654 | 7698 | 1990-12-16 | 1970-01-01 01:16:00 |
+|  7698 | 7839 | 1991-01-29 | 1970-01-01 01:18:00 |
+|  7782 | 7839 | 1991-04-23 | 1970-01-01 01:18:00 |
+|  7788 | 7566 | 1991-04-29 | 1970-01-01 01:15:00 |
+|  7839 |      | 1991-06-19 |                     |
+|  7844 | 7698 | 1991-06-24 | 1970-01-01 01:16:00 |
+|  7876 | 7788 | 1991-07-26 | 1970-01-01 01:17:00 |
+|  7900 | 7698 | 1991-08-19 | 1970-01-01 01:16:00 |
+|  7902 | 7566 | 1991-08-21 | 1970-01-01 01:15:00 |
+|  7934 | 7782 | 1991-09-22 | 1970-01-01 01:17:00 |
++-------+------+------------+---------------------+
+(14 rows)
+
+!ok
+
 # [CALCITE-1486] Invalid "Invalid literal" error for complex expression
 select 8388608/(60+27.39);
 +-------------------+


[calcite] 05/09: In SqlReturnType, add methods orElse and andThen

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

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

commit e17a5c9f640be399b1afeaccfbc660482e5de187
Author: Julian Hyde <jh...@apache.org>
AuthorDate: Mon Jul 20 15:57:09 2020 -0700

    In SqlReturnType, add methods orElse and andThen
    
    Given `SqlReturnType rule`, `rule.andThen(transform)` is shorthand for
    `ReturnTypes.cascade(rule, transform)` and `rule.orElse(rule2)` is
    shorthand for `ReturnTypes.chain(rule, rule2)`.
    
    Refactor existing code to use these new methods. There are no
    changes in functionality.
---
 .../calcite/sql/fun/SqlCoalesceFunction.java       |   3 +-
 .../calcite/sql/fun/SqlJsonDepthFunction.java      |   3 +-
 .../calcite/sql/fun/SqlJsonExistsFunction.java     |   5 +-
 .../calcite/sql/fun/SqlJsonKeysFunction.java       |  10 +-
 .../calcite/sql/fun/SqlJsonLengthFunction.java     |   3 +-
 .../calcite/sql/fun/SqlJsonPrettyFunction.java     |   9 +-
 .../calcite/sql/fun/SqlJsonQueryFunction.java      |   7 +-
 .../calcite/sql/fun/SqlJsonRemoveFunction.java     |  13 +--
 .../sql/fun/SqlJsonStorageSizeFunction.java        |  10 +-
 .../calcite/sql/fun/SqlJsonTypeFunction.java       |  12 +--
 .../sql/fun/SqlJsonValueExpressionOperator.java    |   5 +-
 .../calcite/sql/fun/SqlLeadLagAggFunction.java     |  25 +++--
 .../calcite/sql/fun/SqlLibraryOperators.java       | 105 ++++++++++-----------
 .../calcite/sql/fun/SqlRegexpReplaceFunction.java  |  14 ++-
 .../apache/calcite/sql/fun/SqlTrimFunction.java    |   8 +-
 .../org/apache/calcite/sql/type/ReturnTypes.java   |  99 +++++++++++--------
 .../calcite/sql/type/SqlReturnTypeInference.java   |  13 +++
 .../sql/type/SqlReturnTypeInferenceChain.java      |   7 +-
 .../calcite/sql/type/SqlTypeTransformCascade.java  |   8 +-
 19 files changed, 181 insertions(+), 178 deletions(-)

diff --git a/core/src/main/java/org/apache/calcite/sql/fun/SqlCoalesceFunction.java b/core/src/main/java/org/apache/calcite/sql/fun/SqlCoalesceFunction.java
index 65c069e..79f058e 100644
--- a/core/src/main/java/org/apache/calcite/sql/fun/SqlCoalesceFunction.java
+++ b/core/src/main/java/org/apache/calcite/sql/fun/SqlCoalesceFunction.java
@@ -45,8 +45,7 @@ public class SqlCoalesceFunction extends SqlFunction {
     // strategies are used.
     super("COALESCE",
         SqlKind.COALESCE,
-        ReturnTypes.cascade(ReturnTypes.LEAST_RESTRICTIVE,
-            SqlTypeTransforms.LEAST_NULLABLE),
+        ReturnTypes.LEAST_RESTRICTIVE.andThen(SqlTypeTransforms.LEAST_NULLABLE),
         null,
         OperandTypes.SAME_VARIADIC,
         SqlFunctionCategory.SYSTEM);
diff --git a/core/src/main/java/org/apache/calcite/sql/fun/SqlJsonDepthFunction.java b/core/src/main/java/org/apache/calcite/sql/fun/SqlJsonDepthFunction.java
index 7f16ac8..94dbb7b 100644
--- a/core/src/main/java/org/apache/calcite/sql/fun/SqlJsonDepthFunction.java
+++ b/core/src/main/java/org/apache/calcite/sql/fun/SqlJsonDepthFunction.java
@@ -38,8 +38,7 @@ public class SqlJsonDepthFunction extends SqlFunction {
   public SqlJsonDepthFunction() {
     super("JSON_DEPTH",
         SqlKind.OTHER_FUNCTION,
-        ReturnTypes.cascade(ReturnTypes.INTEGER,
-            SqlTypeTransforms.FORCE_NULLABLE),
+        ReturnTypes.INTEGER.andThen(SqlTypeTransforms.FORCE_NULLABLE),
         null,
         OperandTypes.ANY,
         SqlFunctionCategory.SYSTEM);
diff --git a/core/src/main/java/org/apache/calcite/sql/fun/SqlJsonExistsFunction.java b/core/src/main/java/org/apache/calcite/sql/fun/SqlJsonExistsFunction.java
index b91b334..f547b45 100644
--- a/core/src/main/java/org/apache/calcite/sql/fun/SqlJsonExistsFunction.java
+++ b/core/src/main/java/org/apache/calcite/sql/fun/SqlJsonExistsFunction.java
@@ -32,10 +32,11 @@ import org.apache.calcite.sql.type.SqlTypeTransforms;
 public class SqlJsonExistsFunction extends SqlFunction {
   public SqlJsonExistsFunction() {
     super("JSON_EXISTS", SqlKind.OTHER_FUNCTION,
-        ReturnTypes.cascade(ReturnTypes.BOOLEAN, SqlTypeTransforms.FORCE_NULLABLE), null,
+        ReturnTypes.BOOLEAN.andThen(SqlTypeTransforms.FORCE_NULLABLE), null,
         OperandTypes.or(
             OperandTypes.family(SqlTypeFamily.ANY, SqlTypeFamily.CHARACTER),
-            OperandTypes.family(SqlTypeFamily.ANY, SqlTypeFamily.CHARACTER, SqlTypeFamily.ANY)),
+            OperandTypes.family(SqlTypeFamily.ANY, SqlTypeFamily.CHARACTER,
+                SqlTypeFamily.ANY)),
         SqlFunctionCategory.SYSTEM);
   }
 
diff --git a/core/src/main/java/org/apache/calcite/sql/fun/SqlJsonKeysFunction.java b/core/src/main/java/org/apache/calcite/sql/fun/SqlJsonKeysFunction.java
index 5251d3e..f71ae0e 100644
--- a/core/src/main/java/org/apache/calcite/sql/fun/SqlJsonKeysFunction.java
+++ b/core/src/main/java/org/apache/calcite/sql/fun/SqlJsonKeysFunction.java
@@ -30,10 +30,10 @@ import org.apache.calcite.sql.type.SqlTypeTransforms;
 public class SqlJsonKeysFunction extends SqlFunction {
   public SqlJsonKeysFunction() {
     super("JSON_KEYS", SqlKind.OTHER_FUNCTION,
-          ReturnTypes.cascade(ReturnTypes.VARCHAR_2000, SqlTypeTransforms.FORCE_NULLABLE),
-          null,
-          OperandTypes.or(OperandTypes.ANY,
-              OperandTypes.family(SqlTypeFamily.ANY, SqlTypeFamily.CHARACTER)),
-          SqlFunctionCategory.SYSTEM);
+        ReturnTypes.VARCHAR_2000.andThen(SqlTypeTransforms.FORCE_NULLABLE),
+        null,
+        OperandTypes.or(OperandTypes.ANY,
+            OperandTypes.family(SqlTypeFamily.ANY, SqlTypeFamily.CHARACTER)),
+        SqlFunctionCategory.SYSTEM);
   }
 }
diff --git a/core/src/main/java/org/apache/calcite/sql/fun/SqlJsonLengthFunction.java b/core/src/main/java/org/apache/calcite/sql/fun/SqlJsonLengthFunction.java
index 552cc1c..27f33ed 100644
--- a/core/src/main/java/org/apache/calcite/sql/fun/SqlJsonLengthFunction.java
+++ b/core/src/main/java/org/apache/calcite/sql/fun/SqlJsonLengthFunction.java
@@ -30,8 +30,7 @@ import org.apache.calcite.sql.type.SqlTypeTransforms;
 public class SqlJsonLengthFunction extends SqlFunction {
   public SqlJsonLengthFunction() {
     super("JSON_LENGTH", SqlKind.OTHER_FUNCTION,
-        ReturnTypes.cascade(ReturnTypes.INTEGER,
-            SqlTypeTransforms.FORCE_NULLABLE),
+        ReturnTypes.INTEGER.andThen(SqlTypeTransforms.FORCE_NULLABLE),
         null,
         OperandTypes.or(OperandTypes.ANY,
             OperandTypes.family(SqlTypeFamily.ANY, SqlTypeFamily.CHARACTER)),
diff --git a/core/src/main/java/org/apache/calcite/sql/fun/SqlJsonPrettyFunction.java b/core/src/main/java/org/apache/calcite/sql/fun/SqlJsonPrettyFunction.java
index b5b1b3d..23d77b6 100644
--- a/core/src/main/java/org/apache/calcite/sql/fun/SqlJsonPrettyFunction.java
+++ b/core/src/main/java/org/apache/calcite/sql/fun/SqlJsonPrettyFunction.java
@@ -37,12 +37,9 @@ import org.apache.calcite.sql.validate.SqlValidator;
 public class SqlJsonPrettyFunction extends SqlFunction {
 
   public SqlJsonPrettyFunction() {
-    super("JSON_PRETTY",
-        SqlKind.OTHER_FUNCTION,
-        ReturnTypes.cascade(ReturnTypes.VARCHAR_2000, SqlTypeTransforms.FORCE_NULLABLE),
-        null,
-        OperandTypes.ANY,
-        SqlFunctionCategory.SYSTEM);
+    super("JSON_PRETTY", SqlKind.OTHER_FUNCTION,
+        ReturnTypes.VARCHAR_2000.andThen(SqlTypeTransforms.FORCE_NULLABLE),
+        null, OperandTypes.ANY, SqlFunctionCategory.SYSTEM);
   }
 
   @Override public SqlOperandCountRange getOperandCountRange() {
diff --git a/core/src/main/java/org/apache/calcite/sql/fun/SqlJsonQueryFunction.java b/core/src/main/java/org/apache/calcite/sql/fun/SqlJsonQueryFunction.java
index 52dd020..83a210e 100644
--- a/core/src/main/java/org/apache/calcite/sql/fun/SqlJsonQueryFunction.java
+++ b/core/src/main/java/org/apache/calcite/sql/fun/SqlJsonQueryFunction.java
@@ -37,11 +37,10 @@ import org.apache.calcite.sql.type.SqlTypeTransforms;
 public class SqlJsonQueryFunction extends SqlFunction {
   public SqlJsonQueryFunction() {
     super("JSON_QUERY", SqlKind.OTHER_FUNCTION,
-        ReturnTypes.cascade(ReturnTypes.VARCHAR_2000,
-            SqlTypeTransforms.FORCE_NULLABLE),
+        ReturnTypes.VARCHAR_2000.andThen(SqlTypeTransforms.FORCE_NULLABLE),
         null,
-        OperandTypes.family(SqlTypeFamily.ANY,
-            SqlTypeFamily.CHARACTER, SqlTypeFamily.ANY, SqlTypeFamily.ANY, SqlTypeFamily.ANY),
+        OperandTypes.family(SqlTypeFamily.ANY, SqlTypeFamily.CHARACTER,
+            SqlTypeFamily.ANY, SqlTypeFamily.ANY, SqlTypeFamily.ANY),
         SqlFunctionCategory.SYSTEM);
   }
 
diff --git a/core/src/main/java/org/apache/calcite/sql/fun/SqlJsonRemoveFunction.java b/core/src/main/java/org/apache/calcite/sql/fun/SqlJsonRemoveFunction.java
index 59cd4b7..6860f20 100644
--- a/core/src/main/java/org/apache/calcite/sql/fun/SqlJsonRemoveFunction.java
+++ b/core/src/main/java/org/apache/calcite/sql/fun/SqlJsonRemoveFunction.java
@@ -35,20 +35,17 @@ import java.util.Locale;
 public class SqlJsonRemoveFunction extends SqlFunction {
 
   public SqlJsonRemoveFunction() {
-    super("JSON_REMOVE",
-        SqlKind.OTHER_FUNCTION,
-        ReturnTypes.cascade(ReturnTypes.VARCHAR_2000,
-            SqlTypeTransforms.FORCE_NULLABLE),
-        null,
-        null,
-        SqlFunctionCategory.SYSTEM);
+    super("JSON_REMOVE", SqlKind.OTHER_FUNCTION,
+        ReturnTypes.VARCHAR_2000.andThen(SqlTypeTransforms.FORCE_NULLABLE),
+        null, null, SqlFunctionCategory.SYSTEM);
   }
 
   @Override public SqlOperandCountRange getOperandCountRange() {
     return SqlOperandCountRanges.from(2);
   }
 
-  @Override public boolean checkOperandTypes(SqlCallBinding callBinding, boolean throwOnFailure) {
+  @Override public boolean checkOperandTypes(SqlCallBinding callBinding,
+      boolean throwOnFailure) {
     final int operandCount = callBinding.getOperandCount();
     assert operandCount >= 2;
     if (!OperandTypes.ANY.checkSingleOperandType(
diff --git a/core/src/main/java/org/apache/calcite/sql/fun/SqlJsonStorageSizeFunction.java b/core/src/main/java/org/apache/calcite/sql/fun/SqlJsonStorageSizeFunction.java
index 8608e9c..5abbfea 100644
--- a/core/src/main/java/org/apache/calcite/sql/fun/SqlJsonStorageSizeFunction.java
+++ b/core/src/main/java/org/apache/calcite/sql/fun/SqlJsonStorageSizeFunction.java
@@ -29,12 +29,8 @@ import org.apache.calcite.sql.type.SqlTypeTransforms;
 public class SqlJsonStorageSizeFunction extends SqlFunction {
 
   public SqlJsonStorageSizeFunction() {
-    super("JSON_STORAGE_SIZE",
-        SqlKind.OTHER_FUNCTION,
-        ReturnTypes.cascade(ReturnTypes.INTEGER,
-            SqlTypeTransforms.FORCE_NULLABLE),
-        null,
-        OperandTypes.ANY,
-        SqlFunctionCategory.SYSTEM);
+    super("JSON_STORAGE_SIZE", SqlKind.OTHER_FUNCTION,
+        ReturnTypes.INTEGER.andThen(SqlTypeTransforms.FORCE_NULLABLE),
+        null, OperandTypes.ANY, SqlFunctionCategory.SYSTEM);
   }
 }
diff --git a/core/src/main/java/org/apache/calcite/sql/fun/SqlJsonTypeFunction.java b/core/src/main/java/org/apache/calcite/sql/fun/SqlJsonTypeFunction.java
index 90ff7c0..e477194 100644
--- a/core/src/main/java/org/apache/calcite/sql/fun/SqlJsonTypeFunction.java
+++ b/core/src/main/java/org/apache/calcite/sql/fun/SqlJsonTypeFunction.java
@@ -37,14 +37,10 @@ import org.apache.calcite.sql.validate.SqlValidator;
  */
 public class SqlJsonTypeFunction extends SqlFunction {
   public SqlJsonTypeFunction() {
-    super("JSON_TYPE",
-        SqlKind.OTHER_FUNCTION,
-        ReturnTypes.cascade(
-            ReturnTypes.explicit(SqlTypeName.VARCHAR, 20),
-            SqlTypeTransforms.FORCE_NULLABLE),
-        null,
-        OperandTypes.ANY,
-        SqlFunctionCategory.SYSTEM);
+    super("JSON_TYPE", SqlKind.OTHER_FUNCTION,
+        ReturnTypes.explicit(SqlTypeName.VARCHAR, 20)
+            .andThen(SqlTypeTransforms.FORCE_NULLABLE),
+        null, OperandTypes.ANY, SqlFunctionCategory.SYSTEM);
   }
 
   @Override public SqlOperandCountRange getOperandCountRange() {
diff --git a/core/src/main/java/org/apache/calcite/sql/fun/SqlJsonValueExpressionOperator.java b/core/src/main/java/org/apache/calcite/sql/fun/SqlJsonValueExpressionOperator.java
index 4215b26..450b758 100644
--- a/core/src/main/java/org/apache/calcite/sql/fun/SqlJsonValueExpressionOperator.java
+++ b/core/src/main/java/org/apache/calcite/sql/fun/SqlJsonValueExpressionOperator.java
@@ -31,7 +31,8 @@ public class SqlJsonValueExpressionOperator extends SqlPostfixOperator {
 
   public SqlJsonValueExpressionOperator() {
     super("FORMAT JSON", SqlKind.JSON_VALUE_EXPRESSION, 28,
-        ReturnTypes.cascade(ReturnTypes.explicit(SqlTypeName.ANY),
-            SqlTypeTransforms.TO_NULLABLE), null, OperandTypes.CHARACTER);
+        ReturnTypes.explicit(SqlTypeName.ANY)
+            .andThen(SqlTypeTransforms.TO_NULLABLE),
+        null, OperandTypes.CHARACTER);
   }
 }
diff --git a/core/src/main/java/org/apache/calcite/sql/fun/SqlLeadLagAggFunction.java b/core/src/main/java/org/apache/calcite/sql/fun/SqlLeadLagAggFunction.java
index ba41584..7d8f7f2 100644
--- a/core/src/main/java/org/apache/calcite/sql/fun/SqlLeadLagAggFunction.java
+++ b/core/src/main/java/org/apache/calcite/sql/fun/SqlLeadLagAggFunction.java
@@ -20,6 +20,7 @@ import org.apache.calcite.rel.type.RelDataType;
 import org.apache.calcite.sql.SqlAggFunction;
 import org.apache.calcite.sql.SqlFunctionCategory;
 import org.apache.calcite.sql.SqlKind;
+import org.apache.calcite.sql.SqlOperatorBinding;
 import org.apache.calcite.sql.type.OperandTypes;
 import org.apache.calcite.sql.type.ReturnTypes;
 import org.apache.calcite.sql.type.SameOperandTypeChecker;
@@ -56,19 +57,7 @@ public class SqlLeadLagAggFunction extends SqlAggFunction {
               }));
 
   private static final SqlReturnTypeInference RETURN_TYPE =
-      ReturnTypes.cascade(ReturnTypes.ARG0, (binding, type) -> {
-        // Result is NOT NULL if NOT NULL default value is provided
-        SqlTypeTransform transform;
-        if (binding.getOperandCount() < 3) {
-          transform = SqlTypeTransforms.FORCE_NULLABLE;
-        } else {
-          RelDataType defValueType = binding.getOperandType(2);
-          transform = defValueType.isNullable()
-              ? SqlTypeTransforms.FORCE_NULLABLE
-              : SqlTypeTransforms.TO_NOT_NULLABLE;
-        }
-        return transform.transformType(binding, type);
-      });
+      ReturnTypes.ARG0.andThen(SqlLeadLagAggFunction::transformType);
 
   public SqlLeadLagAggFunction(SqlKind kind) {
     super(kind.name(),
@@ -90,6 +79,16 @@ public class SqlLeadLagAggFunction extends SqlAggFunction {
     this(isLead ? SqlKind.LEAD : SqlKind.LAG);
   }
 
+  // Result is NOT NULL if NOT NULL default value is provided
+  private static RelDataType transformType(SqlOperatorBinding binding,
+      RelDataType type) {
+    SqlTypeTransform transform =
+        binding.getOperandCount() < 3 || binding.getOperandType(2).isNullable()
+            ? SqlTypeTransforms.FORCE_NULLABLE
+            : SqlTypeTransforms.TO_NOT_NULLABLE;
+    return transform.transformType(binding, type);
+  }
+
   @Override public boolean allowsFraming() {
     return false;
   }
diff --git a/core/src/main/java/org/apache/calcite/sql/fun/SqlLibraryOperators.java b/core/src/main/java/org/apache/calcite/sql/fun/SqlLibraryOperators.java
index f23c884..d578496 100644
--- a/core/src/main/java/org/apache/calcite/sql/fun/SqlLibraryOperators.java
+++ b/core/src/main/java/org/apache/calcite/sql/fun/SqlLibraryOperators.java
@@ -91,24 +91,24 @@ public abstract class SqlLibraryOperators {
   @LibraryOperator(libraries = {ORACLE})
   public static final SqlFunction NVL =
       new SqlFunction("NVL", SqlKind.NVL,
-          ReturnTypes.cascade(ReturnTypes.LEAST_RESTRICTIVE,
-              SqlTypeTransforms.TO_NULLABLE_ALL),
+          ReturnTypes.LEAST_RESTRICTIVE
+              .andThen(SqlTypeTransforms.TO_NULLABLE_ALL),
           null, OperandTypes.SAME_SAME, SqlFunctionCategory.SYSTEM);
 
   /** The "LTRIM(string)" function. */
   @LibraryOperator(libraries = {ORACLE})
   public static final SqlFunction LTRIM =
       new SqlFunction("LTRIM", SqlKind.LTRIM,
-          ReturnTypes.cascade(ReturnTypes.ARG0, SqlTypeTransforms.TO_NULLABLE,
-              SqlTypeTransforms.TO_VARYING), null,
+          ReturnTypes.ARG0.andThen(SqlTypeTransforms.TO_NULLABLE)
+              .andThen(SqlTypeTransforms.TO_VARYING), null,
           OperandTypes.STRING, SqlFunctionCategory.STRING);
 
   /** The "RTRIM(string)" function. */
   @LibraryOperator(libraries = {ORACLE})
   public static final SqlFunction RTRIM =
       new SqlFunction("RTRIM", SqlKind.RTRIM,
-          ReturnTypes.cascade(ReturnTypes.ARG0, SqlTypeTransforms.TO_NULLABLE,
-              SqlTypeTransforms.TO_VARYING), null,
+          ReturnTypes.ARG0.andThen(SqlTypeTransforms.TO_NULLABLE)
+              .andThen(SqlTypeTransforms.TO_VARYING), null,
           OperandTypes.STRING, SqlFunctionCategory.STRING);
 
   /** Oracle's "SUBSTR(string, position [, substringLength ])" function.
@@ -125,17 +125,15 @@ public abstract class SqlLibraryOperators {
   @LibraryOperator(libraries = {ORACLE})
   public static final SqlFunction GREATEST =
       new SqlFunction("GREATEST", SqlKind.GREATEST,
-          ReturnTypes.cascade(ReturnTypes.LEAST_RESTRICTIVE,
-              SqlTypeTransforms.TO_NULLABLE), null,
-          OperandTypes.SAME_VARIADIC, SqlFunctionCategory.SYSTEM);
+          ReturnTypes.LEAST_RESTRICTIVE.andThen(SqlTypeTransforms.TO_NULLABLE),
+          null, OperandTypes.SAME_VARIADIC, SqlFunctionCategory.SYSTEM);
 
   /** The "LEAST(value, value)" function. */
   @LibraryOperator(libraries = {ORACLE})
   public static final SqlFunction LEAST =
       new SqlFunction("LEAST", SqlKind.LEAST,
-          ReturnTypes.cascade(ReturnTypes.LEAST_RESTRICTIVE,
-              SqlTypeTransforms.TO_NULLABLE), null,
-          OperandTypes.SAME_VARIADIC, SqlFunctionCategory.SYSTEM);
+          ReturnTypes.LEAST_RESTRICTIVE.andThen(SqlTypeTransforms.TO_NULLABLE),
+          null, OperandTypes.SAME_VARIADIC, SqlFunctionCategory.SYSTEM);
 
   /**
    * The <code>TRANSLATE(<i>string_expr</i>, <i>search_chars</i>,
@@ -174,34 +172,38 @@ public abstract class SqlLibraryOperators {
   public static final SqlFunction REGEXP_REPLACE = new SqlRegexpReplaceFunction();
 
   @LibraryOperator(libraries = {MYSQL})
-  public static final SqlFunction COMPRESS = new SqlFunction("COMPRESS", SqlKind.OTHER_FUNCTION,
-      ReturnTypes.cascade(ReturnTypes.explicit(SqlTypeName.VARBINARY),
-          SqlTypeTransforms.TO_NULLABLE), null, OperandTypes.STRING, SqlFunctionCategory.STRING);
+  public static final SqlFunction COMPRESS =
+      new SqlFunction("COMPRESS", SqlKind.OTHER_FUNCTION,
+          ReturnTypes.explicit(SqlTypeName.VARBINARY)
+              .andThen(SqlTypeTransforms.TO_NULLABLE),
+          null, OperandTypes.STRING, SqlFunctionCategory.STRING);
 
 
   @LibraryOperator(libraries = {MYSQL})
-  public static final SqlFunction EXTRACT_VALUE = new SqlFunction(
-      "EXTRACTVALUE", SqlKind.OTHER_FUNCTION,
-      ReturnTypes.cascade(ReturnTypes.VARCHAR_2000, SqlTypeTransforms.FORCE_NULLABLE),
-      null, OperandTypes.STRING_STRING, SqlFunctionCategory.SYSTEM);
+  public static final SqlFunction EXTRACT_VALUE =
+      new SqlFunction("EXTRACTVALUE", SqlKind.OTHER_FUNCTION,
+          ReturnTypes.VARCHAR_2000.andThen(SqlTypeTransforms.FORCE_NULLABLE),
+          null, OperandTypes.STRING_STRING, SqlFunctionCategory.SYSTEM);
 
   @LibraryOperator(libraries = {ORACLE})
-  public static final SqlFunction XML_TRANSFORM = new SqlFunction(
-      "XMLTRANSFORM", SqlKind.OTHER_FUNCTION,
-      ReturnTypes.cascade(ReturnTypes.VARCHAR_2000, SqlTypeTransforms.FORCE_NULLABLE),
-      null, OperandTypes.STRING_STRING, SqlFunctionCategory.SYSTEM);
+  public static final SqlFunction XML_TRANSFORM =
+      new SqlFunction("XMLTRANSFORM", SqlKind.OTHER_FUNCTION,
+          ReturnTypes.VARCHAR_2000.andThen(SqlTypeTransforms.FORCE_NULLABLE),
+          null, OperandTypes.STRING_STRING, SqlFunctionCategory.SYSTEM);
 
   @LibraryOperator(libraries = {ORACLE})
-  public static final SqlFunction EXTRACT_XML = new SqlFunction(
-      "EXTRACT", SqlKind.OTHER_FUNCTION,
-      ReturnTypes.cascade(ReturnTypes.VARCHAR_2000, SqlTypeTransforms.FORCE_NULLABLE),
-      null, OperandTypes.STRING_STRING_OPTIONAL_STRING, SqlFunctionCategory.SYSTEM);
+  public static final SqlFunction EXTRACT_XML =
+      new SqlFunction("EXTRACT", SqlKind.OTHER_FUNCTION,
+          ReturnTypes.VARCHAR_2000.andThen(SqlTypeTransforms.FORCE_NULLABLE),
+          null, OperandTypes.STRING_STRING_OPTIONAL_STRING,
+          SqlFunctionCategory.SYSTEM);
 
   @LibraryOperator(libraries = {ORACLE})
-  public static final SqlFunction EXISTS_NODE = new SqlFunction(
-      "EXISTSNODE", SqlKind.OTHER_FUNCTION,
-      ReturnTypes.cascade(ReturnTypes.INTEGER_NULLABLE, SqlTypeTransforms.FORCE_NULLABLE),
-      null, OperandTypes.STRING_STRING_OPTIONAL_STRING, SqlFunctionCategory.SYSTEM);
+  public static final SqlFunction EXISTS_NODE =
+      new SqlFunction("EXISTSNODE", SqlKind.OTHER_FUNCTION,
+          ReturnTypes.INTEGER_NULLABLE
+              .andThen(SqlTypeTransforms.FORCE_NULLABLE), null,
+          OperandTypes.STRING_STRING_OPTIONAL_STRING, SqlFunctionCategory.SYSTEM);
 
   /** The "DATE(string)" function, equivalent to "CAST(string AS DATE). */
   @LibraryOperator(libraries = {BIG_QUERY})
@@ -334,22 +336,17 @@ public abstract class SqlLibraryOperators {
 
   @LibraryOperator(libraries = {MYSQL})
   public static final SqlFunction FROM_BASE64 =
-      new SqlFunction("FROM_BASE64",
-          SqlKind.OTHER_FUNCTION,
-          ReturnTypes.cascade(ReturnTypes.explicit(SqlTypeName.VARBINARY),
-              SqlTypeTransforms.TO_NULLABLE),
-          null,
-          OperandTypes.STRING,
-          SqlFunctionCategory.STRING);
+      new SqlFunction("FROM_BASE64", SqlKind.OTHER_FUNCTION,
+          ReturnTypes.explicit(SqlTypeName.VARBINARY)
+              .andThen(SqlTypeTransforms.TO_NULLABLE),
+          null, OperandTypes.STRING, SqlFunctionCategory.STRING);
 
   @LibraryOperator(libraries = {MYSQL})
   public static final SqlFunction TO_BASE64 =
-      new SqlFunction("TO_BASE64",
-          SqlKind.OTHER_FUNCTION,
-          ReturnTypes.cascade(ReturnTypes.explicit(SqlTypeName.VARCHAR),
-              SqlTypeTransforms.TO_NULLABLE),
-          null,
-          OperandTypes.or(OperandTypes.STRING, OperandTypes.BINARY),
+      new SqlFunction("TO_BASE64", SqlKind.OTHER_FUNCTION,
+          ReturnTypes.explicit(SqlTypeName.VARCHAR)
+              .andThen(SqlTypeTransforms.TO_NULLABLE),
+          null, OperandTypes.or(OperandTypes.STRING, OperandTypes.BINARY),
           SqlFunctionCategory.STRING);
 
   /** The "TO_DATE(string1, string2)" function; casts string1
@@ -460,22 +457,18 @@ public abstract class SqlLibraryOperators {
 
   @LibraryOperator(libraries = {MYSQL, POSTGRESQL})
   public static final SqlFunction MD5 =
-      new SqlFunction("MD5",
-          SqlKind.OTHER_FUNCTION,
-          ReturnTypes.cascade(ReturnTypes.explicit(SqlTypeName.VARCHAR),
-              SqlTypeTransforms.TO_NULLABLE),
-          null,
-          OperandTypes.or(OperandTypes.STRING, OperandTypes.BINARY),
+      new SqlFunction("MD5", SqlKind.OTHER_FUNCTION,
+          ReturnTypes.explicit(SqlTypeName.VARCHAR)
+              .andThen(SqlTypeTransforms.TO_NULLABLE),
+          null, OperandTypes.or(OperandTypes.STRING, OperandTypes.BINARY),
           SqlFunctionCategory.STRING);
 
   @LibraryOperator(libraries = {MYSQL, POSTGRESQL})
   public static final SqlFunction SHA1 =
-      new SqlFunction("SHA1",
-          SqlKind.OTHER_FUNCTION,
-          ReturnTypes.cascade(ReturnTypes.explicit(SqlTypeName.VARCHAR),
-              SqlTypeTransforms.TO_NULLABLE),
-          null,
-          OperandTypes.or(OperandTypes.STRING, OperandTypes.BINARY),
+      new SqlFunction("SHA1", SqlKind.OTHER_FUNCTION,
+          ReturnTypes.explicit(SqlTypeName.VARCHAR)
+              .andThen(SqlTypeTransforms.TO_NULLABLE),
+          null, OperandTypes.or(OperandTypes.STRING, OperandTypes.BINARY),
           SqlFunctionCategory.STRING);
 
   /** Infix "::" cast operator used by PostgreSQL, for example
diff --git a/core/src/main/java/org/apache/calcite/sql/fun/SqlRegexpReplaceFunction.java b/core/src/main/java/org/apache/calcite/sql/fun/SqlRegexpReplaceFunction.java
index 7ee4d42..777e184 100644
--- a/core/src/main/java/org/apache/calcite/sql/fun/SqlRegexpReplaceFunction.java
+++ b/core/src/main/java/org/apache/calcite/sql/fun/SqlRegexpReplaceFunction.java
@@ -39,20 +39,18 @@ import java.util.List;
 public class SqlRegexpReplaceFunction extends SqlFunction {
 
   public SqlRegexpReplaceFunction() {
-    super("REGEXP_REPLACE",
-        SqlKind.OTHER_FUNCTION,
-        ReturnTypes.cascade(ReturnTypes.explicit(SqlTypeName.VARCHAR),
-            SqlTypeTransforms.TO_NULLABLE),
-        null,
-        null,
-        SqlFunctionCategory.STRING);
+    super("REGEXP_REPLACE", SqlKind.OTHER_FUNCTION,
+        ReturnTypes.explicit(SqlTypeName.VARCHAR)
+            .andThen(SqlTypeTransforms.TO_NULLABLE),
+        null, null, SqlFunctionCategory.STRING);
   }
 
   @Override public SqlOperandCountRange getOperandCountRange() {
     return SqlOperandCountRanges.between(3, 6);
   }
 
-  @Override public boolean checkOperandTypes(SqlCallBinding callBinding, boolean throwOnFailure) {
+  @Override public boolean checkOperandTypes(SqlCallBinding callBinding,
+      boolean throwOnFailure) {
     final int operandCount = callBinding.getOperandCount();
     assert operandCount >= 3;
     if (operandCount == 3) {
diff --git a/core/src/main/java/org/apache/calcite/sql/fun/SqlTrimFunction.java b/core/src/main/java/org/apache/calcite/sql/fun/SqlTrimFunction.java
index 1702659..aff3235 100644
--- a/core/src/main/java/org/apache/calcite/sql/fun/SqlTrimFunction.java
+++ b/core/src/main/java/org/apache/calcite/sql/fun/SqlTrimFunction.java
@@ -29,9 +29,9 @@ import org.apache.calcite.sql.parser.SqlParserPos;
 import org.apache.calcite.sql.type.OperandTypes;
 import org.apache.calcite.sql.type.ReturnTypes;
 import org.apache.calcite.sql.type.SameOperandTypeChecker;
+import org.apache.calcite.sql.type.SqlReturnTypeInference;
 import org.apache.calcite.sql.type.SqlSingleOperandTypeChecker;
 import org.apache.calcite.sql.type.SqlTypeFamily;
-import org.apache.calcite.sql.type.SqlTypeTransformCascade;
 import org.apache.calcite.sql.type.SqlTypeTransforms;
 import org.apache.calcite.sql.type.SqlTypeUtil;
 
@@ -46,8 +46,8 @@ import java.util.List;
 public class SqlTrimFunction extends SqlFunction {
   protected static final SqlTrimFunction INSTANCE =
       new SqlTrimFunction("TRIM", SqlKind.TRIM,
-          ReturnTypes.cascade(ReturnTypes.ARG2, SqlTypeTransforms.TO_NULLABLE,
-              SqlTypeTransforms.TO_VARYING),
+          ReturnTypes.ARG2.andThen(SqlTypeTransforms.TO_NULLABLE)
+              .andThen(SqlTypeTransforms.TO_VARYING),
           OperandTypes.and(
               OperandTypes.family(SqlTypeFamily.ANY, SqlTypeFamily.STRING,
                   SqlTypeFamily.STRING),
@@ -87,7 +87,7 @@ public class SqlTrimFunction extends SqlFunction {
   //~ Constructors -----------------------------------------------------------
 
   public SqlTrimFunction(String name, SqlKind kind,
-      SqlTypeTransformCascade returnTypeInference,
+      SqlReturnTypeInference returnTypeInference,
       SqlSingleOperandTypeChecker operandTypeChecker) {
     super(name, kind, returnTypeInference, null, operandTypeChecker,
         SqlFunctionCategory.STRING);
diff --git a/core/src/main/java/org/apache/calcite/sql/type/ReturnTypes.java b/core/src/main/java/org/apache/calcite/sql/type/ReturnTypes.java
index 7c76f65..98bfcb3 100644
--- a/core/src/main/java/org/apache/calcite/sql/type/ReturnTypes.java
+++ b/core/src/main/java/org/apache/calcite/sql/type/ReturnTypes.java
@@ -43,13 +43,19 @@ public abstract class ReturnTypes {
   private ReturnTypes() {
   }
 
+  /** Creates a return-type inference that applies a rule then a sequence of
+   * rules, returning the first non-null result.
+   *
+   * @see SqlReturnTypeInference#orElse(SqlReturnTypeInference) */
   public static SqlReturnTypeInferenceChain chain(
       SqlReturnTypeInference... rules) {
     return new SqlReturnTypeInferenceChain(rules);
   }
 
   /** Creates a return-type inference that applies a rule then a sequence of
-   * transforms. */
+   * transforms.
+   *
+   * @see SqlReturnTypeInference#andThen(SqlTypeTransform) */
   public static SqlTypeTransformCascade cascade(SqlReturnTypeInference rule,
       SqlTypeTransform... transforms) {
     return new SqlTypeTransformCascade(rule, transforms);
@@ -90,6 +96,7 @@ public abstract class ReturnTypes {
    */
   public static final SqlReturnTypeInference ARG0 =
       new OrdinalReturnTypeInference(0);
+
   /**
    * Type-inference strategy whereby the result type of a call is VARYING the
    * type of the first argument. The length returned is the same as length of
@@ -97,8 +104,8 @@ public abstract class ReturnTypes {
    * returned type will also be nullable. First Arg must be of string type.
    */
   public static final SqlReturnTypeInference ARG0_NULLABLE_VARYING =
-      cascade(ARG0, SqlTypeTransforms.TO_NULLABLE,
-          SqlTypeTransforms.TO_VARYING);
+      ARG0.andThen(SqlTypeTransforms.TO_NULLABLE)
+          .andThen(SqlTypeTransforms.TO_VARYING);
 
   /**
    * Type-inference strategy whereby the result type of a call is the type of
@@ -106,21 +113,21 @@ public abstract class ReturnTypes {
    * returned type will also be nullable.
    */
   public static final SqlReturnTypeInference ARG0_NULLABLE =
-      cascade(ARG0, SqlTypeTransforms.TO_NULLABLE);
+      ARG0.andThen(SqlTypeTransforms.TO_NULLABLE);
 
   /**
    * Type-inference strategy whereby the result type of a call is the type of
    * the operand #0 (0-based), with nulls always allowed.
    */
   public static final SqlReturnTypeInference ARG0_FORCE_NULLABLE =
-      cascade(ARG0, SqlTypeTransforms.FORCE_NULLABLE);
+      ARG0.andThen(SqlTypeTransforms.FORCE_NULLABLE);
 
   public static final SqlReturnTypeInference ARG0_INTERVAL =
       new MatchReturnTypeInference(0,
           SqlTypeFamily.DATETIME_INTERVAL.getTypeNames());
 
   public static final SqlReturnTypeInference ARG0_INTERVAL_NULLABLE =
-      cascade(ARG0_INTERVAL, SqlTypeTransforms.TO_NULLABLE);
+      ARG0_INTERVAL.andThen(SqlTypeTransforms.TO_NULLABLE);
 
   /**
    * Type-inference strategy whereby the result type of a call is the type of
@@ -148,26 +155,30 @@ public abstract class ReturnTypes {
    */
   public static final SqlReturnTypeInference ARG1 =
       new OrdinalReturnTypeInference(1);
+
   /**
    * Type-inference strategy whereby the result type of a call is the type of
    * the operand #1 (0-based). If any of the other operands are nullable the
    * returned type will also be nullable.
    */
   public static final SqlReturnTypeInference ARG1_NULLABLE =
-      cascade(ARG1, SqlTypeTransforms.TO_NULLABLE);
+      ARG1.andThen(SqlTypeTransforms.TO_NULLABLE);
+
   /**
    * Type-inference strategy whereby the result type of a call is the type of
    * operand #2 (0-based).
    */
   public static final SqlReturnTypeInference ARG2 =
       new OrdinalReturnTypeInference(2);
+
   /**
    * Type-inference strategy whereby the result type of a call is the type of
    * operand #2 (0-based). If any of the other operands are nullable the
    * returned type will also be nullable.
    */
   public static final SqlReturnTypeInference ARG2_NULLABLE =
-      cascade(ARG2, SqlTypeTransforms.TO_NULLABLE);
+      ARG2.andThen(SqlTypeTransforms.TO_NULLABLE);
+
   /**
    * Type-inference strategy whereby the result type of a call is Boolean.
    */
@@ -178,7 +189,7 @@ public abstract class ReturnTypes {
    * with nulls allowed if any of the operands allow nulls.
    */
   public static final SqlReturnTypeInference BOOLEAN_NULLABLE =
-      cascade(BOOLEAN, SqlTypeTransforms.TO_NULLABLE);
+      BOOLEAN.andThen(SqlTypeTransforms.TO_NULLABLE);
 
   /**
    * Type-inference strategy with similar effect to {@link #BOOLEAN_NULLABLE},
@@ -207,14 +218,14 @@ public abstract class ReturnTypes {
    * Boolean.
    */
   public static final SqlReturnTypeInference BOOLEAN_FORCE_NULLABLE =
-      cascade(BOOLEAN, SqlTypeTransforms.FORCE_NULLABLE);
+      BOOLEAN.andThen(SqlTypeTransforms.FORCE_NULLABLE);
 
   /**
    * Type-inference strategy whereby the result type of a call is BOOLEAN
    * NOT NULL.
    */
   public static final SqlReturnTypeInference BOOLEAN_NOT_NULL =
-      cascade(BOOLEAN, SqlTypeTransforms.TO_NOT_NULLABLE);
+      BOOLEAN.andThen(SqlTypeTransforms.TO_NOT_NULLABLE);
 
   /**
    * Type-inference strategy whereby the result type of a call is DATE.
@@ -227,7 +238,7 @@ public abstract class ReturnTypes {
    * DATE.
    */
   public static final SqlReturnTypeInference DATE_NULLABLE =
-      cascade(DATE, SqlTypeTransforms.TO_NULLABLE);
+      DATE.andThen(SqlTypeTransforms.TO_NULLABLE);
 
   /**
    * Type-inference strategy whereby the result type of a call is TIME(0).
@@ -240,7 +251,7 @@ public abstract class ReturnTypes {
    * TIME(0).
    */
   public static final SqlReturnTypeInference TIME_NULLABLE =
-      cascade(TIME, SqlTypeTransforms.TO_NULLABLE);
+      TIME.andThen(SqlTypeTransforms.TO_NULLABLE);
 
   /**
    * Type-inference strategy whereby the result type of a call is TIMESTAMP.
@@ -253,19 +264,20 @@ public abstract class ReturnTypes {
    * TIMESTAMP.
    */
   public static final SqlReturnTypeInference TIMESTAMP_NULLABLE =
-      cascade(TIMESTAMP, SqlTypeTransforms.TO_NULLABLE);
+      TIMESTAMP.andThen(SqlTypeTransforms.TO_NULLABLE);
 
   /**
    * Type-inference strategy whereby the result type of a call is Double.
    */
   public static final SqlReturnTypeInference DOUBLE =
       explicit(SqlTypeName.DOUBLE);
+
   /**
    * Type-inference strategy whereby the result type of a call is Double with
    * nulls allowed if any of the operands allow nulls.
    */
   public static final SqlReturnTypeInference DOUBLE_NULLABLE =
-      cascade(DOUBLE, SqlTypeTransforms.TO_NULLABLE);
+      DOUBLE.andThen(SqlTypeTransforms.TO_NULLABLE);
 
   /**
    * Type-inference strategy whereby the result type of a call is a Char.
@@ -284,7 +296,7 @@ public abstract class ReturnTypes {
    * with nulls allowed if any of the operands allow nulls.
    */
   public static final SqlReturnTypeInference INTEGER_NULLABLE =
-      cascade(INTEGER, SqlTypeTransforms.TO_NULLABLE);
+      INTEGER.andThen(SqlTypeTransforms.TO_NULLABLE);
 
   /**
    * Type-inference strategy whereby the result type of a call is a BIGINT.
@@ -297,13 +309,14 @@ public abstract class ReturnTypes {
    * BIGINT.
    */
   public static final SqlReturnTypeInference BIGINT_FORCE_NULLABLE =
-      cascade(BIGINT, SqlTypeTransforms.FORCE_NULLABLE);
+      BIGINT.andThen(SqlTypeTransforms.FORCE_NULLABLE);
+
   /**
-   * Type-inference strategy whereby the result type of a call is an Bigint
+   * Type-inference strategy whereby the result type of a call is a BIGINT
    * with nulls allowed if any of the operands allow nulls.
    */
   public static final SqlReturnTypeInference BIGINT_NULLABLE =
-      cascade(BIGINT, SqlTypeTransforms.TO_NULLABLE);
+      BIGINT.andThen(SqlTypeTransforms.TO_NULLABLE);
 
   /**
    * Type-inference strategy that always returns "VARCHAR(4)".
@@ -316,7 +329,7 @@ public abstract class ReturnTypes {
    * allowed if any of the operands allow nulls.
    */
   public static final SqlReturnTypeInference VARCHAR_4_NULLABLE =
-      cascade(VARCHAR_4, SqlTypeTransforms.TO_NULLABLE);
+      VARCHAR_4.andThen(SqlTypeTransforms.TO_NULLABLE);
 
   /**
    * Type-inference strategy that always returns "VARCHAR(2000)".
@@ -329,7 +342,7 @@ public abstract class ReturnTypes {
    * allowed if any of the operands allow nulls.
    */
   public static final SqlReturnTypeInference VARCHAR_2000_NULLABLE =
-      cascade(VARCHAR_2000, SqlTypeTransforms.TO_NULLABLE);
+      VARCHAR_2000.andThen(SqlTypeTransforms.TO_NULLABLE);
 
   /**
    * Type-inference strategy for Histogram agg support.
@@ -348,6 +361,7 @@ public abstract class ReturnTypes {
    */
   public static final SqlReturnTypeInference COLUMN_LIST =
       explicit(SqlTypeName.COLUMN_LIST);
+
   /**
    * Type-inference strategy whereby the result type of a call is using its
    * operands biggest type, using the SQL:1999 rules described in "Data types
@@ -359,6 +373,7 @@ public abstract class ReturnTypes {
   public static final SqlReturnTypeInference LEAST_RESTRICTIVE =
       opBinding -> opBinding.getTypeFactory().leastRestrictive(
           opBinding.collectOperandTypes());
+
   /**
    * Returns the same type as the multiset carries. The multiset type returned
    * is the least restrictive of the call's multiset operands
@@ -395,20 +410,20 @@ public abstract class ReturnTypes {
    * <code>INTEGER MULTISET</code>.
    */
   public static final SqlReturnTypeInference TO_MULTISET =
-      cascade(ARG0, SqlTypeTransforms.TO_MULTISET);
+      ARG0.andThen(SqlTypeTransforms.TO_MULTISET);
 
   /**
    * Returns the element type of a MULTISET.
    */
   public static final SqlReturnTypeInference MULTISET_ELEMENT_NULLABLE =
-      cascade(MULTISET, SqlTypeTransforms.TO_MULTISET_ELEMENT_TYPE);
+      MULTISET.andThen(SqlTypeTransforms.TO_MULTISET_ELEMENT_TYPE);
 
   /**
    * Same as {@link #MULTISET} but returns with nullability if any of the
    * operands is nullable.
    */
   public static final SqlReturnTypeInference MULTISET_NULLABLE =
-      cascade(MULTISET, SqlTypeTransforms.TO_NULLABLE);
+      MULTISET.andThen(SqlTypeTransforms.TO_NULLABLE);
 
   /**
    * Returns the type of the only column of a multiset.
@@ -417,7 +432,7 @@ public abstract class ReturnTypes {
    * <code>INTEGER MULTISET</code>.
    */
   public static final SqlReturnTypeInference MULTISET_PROJECT_ONLY =
-      cascade(MULTISET, SqlTypeTransforms.ONLY_COLUMN);
+      MULTISET.andThen(SqlTypeTransforms.ONLY_COLUMN);
 
   /**
    * Type-inference strategy whereby the result type of a call is
@@ -425,7 +440,7 @@ public abstract class ReturnTypes {
    * are used for integer division.
    */
   public static final SqlReturnTypeInference INTEGER_QUOTIENT_NULLABLE =
-      chain(ARG0_INTERVAL_NULLABLE, LEAST_RESTRICTIVE);
+      ARG0_INTERVAL_NULLABLE.orElse(LEAST_RESTRICTIVE);
 
   /**
    * Type-inference strategy for a call where the first argument is a decimal.
@@ -462,7 +477,7 @@ public abstract class ReturnTypes {
    * is used for floor, ceiling.
    */
   public static final SqlReturnTypeInference ARG0_OR_EXACT_NO_SCALE =
-      chain(DECIMAL_SCALE0, ARG0);
+      DECIMAL_SCALE0.orElse(ARG0);
 
   /**
    * Type-inference strategy whereby the result type of a call is the decimal
@@ -482,7 +497,7 @@ public abstract class ReturnTypes {
    * {@link org.apache.calcite.sql.type.SqlTypeTransforms#TO_NULLABLE}.
    */
   public static final SqlReturnTypeInference DECIMAL_PRODUCT_NULLABLE =
-      cascade(DECIMAL_PRODUCT, SqlTypeTransforms.TO_NULLABLE);
+      DECIMAL_PRODUCT.andThen(SqlTypeTransforms.TO_NULLABLE);
 
   /**
    * Type-inference strategy whereby the result type of a call is
@@ -492,8 +507,8 @@ public abstract class ReturnTypes {
    * These rules are used for multiplication.
    */
   public static final SqlReturnTypeInference PRODUCT_NULLABLE =
-      chain(DECIMAL_PRODUCT_NULLABLE, ARG0_INTERVAL_NULLABLE,
-          LEAST_RESTRICTIVE);
+      DECIMAL_PRODUCT_NULLABLE.orElse(ARG0_INTERVAL_NULLABLE)
+          .orElse(LEAST_RESTRICTIVE);
 
   /**
    * Type-inference strategy whereby the result type of a call is the decimal
@@ -513,7 +528,7 @@ public abstract class ReturnTypes {
    * {@link org.apache.calcite.sql.type.SqlTypeTransforms#TO_NULLABLE}.
    */
   public static final SqlReturnTypeInference DECIMAL_QUOTIENT_NULLABLE =
-      cascade(DECIMAL_QUOTIENT, SqlTypeTransforms.TO_NULLABLE);
+      DECIMAL_QUOTIENT.andThen(SqlTypeTransforms.TO_NULLABLE);
 
   /**
    * Type-inference strategy whereby the result type of a call is
@@ -522,8 +537,8 @@ public abstract class ReturnTypes {
    * are used for division.
    */
   public static final SqlReturnTypeInference QUOTIENT_NULLABLE =
-      chain(DECIMAL_QUOTIENT_NULLABLE, ARG0_INTERVAL_NULLABLE,
-          LEAST_RESTRICTIVE);
+      DECIMAL_QUOTIENT_NULLABLE.orElse(ARG0_INTERVAL_NULLABLE)
+          .orElse(LEAST_RESTRICTIVE);
 
   /**
    * Type-inference strategy whereby the result type of a call is the decimal
@@ -543,7 +558,7 @@ public abstract class ReturnTypes {
    * {@link org.apache.calcite.sql.type.SqlTypeTransforms#TO_NULLABLE}.
    */
   public static final SqlReturnTypeInference DECIMAL_SUM_NULLABLE =
-      cascade(DECIMAL_SUM, SqlTypeTransforms.TO_NULLABLE);
+      DECIMAL_SUM.andThen(SqlTypeTransforms.TO_NULLABLE);
 
   /**
    * Type-inference strategy whereby the result type of a call is
@@ -566,14 +581,15 @@ public abstract class ReturnTypes {
    * decimal.
    */
   public static final SqlReturnTypeInference DECIMAL_MOD_NULLABLE =
-          cascade(DECIMAL_MOD, SqlTypeTransforms.TO_NULLABLE);
+      DECIMAL_MOD.andThen(SqlTypeTransforms.TO_NULLABLE);
+
   /**
    * Type-inference strategy whereby the result type of a call is
    * {@link #DECIMAL_MOD_NULLABLE} with a fallback to {@link #ARG1_NULLABLE}
    * These rules are used for modulus.
    */
   public static final SqlReturnTypeInference NULLABLE_MOD =
-          chain(DECIMAL_MOD_NULLABLE, ARG1_NULLABLE);
+      DECIMAL_MOD_NULLABLE.orElse(ARG1_NULLABLE);
 
   /**
    * Type-inference strategy for concatenating two string arguments. The result
@@ -674,7 +690,6 @@ public abstract class ReturnTypes {
         return ret;
       };
 
-
   /**
    * Type-inference strategy for String concatenation.
    * Result is varying if either input is; otherwise fixed.
@@ -725,7 +740,7 @@ public abstract class ReturnTypes {
    * {@link org.apache.calcite.sql.type.SqlTypeTransforms#TO_NULLABLE}.
    */
   public static final SqlReturnTypeInference MULTIVALENT_STRING_SUM_PRECISION_NULLABLE =
-      cascade(MULTIVALENT_STRING_SUM_PRECISION, SqlTypeTransforms.TO_NULLABLE);
+      MULTIVALENT_STRING_SUM_PRECISION.andThen(SqlTypeTransforms.TO_NULLABLE);
 
   /**
    * Same as {@link #DYADIC_STRING_SUM_PRECISION} and using
@@ -733,15 +748,15 @@ public abstract class ReturnTypes {
    * {@link org.apache.calcite.sql.type.SqlTypeTransforms#TO_VARYING}.
    */
   public static final SqlReturnTypeInference DYADIC_STRING_SUM_PRECISION_NULLABLE_VARYING =
-      cascade(DYADIC_STRING_SUM_PRECISION, SqlTypeTransforms.TO_NULLABLE,
-          SqlTypeTransforms.TO_VARYING);
+      DYADIC_STRING_SUM_PRECISION.andThen(SqlTypeTransforms.TO_NULLABLE)
+          .andThen(SqlTypeTransforms.TO_VARYING);
 
   /**
    * Same as {@link #DYADIC_STRING_SUM_PRECISION} and using
    * {@link org.apache.calcite.sql.type.SqlTypeTransforms#TO_NULLABLE}.
    */
   public static final SqlReturnTypeInference DYADIC_STRING_SUM_PRECISION_NULLABLE =
-      cascade(DYADIC_STRING_SUM_PRECISION, SqlTypeTransforms.TO_NULLABLE);
+      DYADIC_STRING_SUM_PRECISION.andThen(SqlTypeTransforms.TO_NULLABLE);
 
   /**
    * Type-inference strategy where the expression is assumed to be registered
@@ -775,6 +790,7 @@ public abstract class ReturnTypes {
         firstColType,
         -1);
   };
+
   /**
    * Returns a multiset of the first column of a multiset. For example, given
    * <code>INTEGER MULTISET</code>, returns <code>RECORD(x INTEGER)
@@ -791,6 +807,7 @@ public abstract class ReturnTypes {
         .add(SqlUtil.deriveAliasFromOrdinal(0), componentType).build();
     return typeFactory.createMultisetType(type, -1);
   };
+
   /**
    * Returns the field type of a structured type which has only one field. For
    * example, given {@code RECORD(x INTEGER)} returns {@code INTEGER}.
diff --git a/core/src/main/java/org/apache/calcite/sql/type/SqlReturnTypeInference.java b/core/src/main/java/org/apache/calcite/sql/type/SqlReturnTypeInference.java
index 3e56728..c9e31d2 100644
--- a/core/src/main/java/org/apache/calcite/sql/type/SqlReturnTypeInference.java
+++ b/core/src/main/java/org/apache/calcite/sql/type/SqlReturnTypeInference.java
@@ -30,6 +30,7 @@ import org.apache.calcite.sql.SqlOperatorBinding;
  * sense because many operators have similar, straightforward strategies, such
  * as to take the type of the first operand.</p>
  */
+@FunctionalInterface
 public interface SqlReturnTypeInference {
   //~ Methods ----------------------------------------------------------------
 
@@ -41,4 +42,16 @@ public interface SqlReturnTypeInference {
    */
   RelDataType inferReturnType(
       SqlOperatorBinding opBinding);
+
+  /** Returns a return-type inference that applies this rule then a
+   * transform. */
+  default SqlReturnTypeInference andThen(SqlTypeTransform transform) {
+    return ReturnTypes.cascade(this, transform);
+  }
+
+  /** Returns a return-type inference that applies this rule then another
+   * rule, until one of them returns a not-null result. */
+  default SqlReturnTypeInference orElse(SqlReturnTypeInference transform) {
+    return ReturnTypes.chain(this, transform);
+  }
 }
diff --git a/core/src/main/java/org/apache/calcite/sql/type/SqlReturnTypeInferenceChain.java b/core/src/main/java/org/apache/calcite/sql/type/SqlReturnTypeInferenceChain.java
index a0db292..67163d9 100644
--- a/core/src/main/java/org/apache/calcite/sql/type/SqlReturnTypeInferenceChain.java
+++ b/core/src/main/java/org/apache/calcite/sql/type/SqlReturnTypeInferenceChain.java
@@ -19,6 +19,7 @@ package org.apache.calcite.sql.type;
 import org.apache.calcite.rel.type.RelDataType;
 import org.apache.calcite.sql.SqlOperatorBinding;
 
+import com.google.common.base.Preconditions;
 import com.google.common.collect.ImmutableList;
 
 /**
@@ -41,11 +42,7 @@ public class SqlReturnTypeInferenceChain implements SqlReturnTypeInference {
    * Use {@link org.apache.calcite.sql.type.ReturnTypes#chain}.</p>
    */
   SqlReturnTypeInferenceChain(SqlReturnTypeInference... rules) {
-    assert rules != null;
-    assert rules.length > 1;
-    for (SqlReturnTypeInference rule : rules) {
-      assert rule != null;
-    }
+    Preconditions.checkArgument(rules.length > 1);
     this.rules = ImmutableList.copyOf(rules);
   }
 
diff --git a/core/src/main/java/org/apache/calcite/sql/type/SqlTypeTransformCascade.java b/core/src/main/java/org/apache/calcite/sql/type/SqlTypeTransformCascade.java
index 8b60f48..474a0f0 100644
--- a/core/src/main/java/org/apache/calcite/sql/type/SqlTypeTransformCascade.java
+++ b/core/src/main/java/org/apache/calcite/sql/type/SqlTypeTransformCascade.java
@@ -19,8 +19,11 @@ package org.apache.calcite.sql.type;
 import org.apache.calcite.rel.type.RelDataType;
 import org.apache.calcite.sql.SqlOperatorBinding;
 
+import com.google.common.base.Preconditions;
 import com.google.common.collect.ImmutableList;
 
+import java.util.Objects;
+
 /**
  * Strategy to infer the type of an operator call from the type of the operands
  * by using one {@link SqlReturnTypeInference} rule and a combination of
@@ -41,9 +44,8 @@ public class SqlTypeTransformCascade implements SqlReturnTypeInference {
   public SqlTypeTransformCascade(
       SqlReturnTypeInference rule,
       SqlTypeTransform... transforms) {
-    assert rule != null;
-    assert transforms.length > 0;
-    this.rule = rule;
+    Preconditions.checkArgument(transforms.length > 0);
+    this.rule = Objects.requireNonNull(rule);
     this.transforms = ImmutableList.copyOf(transforms);
   }
 


[calcite] 01/09: [CALCITE-3923] Refactor how planner rules are parameterized

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

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

commit 998cd83ebe2fcd7ff3ee565299fed6e199269715
Author: Julian Hyde <jh...@apache.org>
AuthorDate: Wed Jul 8 14:31:40 2020 -0700

    [CALCITE-3923] Refactor how planner rules are parameterized
    
    Create operands using a builder interface, OperandBuilder, and
    deprecate methods RelOptRule.operand etc.
    
    The change is backwards compatible, in the sense that
    existing rule constructors are deprecated but still work.
    From now on, to create rules, call RelOptRule.Config.toRule()
    (which calls the rule's (Config) constructor).
    
    Sub-classes of ConverterRule are a little different.  They
    don't need their own sub-class of Config.  You just need to
    call Config.withRuleFactory to specify the constructor of the
    sub-class of ConverterRule.
    
    Move rule instances into holder classes such as CoreRules,
    MaterializedViewRules. Deprecate existing rule INSTANCE fields
    (to be removed in 1.25). Deprecate previous rule constructors
    (to be removed in 2.0).
    
    Describe how to write rules in howto and tutorial.
    
    Remove rule instances marked 'deprecated, to be removed before
    1.25'.
    
    Close apache/calcite#2024
---
 .../calcite/adapter/cassandra/CassandraRules.java  | 178 +++--
 .../CassandraToEnumerableConverterRule.java        |  27 +-
 .../enumerable/EnumerableAggregateRule.java        |  16 +-
 .../EnumerableBatchNestedLoopJoinRule.java         |  68 +-
 .../adapter/enumerable/EnumerableBindable.java     |  26 +-
 .../adapter/enumerable/EnumerableCalcRule.java     |  26 +-
 .../adapter/enumerable/EnumerableCollectRule.java  |  16 +-
 .../enumerable/EnumerableCorrelateRule.java        |  26 +-
 .../adapter/enumerable/EnumerableFilterRule.java   |  23 +-
 .../enumerable/EnumerableFilterToCalcRule.java     |  37 +-
 .../enumerable/EnumerableInterpreterRule.java      |  25 +-
 .../enumerable/EnumerableIntersectRule.java        |  16 +-
 .../adapter/enumerable/EnumerableJoinRule.java     |  19 +-
 .../adapter/enumerable/EnumerableLimitRule.java    |  28 +-
 .../adapter/enumerable/EnumerableMatchRule.java    |  14 +-
 .../enumerable/EnumerableMergeJoinRule.java        |  15 +-
 .../adapter/enumerable/EnumerableMinusRule.java    |  16 +-
 .../adapter/enumerable/EnumerableProjectRule.java  |  23 +-
 .../enumerable/EnumerableProjectToCalcRule.java    |  36 +-
 .../enumerable/EnumerableRepeatUnionRule.java      |  20 +-
 .../adapter/enumerable/EnumerableRules.java        | 126 +++-
 .../adapter/enumerable/EnumerableSortRule.java     |  14 +-
 .../enumerable/EnumerableSortedAggregateRule.java  |  14 +-
 .../EnumerableTableFunctionScanRule.java           |  40 +-
 .../enumerable/EnumerableTableModifyRule.java      |  28 +-
 .../enumerable/EnumerableTableScanRule.java        |  36 +-
 .../enumerable/EnumerableTableSpoolRule.java       |  15 +-
 .../enumerable/EnumerableUncollectRule.java        |  16 +-
 .../adapter/enumerable/EnumerableUnionRule.java    |  16 +-
 .../adapter/enumerable/EnumerableValuesRule.java   |  25 +-
 .../adapter/enumerable/EnumerableWindowRule.java   |  16 +-
 .../org/apache/calcite/adapter/jdbc/JdbcRules.java | 274 +++----
 .../jdbc/JdbcToEnumerableConverterRule.java        |  19 +-
 .../org/apache/calcite/interpreter/Bindables.java  | 269 +++----
 .../interpreter/NoneToBindableConverterRule.java   |  25 +-
 .../apache/calcite/plan/CommonRelSubExprRule.java  |  23 +-
 .../apache/calcite/plan/RelOptMaterialization.java |   4 +-
 .../java/org/apache/calcite/plan/RelOptRule.java   |  34 +-
 .../org/apache/calcite/plan/RelOptRuleCall.java    |   2 +-
 .../calcite/plan/RelOptRuleOperandChildren.java    |   5 +-
 .../java/org/apache/calcite/plan/RelOptUtil.java   |  12 +-
 .../main/java/org/apache/calcite/plan/RelRule.java | 324 +++++++++
 .../org/apache/calcite/plan/hep/HepPlanner.java    |   3 +-
 .../apache/calcite/plan/hep/HepProgramBuilder.java |   3 +-
 .../org/apache/calcite/plan/hep/HepRuleCall.java   |   3 +-
 .../calcite/plan/volcano/AbstractConverter.java    |  44 +-
 .../calcite/plan/volcano/VolcanoRuleCall.java      |   3 +-
 .../apache/calcite/rel/convert/ConverterRule.java  | 100 ++-
 .../calcite/rel/convert/TraitMatchingRule.java     |  77 +-
 .../java/org/apache/calcite/rel/core/Calc.java     |   6 +
 .../java/org/apache/calcite/rel/core/Filter.java   |   6 +
 .../java/org/apache/calcite/rel/core/Project.java  |   6 +
 .../org/apache/calcite/rel/core/RelFactories.java  |   2 +-
 .../apache/calcite/rel/rel2sql/SqlImplementor.java |   2 +-
 .../rel/rules/AbstractJoinExtractFilterRule.java   |  22 +-
 .../rel/rules/AbstractMaterializedViewRule.java    |  48 --
 .../rel/rules/AggregateCaseToFilterRule.java       |  34 +-
 .../AggregateExpandDistinctAggregatesRule.java     |  62 +-
 .../rel/rules/AggregateExtractProjectRule.java     |  61 +-
 .../rel/rules/AggregateFilterTransposeRule.java    |  58 +-
 .../rel/rules/AggregateJoinJoinRemoveRule.java     |  66 +-
 .../calcite/rel/rules/AggregateJoinRemoveRule.java |  51 +-
 .../rel/rules/AggregateJoinTransposeRule.java      |  79 ++-
 .../calcite/rel/rules/AggregateMergeRule.java      |  48 +-
 .../rel/rules/AggregateProjectMergeRule.java       |  46 +-
 .../rules/AggregateProjectPullUpConstantsRule.java |  71 +-
 .../rel/rules/AggregateProjectStarTableRule.java   |  78 ++
 .../rel/rules/AggregateReduceFunctionsRule.java    | 130 ++--
 .../calcite/rel/rules/AggregateRemoveRule.java     |  52 +-
 .../calcite/rel/rules/AggregateStarTableRule.java  |  62 +-
 .../rel/rules/AggregateUnionAggregateRule.java     |  89 ++-
 .../rel/rules/AggregateUnionTransposeRule.java     |  48 +-
 .../calcite/rel/rules/AggregateValuesRule.java     |  53 +-
 .../apache/calcite/rel/rules/CalcMergeRule.java    |  46 +-
 .../apache/calcite/rel/rules/CalcRemoveRule.java   |  50 +-
 .../apache/calcite/rel/rules/CalcSplitRule.java    |  36 +-
 .../apache/calcite/rel/rules/CoerceInputsRule.java |  87 ++-
 .../org/apache/calcite/rel/rules/CoreRules.java    | 470 ++++--------
 .../apache/calcite/rel/rules/DateRangeRules.java   |  68 +-
 .../rel/rules/ExchangeRemoveConstantKeysRule.java  | 199 +++---
 .../rel/rules/FilterAggregateTransposeRule.java    |  72 +-
 .../calcite/rel/rules/FilterCalcMergeRule.java     |  57 +-
 .../calcite/rel/rules/FilterCorrelateRule.java     |  59 +-
 .../apache/calcite/rel/rules/FilterJoinRule.java   | 195 ++---
 .../apache/calcite/rel/rules/FilterMergeRule.java  |  44 +-
 .../rel/rules/FilterMultiJoinMergeRule.java        |  61 +-
 .../rel/rules/FilterProjectTransposeRule.java      | 153 +++-
 .../rules/FilterRemoveIsNotDistinctFromRule.java   |  46 +-
 .../rel/rules/FilterSetOpTransposeRule.java        |  47 +-
 .../rules/FilterTableFunctionTransposeRule.java    |  42 +-
 .../calcite/rel/rules/FilterTableScanRule.java     |  74 +-
 .../apache/calcite/rel/rules/FilterToCalcRule.java |  42 +-
 .../calcite/rel/rules/IntersectToDistinctRule.java |  43 +-
 .../rel/rules/JoinAddRedundantSemiJoinRule.java    |  46 +-
 .../calcite/rel/rules/JoinAssociateRule.java       |  51 +-
 .../apache/calcite/rel/rules/JoinCommuteRule.java  |  86 ++-
 .../calcite/rel/rules/JoinExtractFilterRule.java   |  38 +-
 .../rel/rules/JoinProjectTransposeRule.java        | 251 ++++---
 .../calcite/rel/rules/JoinPushExpressionsRule.java |  47 +-
 .../calcite/rel/rules/JoinPushThroughJoinRule.java |  87 ++-
 .../rules/JoinPushTransitivePredicatesRule.java    |  66 +-
 .../calcite/rel/rules/JoinToCorrelateRule.java     |  71 +-
 .../calcite/rel/rules/JoinToMultiJoinRule.java     |  56 +-
 .../calcite/rel/rules/JoinUnionTransposeRule.java  |  65 +-
 .../calcite/rel/rules/LoptOptimizeJoinRule.java    |  42 +-
 .../org/apache/calcite/rel/rules/MatchRule.java    |  32 +-
 .../rel/rules/MaterializedViewFilterScanRule.java  |  61 +-
 .../rel/rules/MultiJoinOptimizeBushyRule.java      |  36 +-
 .../rel/rules/MultiJoinProjectTransposeRule.java   |  87 ++-
 .../calcite/rel/rules/ProjectCalcMergeRule.java    |  60 +-
 .../rel/rules/ProjectCorrelateTransposeRule.java   | 129 ++--
 .../rel/rules/ProjectFilterTransposeRule.java      | 163 +++--
 .../rel/rules/ProjectJoinJoinRemoveRule.java       |  67 +-
 .../calcite/rel/rules/ProjectJoinRemoveRule.java   |  56 +-
 .../rel/rules/ProjectJoinTransposeRule.java        | 124 ++--
 .../apache/calcite/rel/rules/ProjectMergeRule.java |  99 ++-
 .../rel/rules/ProjectMultiJoinMergeRule.java       |  61 +-
 .../calcite/rel/rules/ProjectRemoveRule.java       |  46 +-
 .../rel/rules/ProjectSetOpTransposeRule.java       | 101 +--
 .../rel/rules/ProjectSortTransposeRule.java        |  85 ---
 .../calcite/rel/rules/ProjectTableScanRule.java    |  77 +-
 .../calcite/rel/rules/ProjectToCalcRule.java       |  44 +-
 .../calcite/rel/rules/ProjectToWindowRule.java     | 119 ++--
 .../rel/rules/ProjectWindowTransposeRule.java      |  61 +-
 .../apache/calcite/rel/rules/PruneEmptyRules.java  | 425 ++++++-----
 .../calcite/rel/rules/ReduceDecimalsRule.java      | 104 +--
 .../calcite/rel/rules/ReduceExpressionsRule.java   | 286 +++++---
 .../rel/rules/SemiJoinFilterTransposeRule.java     |  55 +-
 .../rel/rules/SemiJoinJoinTransposeRule.java       |  91 ++-
 .../rel/rules/SemiJoinProjectTransposeRule.java    |  62 +-
 .../calcite/rel/rules/SemiJoinRemoveRule.java      |  45 +-
 .../org/apache/calcite/rel/rules/SemiJoinRule.java | 131 ++--
 .../apache/calcite/rel/rules/SortJoinCopyRule.java |  48 +-
 .../calcite/rel/rules/SortJoinTransposeRule.java   |  51 +-
 .../rel/rules/SortProjectTransposeRule.java        |  85 ++-
 .../rel/rules/SortRemoveConstantKeysRule.java      |  30 +-
 .../apache/calcite/rel/rules/SortRemoveRule.java   |  38 +-
 .../calcite/rel/rules/SortUnionTransposeRule.java  |  82 ++-
 .../calcite/rel/rules/SubQueryRemoveRule.java      | 241 ++++---
 .../apache/calcite/rel/rules/TableScanRule.java    |  35 +-
 .../calcite/rel/rules/UnionEliminatorRule.java     |  45 +-
 .../apache/calcite/rel/rules/UnionMergeRule.java   |  79 ++-
 .../rel/rules/UnionPullUpConstantsRule.java        |  50 +-
 .../calcite/rel/rules/UnionToDistinctRule.java     |  54 +-
 .../apache/calcite/rel/rules/ValuesReduceRule.java | 113 ++-
 .../materialize/MaterializedViewAggregateRule.java | 138 ++--
 .../materialize/MaterializedViewJoinRule.java      |  28 +-
 .../MaterializedViewOnlyAggregateRule.java         |  63 +-
 .../MaterializedViewOnlyFilterRule.java            |  40 +-
 .../materialize/MaterializedViewOnlyJoinRule.java  |  40 +-
 .../MaterializedViewProjectAggregateRule.java      |  67 +-
 .../MaterializedViewProjectFilterRule.java         |  44 +-
 .../MaterializedViewProjectJoinRule.java           |  44 +-
 .../rules/materialize/MaterializedViewRule.java    |  64 +-
 .../rules/materialize/MaterializedViewRules.java   |  34 +-
 .../org/apache/calcite/rel/stream/StreamRules.java | 293 +++++---
 .../calcite/runtime/ResultSetEnumerable.java       |   4 +-
 .../org/apache/calcite/sql/ddl/SqlDdlNodes.java    |   2 +-
 .../apache/calcite/sql2rel/RelDecorrelator.java    | 346 ++++++---
 .../java/org/apache/calcite/tools/Programs.java    |   3 +-
 .../org/apache/calcite/plan/RelOptUtilTest.java    | 102 ++-
 .../plan/volcano/CollationConversionTest.java      |  58 +-
 .../apache/calcite/plan/volcano/ComboRuleTest.java |  60 +-
 .../apache/calcite/plan/volcano/PlannerTests.java  |  73 +-
 .../calcite/plan/volcano/TraitConversionTest.java  |  50 +-
 .../calcite/plan/volcano/TraitPropagationTest.java | 116 ++-
 .../calcite/plan/volcano/VolcanoPlannerTest.java   | 309 +++++---
 .../plan/volcano/VolcanoPlannerTraitTest.java      | 182 +++--
 .../calcite/rel/rel2sql/RelToSqlConverterTest.java |  10 +-
 .../calcite/rel/rules/SortRemoveRuleTest.java      |   3 +-
 .../org/apache/calcite/test/HepPlannerTest.java    |  36 +-
 .../java/org/apache/calcite/test/LatticeTest.java  |  10 +-
 .../org/apache/calcite/test/MockRelOptPlanner.java |   3 +-
 .../org/apache/calcite/test/MutableRelTest.java    |   4 +-
 .../org/apache/calcite/test/RelOptRulesTest.java   | 789 ++++++++++++---------
 .../apache/calcite/test/SqlHintsConverterTest.java |  60 +-
 .../org/apache/calcite/tools/FrameworksTest.java   |  31 +-
 .../java/org/apache/calcite/tools/PlannerTest.java | 167 +++--
 .../apache/calcite/adapter/druid/DruidRules.java   | 452 +++++-------
 .../adapter/elasticsearch/ElasticsearchRules.java  |  64 +-
 .../ElasticsearchToEnumerableConverterRule.java    |  26 +-
 .../adapter/csv/CsvProjectTableScanRule.java       |  39 +-
 .../org/apache/calcite/adapter/csv/CsvRules.java   |   4 +-
 .../calcite/adapter/geode/rel/GeodeRules.java      | 104 ++-
 .../geode/rel/GeodeToEnumerableConverterRule.java  |  12 +-
 .../apache/calcite/adapter/mongodb/MongoRules.java |  61 +-
 .../mongodb/MongoToEnumerableConverterRule.java    |  24 +-
 .../org/apache/calcite/adapter/pig/PigRules.java   |  74 +-
 .../adapter/pig/PigToEnumerableConverterRule.java  |  12 +-
 .../calcite/test/PigRelBuilderStyleTest.java       |  23 +-
 .../org/apache/calcite/piglet/PigConverter.java    |   3 +-
 .../calcite/piglet/PigToSqlAggregateRule.java      |  97 +--
 site/_docs/howto.md                                |  73 ++
 site/_docs/tutorial.md                             |  50 +-
 .../spark/EnumerableToSparkConverterRule.java      |  56 --
 .../adapter/spark/JdbcToSparkConverterRule.java    |  20 +-
 .../apache/calcite/adapter/spark/SparkRules.java   |  98 ++-
 .../calcite/adapter/splunk/SplunkPushDownRule.java |  96 ++-
 198 files changed, 9018 insertions(+), 5503 deletions(-)

diff --git a/cassandra/src/main/java/org/apache/calcite/adapter/cassandra/CassandraRules.java b/cassandra/src/main/java/org/apache/calcite/adapter/cassandra/CassandraRules.java
index 04a2d54..6aad2a5 100644
--- a/cassandra/src/main/java/org/apache/calcite/adapter/cassandra/CassandraRules.java
+++ b/cassandra/src/main/java/org/apache/calcite/adapter/cassandra/CassandraRules.java
@@ -22,15 +22,14 @@ import org.apache.calcite.adapter.java.JavaTypeFactory;
 import org.apache.calcite.plan.Convention;
 import org.apache.calcite.plan.RelOptRule;
 import org.apache.calcite.plan.RelOptRuleCall;
-import org.apache.calcite.plan.RelOptRuleOperand;
 import org.apache.calcite.plan.RelOptUtil;
+import org.apache.calcite.plan.RelRule;
 import org.apache.calcite.plan.RelTraitSet;
 import org.apache.calcite.rel.RelCollation;
 import org.apache.calcite.rel.RelCollations;
 import org.apache.calcite.rel.RelFieldCollation;
 import org.apache.calcite.rel.RelNode;
 import org.apache.calcite.rel.convert.ConverterRule;
-import org.apache.calcite.rel.core.RelFactories;
 import org.apache.calcite.rel.core.Sort;
 import org.apache.calcite.rel.logical.LogicalFilter;
 import org.apache.calcite.rel.logical.LogicalProject;
@@ -54,18 +53,29 @@ import java.util.function.Predicate;
  * calling convention.
  */
 public class CassandraRules {
+
   private CassandraRules() {}
 
+  public static final CassandraFilterRule FILTER =
+      CassandraFilterRule.Config.DEFAULT.toRule();
+  public static final CassandraProjectRule PROJECT =
+      CassandraProjectRule.DEFAULT_CONFIG.toRule(CassandraProjectRule.class);
+  public static final CassandraSortRule SORT =
+      CassandraSortRule.Config.DEFAULT.toRule();
+  public static final CassandraLimitRule LIMIT =
+      CassandraLimitRule.Config.DEFAULT.toRule();
+
   /** Rule to convert a relational expression from
    * {@link CassandraRel#CONVENTION} to {@link EnumerableConvention}. */
-  public static final ConverterRule TO_ENUMERABLE =
-      new CassandraToEnumerableConverterRule(RelFactories.LOGICAL_BUILDER);
+  public static final CassandraToEnumerableConverterRule TO_ENUMERABLE =
+      CassandraToEnumerableConverterRule.DEFAULT_CONFIG
+          .toRule(CassandraToEnumerableConverterRule.class);
 
   public static final RelOptRule[] RULES = {
-      CassandraFilterRule.INSTANCE,
-      CassandraProjectRule.INSTANCE,
-      CassandraSortRule.INSTANCE,
-      CassandraLimitRule.INSTANCE
+      FILTER,
+      PROJECT,
+      SORT,
+      LIMIT
   };
 
   static List<String> cassandraFieldNames(final RelDataType rowType) {
@@ -94,39 +104,29 @@ public class CassandraRules {
   /** Base class for planner rules that convert a relational expression to
    * Cassandra calling convention. */
   abstract static class CassandraConverterRule extends ConverterRule {
-    protected final Convention out;
-
-    CassandraConverterRule(Class<? extends RelNode> clazz,
-        String description) {
-      this(clazz, r -> true, description);
-    }
-
-    <R extends RelNode> CassandraConverterRule(Class<R> clazz,
-        Predicate<? super R> predicate,
-        String description) {
-      super(clazz, predicate, Convention.NONE,
-          CassandraRel.CONVENTION, RelFactories.LOGICAL_BUILDER, description);
-      this.out = CassandraRel.CONVENTION;
+    CassandraConverterRule(Config config) {
+      super(config);
     }
   }
 
   /**
    * Rule to convert a {@link org.apache.calcite.rel.logical.LogicalFilter} to a
    * {@link CassandraFilter}.
+   *
+   * @see #FILTER
    */
-  private static class CassandraFilterRule extends RelOptRule {
+  public static class CassandraFilterRule
+      extends RelRule<CassandraFilterRule.Config> {
+    /** Creates a CassandraFilterRule. */
+    protected CassandraFilterRule(Config config) {
+      super(config);
+    }
+
     private static final Predicate<LogicalFilter> PREDICATE =
         // TODO: Check for an equality predicate on the partition key
         // Right now this just checks if we have a single top-level AND
         filter -> RelOptUtil.disjunctions(filter.getCondition()).size() == 1;
 
-    private static final CassandraFilterRule INSTANCE = new CassandraFilterRule();
-
-    private CassandraFilterRule() {
-      super(operand(LogicalFilter.class, operand(CassandraTableScan.class, none())),
-          "CassandraFilterRule");
-    }
-
     @Override public boolean matches(RelOptRuleCall call) {
       // Get the condition from the filter operation
       LogicalFilter filter = call.rel(0);
@@ -206,7 +206,7 @@ public class CassandraRules {
     }
 
     /** @see org.apache.calcite.rel.convert.ConverterRule */
-    public void onMatch(RelOptRuleCall call) {
+    @Override public void onMatch(RelOptRuleCall call) {
       LogicalFilter filter = call.rel(0);
       CassandraTableScan scan = call.rel(1);
       if (filter.getTraitSet().contains(Convention.NONE)) {
@@ -217,7 +217,7 @@ public class CassandraRules {
       }
     }
 
-    public RelNode convert(LogicalFilter filter, CassandraTableScan scan) {
+    RelNode convert(LogicalFilter filter, CassandraTableScan scan) {
       final RelTraitSet traitSet = filter.getTraitSet().replace(CassandraRel.CONVENTION);
       final Pair<List<String>, List<String>> keyFields = scan.cassandraTable.getKeyFields();
       return new CassandraFilter(
@@ -229,17 +229,37 @@ public class CassandraRules {
           keyFields.right,
           scan.cassandraTable.getClusteringOrder());
     }
+
+    /** Rule configuration. */
+    public interface Config extends RelRule.Config {
+      Config DEFAULT = EMPTY
+          .withOperandSupplier(b0 ->
+              b0.operand(LogicalFilter.class)
+                  .oneInput(b1 -> b1.operand(CassandraTableScan.class)
+                      .noInputs()))
+          .as(Config.class);
+
+      @Override default CassandraFilterRule toRule() {
+        return new CassandraFilterRule(this);
+      }
+    }
   }
 
   /**
    * Rule to convert a {@link org.apache.calcite.rel.logical.LogicalProject}
    * to a {@link CassandraProject}.
+   *
+   * @see #PROJECT
    */
-  private static class CassandraProjectRule extends CassandraConverterRule {
-    private static final CassandraProjectRule INSTANCE = new CassandraProjectRule();
-
-    private CassandraProjectRule() {
-      super(LogicalProject.class, "CassandraProjectRule");
+  public static class CassandraProjectRule extends CassandraConverterRule {
+    /** Default configuration. */
+    private static final Config DEFAULT_CONFIG = Config.INSTANCE
+        .withConversion(LogicalProject.class, Convention.NONE,
+            CassandraRel.CONVENTION, "CassandraProjectRule")
+        .withRuleFactory(CassandraProjectRule::new);
+
+    protected CassandraProjectRule(Config config) {
+      super(config);
     }
 
     @Override public boolean matches(RelOptRuleCall call) {
@@ -253,7 +273,7 @@ public class CassandraRules {
       return true;
     }
 
-    public RelNode convert(RelNode rel) {
+    @Override public RelNode convert(RelNode rel) {
       final LogicalProject project = (LogicalProject) rel;
       final RelTraitSet traitSet = project.getTraitSet().replace(out);
       return new CassandraProject(project.getCluster(), traitSet,
@@ -265,23 +285,14 @@ public class CassandraRules {
   /**
    * Rule to convert a {@link org.apache.calcite.rel.core.Sort} to a
    * {@link CassandraSort}.
+   *
+   * @see #SORT
    */
-  private static class CassandraSortRule extends RelOptRule {
-
-    private static final RelOptRuleOperand CASSANDRA_OP =
-        operand(CassandraToEnumerableConverter.class,
-            operandJ(CassandraFilter.class, null,
-                // We can only use implicit sorting within a single partition
-                CassandraFilter::isSinglePartition, any()));
-
-    private static final CassandraSortRule INSTANCE = new CassandraSortRule();
-
-    private CassandraSortRule() {
-      super(
-          operandJ(Sort.class, null,
-              // Limits are handled by CassandraLimit
-              sort -> sort.offset == null && sort.fetch == null, CASSANDRA_OP),
-          "CassandraSortRule");
+  public static class CassandraSortRule
+      extends RelRule<CassandraSortRule.Config> {
+    /** Creates a CassandraSortRule. */
+    protected CassandraSortRule(Config config) {
+      super(config);
     }
 
     public RelNode convert(Sort sort, CassandraFilter filter) {
@@ -293,7 +304,7 @@ public class CassandraRules {
           sort.getCollation());
     }
 
-    public boolean matches(RelOptRuleCall call) {
+    @Override public boolean matches(RelOptRuleCall call) {
       final Sort sort = call.rel(0);
       final CassandraFilter filter = call.rel(2);
       return collationsCompatible(sort.getCollation(), filter.getImplicitCollation());
@@ -360,7 +371,7 @@ public class CassandraRules {
     }
 
     /** @see org.apache.calcite.rel.convert.ConverterRule */
-    public void onMatch(RelOptRuleCall call) {
+    @Override public void onMatch(RelOptRuleCall call) {
       final Sort sort = call.rel(0);
       CassandraFilter filter = call.rel(2);
       final RelNode converted = convert(sort, filter);
@@ -368,18 +379,44 @@ public class CassandraRules {
         call.transformTo(converted);
       }
     }
+
+    /** Rule configuration. */
+    public interface Config extends RelRule.Config {
+      Config DEFAULT = EMPTY
+          .withOperandSupplier(b0 ->
+              b0.operand(Sort.class)
+                  // Limits are handled by CassandraLimit
+                  .predicate(sort ->
+                      sort.offset == null && sort.fetch == null)
+                  .oneInput(b1 ->
+                      b1.operand(CassandraToEnumerableConverter.class)
+                          .oneInput(b2 ->
+                              b2.operand(CassandraFilter.class)
+                                  // We can only use implicit sorting within a
+                                  // single partition
+                                  .predicate(
+                                      CassandraFilter::isSinglePartition)
+                                  .anyInputs())))
+          .as(Config.class);
+
+      @Override default CassandraSortRule toRule() {
+        return new CassandraSortRule(this);
+      }
+    }
   }
 
   /**
-   * Rule to convert a {@link org.apache.calcite.adapter.enumerable.EnumerableLimit} to a
+   * Rule to convert a
+   * {@link org.apache.calcite.adapter.enumerable.EnumerableLimit} to a
    * {@link CassandraLimit}.
+   *
+   * @see #LIMIT
    */
-  private static class CassandraLimitRule extends RelOptRule {
-    private static final CassandraLimitRule INSTANCE = new CassandraLimitRule();
-
-    private CassandraLimitRule() {
-      super(operand(EnumerableLimit.class, operand(CassandraToEnumerableConverter.class, any())),
-          "CassandraLimitRule");
+  public static class CassandraLimitRule
+      extends RelRule<CassandraLimitRule.Config> {
+    /** Creates a CassandraLimitRule. */
+    protected CassandraLimitRule(Config config) {
+      super(config);
     }
 
     public RelNode convert(EnumerableLimit limit) {
@@ -390,12 +427,27 @@ public class CassandraRules {
     }
 
     /** @see org.apache.calcite.rel.convert.ConverterRule */
-    public void onMatch(RelOptRuleCall call) {
+    @Override public void onMatch(RelOptRuleCall call) {
       final EnumerableLimit limit = call.rel(0);
       final RelNode converted = convert(limit);
       if (converted != null) {
         call.transformTo(converted);
       }
     }
+
+    /** Rule configuration. */
+    public interface Config extends RelRule.Config {
+      Config DEFAULT = EMPTY
+          .withOperandSupplier(b0 ->
+              b0.operand(EnumerableLimit.class)
+                  .oneInput(b1 ->
+                      b1.operand(CassandraToEnumerableConverter.class)
+                          .anyInputs()))
+          .as(Config.class);
+
+      @Override default CassandraLimitRule toRule() {
+        return new CassandraLimitRule(this);
+      }
+    }
   }
 }
diff --git a/cassandra/src/main/java/org/apache/calcite/adapter/cassandra/CassandraToEnumerableConverterRule.java b/cassandra/src/main/java/org/apache/calcite/adapter/cassandra/CassandraToEnumerableConverterRule.java
index 523ecef..3c3fe25 100644
--- a/cassandra/src/main/java/org/apache/calcite/adapter/cassandra/CassandraToEnumerableConverterRule.java
+++ b/cassandra/src/main/java/org/apache/calcite/adapter/cassandra/CassandraToEnumerableConverterRule.java
@@ -20,30 +20,23 @@ import org.apache.calcite.adapter.enumerable.EnumerableConvention;
 import org.apache.calcite.plan.RelTraitSet;
 import org.apache.calcite.rel.RelNode;
 import org.apache.calcite.rel.convert.ConverterRule;
-import org.apache.calcite.tools.RelBuilderFactory;
-
-import java.util.function.Predicate;
 
 /**
  * Rule to convert a relational expression from
  * {@link CassandraRel#CONVENTION} to {@link EnumerableConvention}.
+ *
+ * @see CassandraRules#TO_ENUMERABLE
  */
 public class CassandraToEnumerableConverterRule extends ConverterRule {
-  /** @deprecated Use {@link CassandraRules#TO_ENUMERABLE}. */
-  @Deprecated // to be removed before 1.25
-  public static final ConverterRule INSTANCE =
-      CassandraRules.TO_ENUMERABLE;
+  /** Default configuration. */
+  public static final Config DEFAULT_CONFIG = Config.INSTANCE
+      .withConversion(RelNode.class, CassandraRel.CONVENTION,
+          EnumerableConvention.INSTANCE, "CassandraToEnumerableConverterRule")
+      .withRuleFactory(CassandraToEnumerableConverterRule::new);
 
-  /**
-   * Creates a CassandraToEnumerableConverterRule.
-   *
-   * @param relBuilderFactory Builder for relational expressions
-   */
-  public CassandraToEnumerableConverterRule(
-      RelBuilderFactory relBuilderFactory) {
-    super(RelNode.class, (Predicate<RelNode>) r -> true,
-        CassandraRel.CONVENTION, EnumerableConvention.INSTANCE,
-        relBuilderFactory, "CassandraToEnumerableConverterRule");
+  /** Creates a CassandraToEnumerableConverterRule. */
+  protected CassandraToEnumerableConverterRule(Config config) {
+    super(config);
   }
 
   @Override public RelNode convert(RelNode rel) {
diff --git a/core/src/main/java/org/apache/calcite/adapter/enumerable/EnumerableAggregateRule.java b/core/src/main/java/org/apache/calcite/adapter/enumerable/EnumerableAggregateRule.java
index a34a63a..78612aa 100644
--- a/core/src/main/java/org/apache/calcite/adapter/enumerable/EnumerableAggregateRule.java
+++ b/core/src/main/java/org/apache/calcite/adapter/enumerable/EnumerableAggregateRule.java
@@ -26,14 +26,22 @@ import org.apache.calcite.rel.logical.LogicalAggregate;
 /**
  * Rule to convert a {@link org.apache.calcite.rel.logical.LogicalAggregate}
  * to an {@link EnumerableAggregate}.
+ *
+ * @see EnumerableRules#ENUMERABLE_AGGREGATE_RULE
  */
 class EnumerableAggregateRule extends ConverterRule {
-  EnumerableAggregateRule() {
-    super(LogicalAggregate.class, Convention.NONE,
-        EnumerableConvention.INSTANCE, "EnumerableAggregateRule");
+  /** Default configuration. */
+  static final Config DEFAULT_CONFIG = Config.INSTANCE
+      .withConversion(LogicalAggregate.class, Convention.NONE,
+          EnumerableConvention.INSTANCE, "EnumerableAggregateRule")
+      .withRuleFactory(EnumerableAggregateRule::new);
+
+  /** Called from the Config. */
+  protected EnumerableAggregateRule(Config config) {
+    super(config);
   }
 
-  public RelNode convert(RelNode rel) {
+  @Override public RelNode convert(RelNode rel) {
     final LogicalAggregate agg = (LogicalAggregate) rel;
     final RelTraitSet traitSet = rel.getCluster()
         .traitSet().replace(EnumerableConvention.INSTANCE);
diff --git a/core/src/main/java/org/apache/calcite/adapter/enumerable/EnumerableBatchNestedLoopJoinRule.java b/core/src/main/java/org/apache/calcite/adapter/enumerable/EnumerableBatchNestedLoopJoinRule.java
index 09072b6..eefa7df 100644
--- a/core/src/main/java/org/apache/calcite/adapter/enumerable/EnumerableBatchNestedLoopJoinRule.java
+++ b/core/src/main/java/org/apache/calcite/adapter/enumerable/EnumerableBatchNestedLoopJoinRule.java
@@ -17,8 +17,8 @@
 package org.apache.calcite.adapter.enumerable;
 
 import org.apache.calcite.plan.RelOptCluster;
-import org.apache.calcite.plan.RelOptRule;
 import org.apache.calcite.plan.RelOptRuleCall;
+import org.apache.calcite.plan.RelRule;
 import org.apache.calcite.rel.RelNode;
 import org.apache.calcite.rel.core.CorrelationId;
 import org.apache.calcite.rel.core.Join;
@@ -31,6 +31,7 @@ import org.apache.calcite.rex.RexNode;
 import org.apache.calcite.rex.RexShuttle;
 import org.apache.calcite.tools.RelBuilder;
 import org.apache.calcite.tools.RelBuilderFactory;
+import org.apache.calcite.util.ImmutableBeans;
 import org.apache.calcite.util.ImmutableBitSet;
 
 import java.util.ArrayList;
@@ -40,29 +41,38 @@ import java.util.Set;
 
 /** Planner rule that converts a
  * {@link org.apache.calcite.rel.logical.LogicalJoin} into an
- * {@link org.apache.calcite.adapter.enumerable.EnumerableBatchNestedLoopJoin}. */
-public class EnumerableBatchNestedLoopJoinRule extends RelOptRule {
-
-  private final int batchSize;
-  private static final int DEFAULT_BATCH_SIZE = 100;
-
+ * {@link org.apache.calcite.adapter.enumerable.EnumerableBatchNestedLoopJoin}.
+ *
+ * @see EnumerableRules#ENUMERABLE_BATCH_NESTED_LOOP_JOIN_RULE
+ */
+public class EnumerableBatchNestedLoopJoinRule
+    extends RelRule<EnumerableBatchNestedLoopJoinRule.Config> {
   /** Creates an EnumerableBatchNestedLoopJoinRule. */
+  protected EnumerableBatchNestedLoopJoinRule(Config config) {
+    super(config);
+  }
+
+  @Deprecated // to be removed before 2.0
   protected EnumerableBatchNestedLoopJoinRule(Class<? extends Join> clazz,
       RelBuilderFactory relBuilderFactory, int batchSize) {
-    super(operand(clazz, any()),
-        relBuilderFactory, "EnumerableBatchNestedLoopJoinRule");
-    this.batchSize = batchSize;
+    this(Config.DEFAULT.withRelBuilderFactory(relBuilderFactory)
+        .withOperandSupplier(b -> b.operand(clazz).anyInputs())
+        .as(Config.class)
+        .withBatchSize(batchSize));
   }
-  /** Creates an EnumerableBatchNestedLoopJoinRule with default batch size of 100. */
+
+  @Deprecated // to be removed before 2.0
   public EnumerableBatchNestedLoopJoinRule(RelBuilderFactory relBuilderFactory) {
-    this(LogicalJoin.class, relBuilderFactory, DEFAULT_BATCH_SIZE);
+    this(Config.DEFAULT.withRelBuilderFactory(relBuilderFactory)
+        .as(Config.class));
   }
 
-  /** Creates an EnumerableBatchNestedLoopJoinRule with given batch size.
-   * Warning: if the batch size is around or bigger than 1000 there
-   * can be an error because the generated code exceeds the size limit */
-  public EnumerableBatchNestedLoopJoinRule(RelBuilderFactory relBuilderFactory, int batchSize) {
-    this(LogicalJoin.class, relBuilderFactory, batchSize);
+  @Deprecated // to be removed before 2.0
+  public EnumerableBatchNestedLoopJoinRule(RelBuilderFactory relBuilderFactory,
+      int batchSize) {
+    this(Config.DEFAULT.withRelBuilderFactory(relBuilderFactory)
+        .as(Config.class)
+        .withBatchSize(batchSize));
   }
 
   @Override public boolean matches(RelOptRuleCall call) {
@@ -84,6 +94,7 @@ public class EnumerableBatchNestedLoopJoinRule extends RelOptRule {
     final Set<CorrelationId> correlationIds = new HashSet<>();
     final ArrayList<RexNode> corrVar = new ArrayList<>();
 
+    final int batchSize = config.batchSize();
     for (int i = 0; i < batchSize; i++) {
       CorrelationId correlationId = cluster.createCorrel();
       correlationIds.add(correlationId);
@@ -137,4 +148,27 @@ public class EnumerableBatchNestedLoopJoinRule extends RelOptRule {
             correlationIds,
             joinType));
   }
+
+  /** Rule configuration. */
+  public interface Config extends RelRule.Config {
+    Config DEFAULT = EMPTY
+        .withOperandSupplier(b -> b.operand(LogicalJoin.class).anyInputs())
+        .withDescription("EnumerableBatchNestedLoopJoinRule")
+        .as(Config.class);
+
+    @Override default EnumerableBatchNestedLoopJoinRule toRule() {
+      return new EnumerableBatchNestedLoopJoinRule(this);
+    }
+
+    /** Batch size.
+     *
+     * <p>Warning: if the batch size is around or bigger than 1000 there
+     * can be an error because the generated code exceeds the size limit. */
+    @ImmutableBeans.Property
+    @ImmutableBeans.IntDefault(100)
+    int batchSize();
+
+    /** Sets {@link #batchSize()}. */
+    Config withBatchSize(int batchSize);
+  }
 }
diff --git a/core/src/main/java/org/apache/calcite/adapter/enumerable/EnumerableBindable.java b/core/src/main/java/org/apache/calcite/adapter/enumerable/EnumerableBindable.java
index 61714e0..fc2223f 100644
--- a/core/src/main/java/org/apache/calcite/adapter/enumerable/EnumerableBindable.java
+++ b/core/src/main/java/org/apache/calcite/adapter/enumerable/EnumerableBindable.java
@@ -32,12 +32,10 @@ import org.apache.calcite.rel.convert.ConverterImpl;
 import org.apache.calcite.rel.convert.ConverterRule;
 import org.apache.calcite.runtime.ArrayBindable;
 import org.apache.calcite.runtime.Bindable;
-import org.apache.calcite.tools.RelBuilderFactory;
 
 import com.google.common.collect.ImmutableMap;
 
 import java.util.List;
-import java.util.function.Predicate;
 
 /**
  * Relational expression that converts an enumerable input to interpretable
@@ -83,23 +81,19 @@ public class EnumerableBindable extends ConverterImpl implements BindableRel {
 
   /**
    * Rule that converts any enumerable relational expression to bindable.
+   *
+   * @see EnumerableRules#TO_BINDABLE
    */
   public static class EnumerableToBindableConverterRule extends ConverterRule {
-    /** @deprecated Use {@link EnumerableRules#TO_BINDABLE}. */
-    @Deprecated // to be removed before 1.25
-    public static final EnumerableToBindableConverterRule INSTANCE =
-        EnumerableRules.TO_BINDABLE;
+    /** Default configuration. */
+    public static final Config DEFAULT_CONFIG = Config.INSTANCE
+        .withConversion(EnumerableRel.class,
+            EnumerableConvention.INSTANCE, BindableConvention.INSTANCE,
+            "EnumerableToBindableConverterRule")
+        .withRuleFactory(EnumerableToBindableConverterRule::new);
 
-    /**
-     * Creates an EnumerableToBindableConverterRule.
-     *
-     * @param relBuilderFactory Builder for relational expressions
-     */
-    public EnumerableToBindableConverterRule(
-        RelBuilderFactory relBuilderFactory) {
-      super(EnumerableRel.class, (Predicate<RelNode>) r -> true,
-          EnumerableConvention.INSTANCE, BindableConvention.INSTANCE,
-          relBuilderFactory, "EnumerableToBindableConverterRule");
+    protected EnumerableToBindableConverterRule(Config config) {
+      super(config);
     }
 
     @Override public RelNode convert(RelNode rel) {
diff --git a/core/src/main/java/org/apache/calcite/adapter/enumerable/EnumerableCalcRule.java b/core/src/main/java/org/apache/calcite/adapter/enumerable/EnumerableCalcRule.java
index 07bbfdc..d8c5fe5 100644
--- a/core/src/main/java/org/apache/calcite/adapter/enumerable/EnumerableCalcRule.java
+++ b/core/src/main/java/org/apache/calcite/adapter/enumerable/EnumerableCalcRule.java
@@ -20,27 +20,31 @@ import org.apache.calcite.plan.Convention;
 import org.apache.calcite.plan.RelOptUtil;
 import org.apache.calcite.rel.RelNode;
 import org.apache.calcite.rel.convert.ConverterRule;
-import org.apache.calcite.rel.core.Calc;
-import org.apache.calcite.rel.core.RelFactories;
 import org.apache.calcite.rel.logical.LogicalCalc;
 
-import java.util.function.Predicate;
 
 /**
  * Rule to convert a {@link org.apache.calcite.rel.logical.LogicalCalc} to an
  * {@link EnumerableCalc}.
+ *
+ * @see EnumerableRules#ENUMERABLE_CALC_RULE
  */
 class EnumerableCalcRule extends ConverterRule {
-  EnumerableCalcRule() {
-    // The predicate ensures that if there's a multiset, FarragoMultisetSplitter
-    // will work on it first.
-    super(LogicalCalc.class,
-        (Predicate<Calc>) RelOptUtil::notContainsWindowedAgg,
-        Convention.NONE, EnumerableConvention.INSTANCE,
-        RelFactories.LOGICAL_BUILDER, "EnumerableCalcRule");
+  /** Default configuration. */
+  public static final Config DEFAULT_CONFIG = Config.EMPTY
+      .as(Config.class)
+      // The predicate ensures that if there's a multiset,
+      // FarragoMultisetSplitter will work on it first.
+      .withConversion(LogicalCalc.class, RelOptUtil::notContainsWindowedAgg,
+          Convention.NONE, EnumerableConvention.INSTANCE,
+          "EnumerableCalcRule")
+      .withRuleFactory(EnumerableCalcRule::new);
+
+  protected EnumerableCalcRule(Config config) {
+    super(config);
   }
 
-  public RelNode convert(RelNode rel) {
+  @Override public RelNode convert(RelNode rel) {
     final LogicalCalc calc = (LogicalCalc) rel;
     final RelNode input = calc.getInput();
     return EnumerableCalc.create(
diff --git a/core/src/main/java/org/apache/calcite/adapter/enumerable/EnumerableCollectRule.java b/core/src/main/java/org/apache/calcite/adapter/enumerable/EnumerableCollectRule.java
index b1a0984..d7b486a 100644
--- a/core/src/main/java/org/apache/calcite/adapter/enumerable/EnumerableCollectRule.java
+++ b/core/src/main/java/org/apache/calcite/adapter/enumerable/EnumerableCollectRule.java
@@ -25,14 +25,22 @@ import org.apache.calcite.rel.core.Collect;
 /**
  * Rule to convert an {@link org.apache.calcite.rel.core.Collect} to an
  * {@link EnumerableCollect}.
+ *
+ * @see EnumerableRules#ENUMERABLE_COLLECT_RULE
  */
 class EnumerableCollectRule extends ConverterRule {
-  EnumerableCollectRule() {
-    super(Collect.class, Convention.NONE, EnumerableConvention.INSTANCE,
-        "EnumerableCollectRule");
+  /** Default configuration. */
+  public static final Config DEFAULT_CONFIG = Config.INSTANCE
+      .withConversion(Collect.class, Convention.NONE,
+          EnumerableConvention.INSTANCE, "EnumerableCollectRule")
+      .withRuleFactory(EnumerableCollectRule::new);
+
+  /** Called from the Config. */
+  protected EnumerableCollectRule(Config config) {
+    super(config);
   }
 
-  public RelNode convert(RelNode rel) {
+  @Override public RelNode convert(RelNode rel) {
     final Collect collect = (Collect) rel;
     final RelTraitSet traitSet =
         collect.getTraitSet().replace(EnumerableConvention.INSTANCE);
diff --git a/core/src/main/java/org/apache/calcite/adapter/enumerable/EnumerableCorrelateRule.java b/core/src/main/java/org/apache/calcite/adapter/enumerable/EnumerableCorrelateRule.java
index ea6ad243..fd1456c 100644
--- a/core/src/main/java/org/apache/calcite/adapter/enumerable/EnumerableCorrelateRule.java
+++ b/core/src/main/java/org/apache/calcite/adapter/enumerable/EnumerableCorrelateRule.java
@@ -20,26 +20,26 @@ import org.apache.calcite.plan.Convention;
 import org.apache.calcite.rel.RelNode;
 import org.apache.calcite.rel.convert.ConverterRule;
 import org.apache.calcite.rel.logical.LogicalCorrelate;
-import org.apache.calcite.tools.RelBuilderFactory;
-
-import java.util.function.Predicate;
 
 /**
  * Implementation of nested loops over enumerable inputs.
+ *
+ * @see EnumerableRules#ENUMERABLE_CORRELATE_RULE
  */
 public class EnumerableCorrelateRule extends ConverterRule {
-  /**
-   * Creates an EnumerableCorrelateRule.
-   *
-   * @param relBuilderFactory Builder for relational expressions
-   */
-  public EnumerableCorrelateRule(RelBuilderFactory relBuilderFactory) {
-    super(LogicalCorrelate.class, (Predicate<RelNode>) r -> true,
-        Convention.NONE, EnumerableConvention.INSTANCE, relBuilderFactory,
-        "EnumerableCorrelateRule");
+  /** Default configuration. */
+  public static final Config DEFAULT_CONFIG = Config.EMPTY
+      .as(Config.class)
+      .withConversion(LogicalCorrelate.class, r -> true, Convention.NONE,
+          EnumerableConvention.INSTANCE, "EnumerableCorrelateRule")
+      .withRuleFactory(EnumerableCorrelateRule::new);
+
+  /** Creates an EnumerableCorrelateRule. */
+  protected EnumerableCorrelateRule(Config config) {
+    super(config);
   }
 
-  public RelNode convert(RelNode rel) {
+  @Override public RelNode convert(RelNode rel) {
     final LogicalCorrelate c = (LogicalCorrelate) rel;
     return EnumerableCorrelate.create(
         convert(c.getLeft(), c.getLeft().getTraitSet()
diff --git a/core/src/main/java/org/apache/calcite/adapter/enumerable/EnumerableFilterRule.java b/core/src/main/java/org/apache/calcite/adapter/enumerable/EnumerableFilterRule.java
index df73609..4f1df08 100644
--- a/core/src/main/java/org/apache/calcite/adapter/enumerable/EnumerableFilterRule.java
+++ b/core/src/main/java/org/apache/calcite/adapter/enumerable/EnumerableFilterRule.java
@@ -17,27 +17,30 @@
 package org.apache.calcite.adapter.enumerable;
 
 import org.apache.calcite.plan.Convention;
-import org.apache.calcite.plan.RelOptUtil;
 import org.apache.calcite.rel.RelNode;
 import org.apache.calcite.rel.convert.ConverterRule;
-import org.apache.calcite.rel.core.RelFactories;
 import org.apache.calcite.rel.logical.LogicalFilter;
 
-import java.util.function.Predicate;
-
 /**
  * Rule to convert a {@link org.apache.calcite.rel.logical.LogicalFilter} to an
  * {@link EnumerableFilter}.
+ *
+ * @see EnumerableRules#ENUMERABLE_FILTER_RULE
  */
 class EnumerableFilterRule extends ConverterRule {
-  EnumerableFilterRule() {
-    super(LogicalFilter.class,
-        (Predicate<LogicalFilter>) RelOptUtil::notContainsWindowedAgg,
-        Convention.NONE, EnumerableConvention.INSTANCE,
-        RelFactories.LOGICAL_BUILDER, "EnumerableFilterRule");
+  /** Default configuration. */
+  public static final Config DEFAULT_CONFIG = Config.EMPTY
+      .as(Config.class)
+      .withConversion(LogicalFilter.class, f -> !f.containsOver(),
+          Convention.NONE, EnumerableConvention.INSTANCE,
+          "EnumerableFilterRule")
+      .withRuleFactory(EnumerableFilterRule::new);
+
+  protected EnumerableFilterRule(Config config) {
+    super(config);
   }
 
-  public RelNode convert(RelNode rel) {
+  @Override public RelNode convert(RelNode rel) {
     final LogicalFilter filter = (LogicalFilter) rel;
     return new EnumerableFilter(rel.getCluster(),
         rel.getTraitSet().replace(EnumerableConvention.INSTANCE),
diff --git a/core/src/main/java/org/apache/calcite/adapter/enumerable/EnumerableFilterToCalcRule.java b/core/src/main/java/org/apache/calcite/adapter/enumerable/EnumerableFilterToCalcRule.java
index 8e05a47..db260a0 100644
--- a/core/src/main/java/org/apache/calcite/adapter/enumerable/EnumerableFilterToCalcRule.java
+++ b/core/src/main/java/org/apache/calcite/adapter/enumerable/EnumerableFilterToCalcRule.java
@@ -16,8 +16,8 @@
  */
 package org.apache.calcite.adapter.enumerable;
 
-import org.apache.calcite.plan.RelOptRule;
 import org.apache.calcite.plan.RelOptRuleCall;
+import org.apache.calcite.plan.RelRule;
 import org.apache.calcite.rel.RelNode;
 import org.apache.calcite.rel.type.RelDataType;
 import org.apache.calcite.rex.RexBuilder;
@@ -26,18 +26,23 @@ import org.apache.calcite.rex.RexProgramBuilder;
 import org.apache.calcite.tools.RelBuilderFactory;
 
 /** Variant of {@link org.apache.calcite.rel.rules.FilterToCalcRule} for
- * {@link org.apache.calcite.adapter.enumerable.EnumerableConvention enumerable calling convention}. */
-public class EnumerableFilterToCalcRule extends RelOptRule {
-  /**
-   * Creates an EnumerableFilterToCalcRule.
-   *
-   * @param relBuilderFactory Builder for relational expressions
-   */
+ * {@link org.apache.calcite.adapter.enumerable.EnumerableConvention enumerable calling convention}.
+ *
+ * @see EnumerableRules#ENUMERABLE_FILTER_TO_CALC_RULE */
+public class EnumerableFilterToCalcRule
+    extends RelRule<EnumerableFilterToCalcRule.Config> {
+  /** Creates an EnumerableFilterToCalcRule. */
+  protected EnumerableFilterToCalcRule(Config config) {
+    super(config);
+  }
+
+  @Deprecated // to be removed before 2.0
   public EnumerableFilterToCalcRule(RelBuilderFactory relBuilderFactory) {
-    super(operand(EnumerableFilter.class, any()), relBuilderFactory, null);
+    this(Config.DEFAULT.withRelBuilderFactory(relBuilderFactory)
+        .as(Config.class));
   }
 
-  public void onMatch(RelOptRuleCall call) {
+  @Override public void onMatch(RelOptRuleCall call) {
     final EnumerableFilter filter = call.rel(0);
     final RelNode input = filter.getInput();
 
@@ -53,4 +58,16 @@ public class EnumerableFilterToCalcRule extends RelOptRule {
     final EnumerableCalc calc = EnumerableCalc.create(input, program);
     call.transformTo(calc);
   }
+
+  /** Rule configuration. */
+  public interface Config extends RelRule.Config {
+    Config DEFAULT = EMPTY
+        .withOperandSupplier(b ->
+            b.operand(EnumerableFilter.class).anyInputs())
+        .as(Config.class);
+
+    @Override default EnumerableFilterToCalcRule toRule() {
+      return new EnumerableFilterToCalcRule(this);
+    }
+  }
 }
diff --git a/core/src/main/java/org/apache/calcite/adapter/enumerable/EnumerableInterpreterRule.java b/core/src/main/java/org/apache/calcite/adapter/enumerable/EnumerableInterpreterRule.java
index bb0b635..ccb21d5 100644
--- a/core/src/main/java/org/apache/calcite/adapter/enumerable/EnumerableInterpreterRule.java
+++ b/core/src/main/java/org/apache/calcite/adapter/enumerable/EnumerableInterpreterRule.java
@@ -19,30 +19,23 @@ package org.apache.calcite.adapter.enumerable;
 import org.apache.calcite.interpreter.BindableConvention;
 import org.apache.calcite.rel.RelNode;
 import org.apache.calcite.rel.convert.ConverterRule;
-import org.apache.calcite.tools.RelBuilderFactory;
-
-import java.util.function.Predicate;
 
 /**
  * Planner rule that converts {@link org.apache.calcite.interpreter.BindableRel}
  * to {@link org.apache.calcite.adapter.enumerable.EnumerableRel} by creating
  * an {@link org.apache.calcite.adapter.enumerable.EnumerableInterpreter}.
+ *
+ * @see EnumerableRules#TO_INTERPRETER
  */
 public class EnumerableInterpreterRule extends ConverterRule {
-  /** @deprecated Use {@link EnumerableRules#TO_INTERPRETER}. */
-  @Deprecated // to be removed before 1.25
-  public static final EnumerableInterpreterRule INSTANCE =
-      EnumerableRules.TO_INTERPRETER;
+  /** Default configuration. */
+  public static final Config DEFAULT_CONFIG = Config.INSTANCE
+      .withConversion(RelNode.class, BindableConvention.INSTANCE,
+          EnumerableConvention.INSTANCE, "EnumerableInterpreterRule")
+      .withRuleFactory(EnumerableInterpreterRule::new);
 
-  /**
-   * Creates an EnumerableInterpreterRule.
-   *
-   * @param relBuilderFactory Builder for relational expressions
-   */
-  public EnumerableInterpreterRule(RelBuilderFactory relBuilderFactory) {
-    super(RelNode.class, (Predicate<RelNode>) r -> true,
-        BindableConvention.INSTANCE, EnumerableConvention.INSTANCE,
-        relBuilderFactory, "EnumerableInterpreterRule");
+  protected EnumerableInterpreterRule(Config config) {
+    super(config);
   }
 
   //~ Methods ----------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/adapter/enumerable/EnumerableIntersectRule.java b/core/src/main/java/org/apache/calcite/adapter/enumerable/EnumerableIntersectRule.java
index 4bd6ad2..e5fadf5 100644
--- a/core/src/main/java/org/apache/calcite/adapter/enumerable/EnumerableIntersectRule.java
+++ b/core/src/main/java/org/apache/calcite/adapter/enumerable/EnumerableIntersectRule.java
@@ -26,14 +26,22 @@ import org.apache.calcite.rel.logical.LogicalIntersect;
  * Rule to convert a
  * {@link org.apache.calcite.rel.logical.LogicalIntersect} to an
  * {@link EnumerableIntersect}.
+ *
+ * @see EnumerableRules#ENUMERABLE_INTERSECT_RULE
  */
 class EnumerableIntersectRule extends ConverterRule {
-  EnumerableIntersectRule() {
-    super(LogicalIntersect.class, Convention.NONE,
-        EnumerableConvention.INSTANCE, "EnumerableIntersectRule");
+  /** Default configuration. */
+  public static final Config DEFAULT_CONFIG = Config.INSTANCE
+      .withConversion(LogicalIntersect.class, Convention.NONE,
+          EnumerableConvention.INSTANCE, "EnumerableIntersectRule")
+      .withRuleFactory(EnumerableIntersectRule::new);
+
+  /** Called from the Config. */
+  protected EnumerableIntersectRule(Config config) {
+    super(config);
   }
 
-  public RelNode convert(RelNode rel) {
+  @Override public RelNode convert(RelNode rel) {
     final LogicalIntersect intersect = (LogicalIntersect) rel;
     final EnumerableConvention out = EnumerableConvention.INSTANCE;
     final RelTraitSet traitSet = intersect.getTraitSet().replace(out);
diff --git a/core/src/main/java/org/apache/calcite/adapter/enumerable/EnumerableJoinRule.java b/core/src/main/java/org/apache/calcite/adapter/enumerable/EnumerableJoinRule.java
index fdbde84..a71bf00 100644
--- a/core/src/main/java/org/apache/calcite/adapter/enumerable/EnumerableJoinRule.java
+++ b/core/src/main/java/org/apache/calcite/adapter/enumerable/EnumerableJoinRule.java
@@ -31,14 +31,19 @@ import java.util.List;
 
 /** Planner rule that converts a
  * {@link org.apache.calcite.rel.logical.LogicalJoin} relational expression
- * {@link org.apache.calcite.adapter.enumerable.EnumerableConvention enumerable calling convention}. */
+ * {@link org.apache.calcite.adapter.enumerable.EnumerableConvention enumerable calling convention}.
+ *
+ * @see EnumerableRules#ENUMERABLE_JOIN_RULE */
 class EnumerableJoinRule extends ConverterRule {
-  EnumerableJoinRule() {
-    super(
-        LogicalJoin.class,
-        Convention.NONE,
-        EnumerableConvention.INSTANCE,
-        "EnumerableJoinRule");
+  /** Default configuration. */
+  public static final Config DEFAULT_CONFIG = Config.INSTANCE
+      .withConversion(LogicalJoin.class, Convention.NONE,
+          EnumerableConvention.INSTANCE, "EnumerableJoinRule")
+      .withRuleFactory(EnumerableJoinRule::new);
+
+  /** Called from the Config. */
+  protected EnumerableJoinRule(Config config) {
+    super(config);
   }
 
   @Override public RelNode convert(RelNode rel) {
diff --git a/core/src/main/java/org/apache/calcite/adapter/enumerable/EnumerableLimitRule.java b/core/src/main/java/org/apache/calcite/adapter/enumerable/EnumerableLimitRule.java
index 8440686..1a5292c 100644
--- a/core/src/main/java/org/apache/calcite/adapter/enumerable/EnumerableLimitRule.java
+++ b/core/src/main/java/org/apache/calcite/adapter/enumerable/EnumerableLimitRule.java
@@ -16,8 +16,8 @@
  */
 package org.apache.calcite.adapter.enumerable;
 
-import org.apache.calcite.plan.RelOptRule;
 import org.apache.calcite.plan.RelOptRuleCall;
+import org.apache.calcite.plan.RelRule;
 import org.apache.calcite.rel.RelNode;
 import org.apache.calcite.rel.core.Sort;
 
@@ -26,12 +26,19 @@ import org.apache.calcite.rel.core.Sort;
  * {@code offset} or {@code fetch} set to an
  * {@link EnumerableLimit}
  * on top of a "pure" {@code Sort} that has no offset or fetch.
+ *
+ * @see EnumerableRules#ENUMERABLE_LIMIT_RULE
  */
-class EnumerableLimitRule extends RelOptRule {
+public class EnumerableLimitRule
+    extends RelRule<EnumerableLimitRule.Config> {
+  /** Creates an EnumerableLimitRule. */
+  protected EnumerableLimitRule(Config config) {
+    super(config);
+  }
+
+  @Deprecated // to be removed before 2.0
   EnumerableLimitRule() {
-    super(
-        operand(Sort.class, any()),
-        "EnumerableLimitRule");
+    this(Config.DEFAULT);
   }
 
   @Override public void onMatch(RelOptRuleCall call) {
@@ -55,4 +62,15 @@ class EnumerableLimitRule extends RelOptRule {
             sort.offset,
             sort.fetch));
   }
+
+  /** Rule configuration. */
+  public interface Config extends RelRule.Config {
+    Config DEFAULT = EMPTY
+        .withOperandSupplier(b -> b.operand(Sort.class).anyInputs())
+        .as(Config.class);
+
+    @Override default EnumerableLimitRule toRule() {
+      return new EnumerableLimitRule(this);
+    }
+  }
 }
diff --git a/core/src/main/java/org/apache/calcite/adapter/enumerable/EnumerableMatchRule.java b/core/src/main/java/org/apache/calcite/adapter/enumerable/EnumerableMatchRule.java
index 57a6fdb..73192f9 100644
--- a/core/src/main/java/org/apache/calcite/adapter/enumerable/EnumerableMatchRule.java
+++ b/core/src/main/java/org/apache/calcite/adapter/enumerable/EnumerableMatchRule.java
@@ -24,11 +24,19 @@ import org.apache.calcite.rel.logical.LogicalMatch;
 /**
  * Rule to convert a {@link LogicalMatch} to an
  * {@link EnumerableMatch}.
+ *
+ * @see EnumerableRules#ENUMERABLE_MATCH_RULE
  */
 public class EnumerableMatchRule extends ConverterRule {
-  EnumerableMatchRule() {
-    super(LogicalMatch.class, Convention.NONE, EnumerableConvention.INSTANCE,
-        "EnumerableMatchRule");
+  /** Default configuration. */
+  public static final Config DEFAULT_CONFIG = Config.INSTANCE
+      .withConversion(LogicalMatch.class, Convention.NONE,
+          EnumerableConvention.INSTANCE, "EnumerableMatchRule")
+      .withRuleFactory(EnumerableMatchRule::new);
+
+  /** Creates an EnumerableMatchRule. */
+  protected EnumerableMatchRule(Config config) {
+    super(config);
   }
 
   @Override public RelNode convert(RelNode rel) {
diff --git a/core/src/main/java/org/apache/calcite/adapter/enumerable/EnumerableMergeJoinRule.java b/core/src/main/java/org/apache/calcite/adapter/enumerable/EnumerableMergeJoinRule.java
index 9c06eaa..0a0ad07 100644
--- a/core/src/main/java/org/apache/calcite/adapter/enumerable/EnumerableMergeJoinRule.java
+++ b/core/src/main/java/org/apache/calcite/adapter/enumerable/EnumerableMergeJoinRule.java
@@ -40,13 +40,18 @@ import java.util.List;
  * {@link EnumerableConvention enumerable calling convention}.
  *
  * @see EnumerableJoinRule
+ * @see EnumerableRules#ENUMERABLE_MERGE_JOIN_RULE
  */
 class EnumerableMergeJoinRule extends ConverterRule {
-  EnumerableMergeJoinRule() {
-    super(LogicalJoin.class,
-        Convention.NONE,
-        EnumerableConvention.INSTANCE,
-        "EnumerableMergeJoinRule");
+  /** Default configuration. */
+  static final Config DEFAULT_CONFIG = Config.INSTANCE
+      .withConversion(LogicalJoin.class, Convention.NONE,
+          EnumerableConvention.INSTANCE, "EnumerableMergeJoinRule")
+      .withRuleFactory(EnumerableMergeJoinRule::new);
+
+  /** Called from the Config. */
+  protected EnumerableMergeJoinRule(Config config) {
+    super(config);
   }
 
   @Override public RelNode convert(RelNode rel) {
diff --git a/core/src/main/java/org/apache/calcite/adapter/enumerable/EnumerableMinusRule.java b/core/src/main/java/org/apache/calcite/adapter/enumerable/EnumerableMinusRule.java
index f81df96..ecbd0e0 100644
--- a/core/src/main/java/org/apache/calcite/adapter/enumerable/EnumerableMinusRule.java
+++ b/core/src/main/java/org/apache/calcite/adapter/enumerable/EnumerableMinusRule.java
@@ -25,14 +25,22 @@ import org.apache.calcite.rel.logical.LogicalMinus;
 /**
  * Rule to convert an {@link org.apache.calcite.rel.logical.LogicalMinus} to an
  * {@link EnumerableMinus}.
+ *
+ * @see EnumerableRules#ENUMERABLE_MINUS_RULE
  */
 class EnumerableMinusRule extends ConverterRule {
-  EnumerableMinusRule() {
-    super(LogicalMinus.class, Convention.NONE, EnumerableConvention.INSTANCE,
-        "EnumerableMinusRule");
+  /** Default configuration. */
+  static final Config DEFAULT_CONFIG = Config.INSTANCE
+      .withConversion(LogicalMinus.class, Convention.NONE,
+          EnumerableConvention.INSTANCE, "EnumerableMinusRule")
+      .withRuleFactory(EnumerableMinusRule::new);
+
+  /** Called from the Config. */
+  protected EnumerableMinusRule(Config config) {
+    super(config);
   }
 
-  public RelNode convert(RelNode rel) {
+  @Override public RelNode convert(RelNode rel) {
     final LogicalMinus minus = (LogicalMinus) rel;
     final EnumerableConvention out = EnumerableConvention.INSTANCE;
     final RelTraitSet traitSet =
diff --git a/core/src/main/java/org/apache/calcite/adapter/enumerable/EnumerableProjectRule.java b/core/src/main/java/org/apache/calcite/adapter/enumerable/EnumerableProjectRule.java
index 6ab1a6e..b5da3cb 100644
--- a/core/src/main/java/org/apache/calcite/adapter/enumerable/EnumerableProjectRule.java
+++ b/core/src/main/java/org/apache/calcite/adapter/enumerable/EnumerableProjectRule.java
@@ -17,27 +17,32 @@
 package org.apache.calcite.adapter.enumerable;
 
 import org.apache.calcite.plan.Convention;
-import org.apache.calcite.plan.RelOptUtil;
 import org.apache.calcite.rel.RelNode;
 import org.apache.calcite.rel.convert.ConverterRule;
-import org.apache.calcite.rel.core.RelFactories;
 import org.apache.calcite.rel.logical.LogicalProject;
 
-import java.util.function.Predicate;
 
 /**
  * Rule to convert a {@link org.apache.calcite.rel.logical.LogicalProject} to an
  * {@link EnumerableProject}.
+ *
+ * @see EnumerableRules#ENUMERABLE_PROJECT_RULE
  */
 class EnumerableProjectRule extends ConverterRule {
-  EnumerableProjectRule() {
-    super(LogicalProject.class,
-        (Predicate<LogicalProject>) RelOptUtil::notContainsWindowedAgg,
-        Convention.NONE, EnumerableConvention.INSTANCE,
-        RelFactories.LOGICAL_BUILDER, "EnumerableProjectRule");
+  /** Default configuration. */
+  static final Config DEFAULT_CONFIG = Config.EMPTY
+      .as(Config.class)
+      .withConversion(LogicalProject.class, p -> !p.containsOver(),
+          Convention.NONE, EnumerableConvention.INSTANCE,
+          "EnumerableProjectRule")
+      .withRuleFactory(EnumerableProjectRule::new);
+
+  /** Creates an EnumerableProjectRule. */
+  protected EnumerableProjectRule(Config config) {
+    super(config);
   }
 
-  public RelNode convert(RelNode rel) {
+  @Override public RelNode convert(RelNode rel) {
     final LogicalProject project = (LogicalProject) rel;
     return EnumerableProject.create(
         convert(project.getInput(),
diff --git a/core/src/main/java/org/apache/calcite/adapter/enumerable/EnumerableProjectToCalcRule.java b/core/src/main/java/org/apache/calcite/adapter/enumerable/EnumerableProjectToCalcRule.java
index 4f6f226..dd344ba 100644
--- a/core/src/main/java/org/apache/calcite/adapter/enumerable/EnumerableProjectToCalcRule.java
+++ b/core/src/main/java/org/apache/calcite/adapter/enumerable/EnumerableProjectToCalcRule.java
@@ -16,25 +16,29 @@
  */
 package org.apache.calcite.adapter.enumerable;
 
-import org.apache.calcite.plan.RelOptRule;
 import org.apache.calcite.plan.RelOptRuleCall;
 import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rel.rules.ProjectToCalcRule;
 import org.apache.calcite.rex.RexProgram;
 import org.apache.calcite.tools.RelBuilderFactory;
 
 /** Variant of {@link org.apache.calcite.rel.rules.ProjectToCalcRule} for
- * {@link org.apache.calcite.adapter.enumerable.EnumerableConvention enumerable calling convention}. */
-public class EnumerableProjectToCalcRule extends RelOptRule {
-  /**
-   * Creates an EnumerableProjectToCalcRule.
-   *
-   * @param relBuilderFactory Builder for relational expressions
-   */
+ * {@link org.apache.calcite.adapter.enumerable.EnumerableConvention enumerable calling convention}.
+ *
+ * @see EnumerableRules#ENUMERABLE_PROJECT_TO_CALC_RULE */
+public class EnumerableProjectToCalcRule extends ProjectToCalcRule {
+  /** Creates an EnumerableProjectToCalcRule. */
+  protected EnumerableProjectToCalcRule(Config config) {
+    super(config);
+  }
+
+  @Deprecated // to be removed before 2.0
   public EnumerableProjectToCalcRule(RelBuilderFactory relBuilderFactory) {
-    super(operand(EnumerableProject.class, any()), relBuilderFactory, null);
+    this(Config.DEFAULT.withRelBuilderFactory(relBuilderFactory)
+        .as(Config.class));
   }
 
-  public void onMatch(RelOptRuleCall call) {
+  @Override public void onMatch(RelOptRuleCall call) {
     final EnumerableProject project = call.rel(0);
     final RelNode input = project.getInput();
     final RexProgram program =
@@ -46,4 +50,16 @@ public class EnumerableProjectToCalcRule extends RelOptRule {
     final EnumerableCalc calc = EnumerableCalc.create(input, program);
     call.transformTo(calc);
   }
+
+  /** Rule configuration. */
+  public interface Config extends ProjectToCalcRule.Config {
+    Config DEFAULT = ProjectToCalcRule.Config.DEFAULT
+        .withOperandSupplier(b ->
+            b.operand(EnumerableProject.class).anyInputs())
+        .as(Config.class);
+
+    @Override default EnumerableProjectToCalcRule toRule() {
+      return new EnumerableProjectToCalcRule(this);
+    }
+  }
 }
diff --git a/core/src/main/java/org/apache/calcite/adapter/enumerable/EnumerableRepeatUnionRule.java b/core/src/main/java/org/apache/calcite/adapter/enumerable/EnumerableRepeatUnionRule.java
index 89aa707..1ae3511 100644
--- a/core/src/main/java/org/apache/calcite/adapter/enumerable/EnumerableRepeatUnionRule.java
+++ b/core/src/main/java/org/apache/calcite/adapter/enumerable/EnumerableRepeatUnionRule.java
@@ -23,17 +23,21 @@ import org.apache.calcite.rel.convert.ConverterRule;
 import org.apache.calcite.rel.logical.LogicalRepeatUnion;
 
 /**
- * Rule to convert a {@link LogicalRepeatUnion} into an {@link EnumerableRepeatUnion}.
+ * Rule to convert a {@link LogicalRepeatUnion} into an
+ * {@link EnumerableRepeatUnion}.
+ *
+ * @see EnumerableRules#ENUMERABLE_REPEAT_UNION_RULE
  */
 public class EnumerableRepeatUnionRule extends ConverterRule {
+  /** Default configuration. */
+  public static final Config DEFAULT_CONFIG = Config.INSTANCE
+      .withConversion(LogicalRepeatUnion.class, Convention.NONE,
+          EnumerableConvention.INSTANCE, "EnumerableRepeatUnionRule")
+      .withRuleFactory(EnumerableRepeatUnionRule::new);
 
-  EnumerableRepeatUnionRule() {
-    super(
-        LogicalRepeatUnion.class,
-        Convention.NONE,
-        EnumerableConvention.INSTANCE,
-        "EnumerableRepeatUnionRule");
-
+  /** Called from the Config. */
+  protected EnumerableRepeatUnionRule(Config config) {
+    super(config);
   }
 
   @Override public RelNode convert(RelNode rel) {
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 2672a67..1222353 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
@@ -16,8 +16,12 @@
  */
 package org.apache.calcite.adapter.enumerable;
 
+import org.apache.calcite.linq4j.function.Experimental;
 import org.apache.calcite.plan.RelOptRule;
-import org.apache.calcite.rel.core.RelFactories;
+import org.apache.calcite.rel.logical.LogicalAggregate;
+import org.apache.calcite.rel.logical.LogicalMatch;
+import org.apache.calcite.rel.logical.LogicalRepeatUnion;
+import org.apache.calcite.rel.logical.LogicalTableSpool;
 import org.apache.calcite.util.trace.CalciteTrace;
 
 import com.google.common.collect.ImmutableList;
@@ -38,94 +42,158 @@ public class EnumerableRules {
   private EnumerableRules() {
   }
 
+  /** Rule that converts a
+   * {@link org.apache.calcite.rel.logical.LogicalJoin} to
+   * {@link EnumerableConvention enumerable calling convention}. */
   public static final RelOptRule ENUMERABLE_JOIN_RULE =
-      new EnumerableJoinRule();
+      EnumerableJoinRule.DEFAULT_CONFIG.toRule(EnumerableJoinRule.class);
 
+  /** Rule that converts a
+   * {@link org.apache.calcite.rel.logical.LogicalJoin} to
+   * {@link EnumerableConvention enumerable calling convention}. */
   public static final RelOptRule ENUMERABLE_MERGE_JOIN_RULE =
-      new EnumerableMergeJoinRule();
+      EnumerableMergeJoinRule.DEFAULT_CONFIG
+          .toRule(EnumerableMergeJoinRule.class);
 
   public static final RelOptRule ENUMERABLE_CORRELATE_RULE =
-      new EnumerableCorrelateRule(RelFactories.LOGICAL_BUILDER);
+      EnumerableCorrelateRule.DEFAULT_CONFIG
+          .toRule(EnumerableCorrelateRule.class);
 
+  /** Rule that converts a
+   * {@link org.apache.calcite.rel.logical.LogicalJoin} into an
+   * {@link org.apache.calcite.adapter.enumerable.EnumerableBatchNestedLoopJoin}. */
   public static final RelOptRule ENUMERABLE_BATCH_NESTED_LOOP_JOIN_RULE =
-      new EnumerableBatchNestedLoopJoinRule(RelFactories.LOGICAL_BUILDER);
+      EnumerableBatchNestedLoopJoinRule.Config.DEFAULT.toRule();
 
+  /** Rule that converts a
+   *  {@link org.apache.calcite.rel.logical.LogicalProject} to an
+   * {@link EnumerableProject}. */
   public static final EnumerableProjectRule ENUMERABLE_PROJECT_RULE =
-      new EnumerableProjectRule();
+      EnumerableProjectRule.DEFAULT_CONFIG.toRule(EnumerableProjectRule.class);
 
   public static final EnumerableFilterRule ENUMERABLE_FILTER_RULE =
-      new EnumerableFilterRule();
+      EnumerableFilterRule.DEFAULT_CONFIG.toRule(EnumerableFilterRule.class);
 
   public static final EnumerableCalcRule ENUMERABLE_CALC_RULE =
-      new EnumerableCalcRule();
+      EnumerableCalcRule.DEFAULT_CONFIG.toRule(EnumerableCalcRule.class);
 
   public static final EnumerableAggregateRule ENUMERABLE_AGGREGATE_RULE =
-      new EnumerableAggregateRule();
+      EnumerableAggregateRule.DEFAULT_CONFIG
+          .toRule(EnumerableAggregateRule.class);
 
+  /** Rule that converts a {@link org.apache.calcite.rel.core.Sort} to an
+   * {@link EnumerableSort}. */
   public static final EnumerableSortRule ENUMERABLE_SORT_RULE =
-      new EnumerableSortRule();
+      EnumerableSortRule.DEFAULT_CONFIG.toRule(EnumerableSortRule.class);
 
   public static final EnumerableLimitRule ENUMERABLE_LIMIT_RULE =
-      new EnumerableLimitRule();
+      EnumerableLimitRule.Config.DEFAULT.toRule();
 
+  /** Rule that converts a {@link org.apache.calcite.rel.logical.LogicalUnion}
+   * to an {@link EnumerableUnion}. */
   public static final EnumerableUnionRule ENUMERABLE_UNION_RULE =
-      new EnumerableUnionRule();
+      EnumerableUnionRule.DEFAULT_CONFIG.toRule(EnumerableUnionRule.class);
 
+  /** Rule that converts a {@link LogicalRepeatUnion} into an
+   * {@link EnumerableRepeatUnion}. */
   public static final EnumerableRepeatUnionRule ENUMERABLE_REPEAT_UNION_RULE =
-      new EnumerableRepeatUnionRule();
+      EnumerableRepeatUnionRule.DEFAULT_CONFIG
+          .toRule(EnumerableRepeatUnionRule.class);
 
+  /** Rule that converts a {@link LogicalTableSpool} into an
+   * {@link EnumerableTableSpool}. */
+  @Experimental
   public static final EnumerableTableSpoolRule ENUMERABLE_TABLE_SPOOL_RULE =
-      new EnumerableTableSpoolRule();
+      EnumerableTableSpoolRule.DEFAULT_CONFIG
+          .toRule(EnumerableTableSpoolRule.class);
 
+  /** Rule that converts a
+   * {@link org.apache.calcite.rel.logical.LogicalIntersect} to an
+   * {@link EnumerableIntersect}. */
   public static final EnumerableIntersectRule ENUMERABLE_INTERSECT_RULE =
-      new EnumerableIntersectRule();
+      EnumerableIntersectRule.DEFAULT_CONFIG
+          .toRule(EnumerableIntersectRule.class);
 
+  /** Rule that converts a
+   * {@link org.apache.calcite.rel.logical.LogicalMinus} to an
+   * {@link EnumerableMinus}. */
   public static final EnumerableMinusRule ENUMERABLE_MINUS_RULE =
-      new EnumerableMinusRule();
+      EnumerableMinusRule.DEFAULT_CONFIG.toRule(EnumerableMinusRule.class);
 
+  /** Rule that converts a
+   * {@link org.apache.calcite.rel.logical.LogicalTableModify} to
+   * {@link EnumerableConvention enumerable calling convention}. */
   public static final EnumerableTableModifyRule ENUMERABLE_TABLE_MODIFICATION_RULE =
-      new EnumerableTableModifyRule(RelFactories.LOGICAL_BUILDER);
+      EnumerableTableModifyRule.DEFAULT_CONFIG
+          .toRule(EnumerableTableModifyRule.class);
 
+  /** Rule that converts a
+   * {@link org.apache.calcite.rel.logical.LogicalValues} to
+   * {@link EnumerableConvention enumerable calling convention}. */
   public static final EnumerableValuesRule ENUMERABLE_VALUES_RULE =
-      new EnumerableValuesRule(RelFactories.LOGICAL_BUILDER);
+      EnumerableValuesRule.DEFAULT_CONFIG.toRule(EnumerableValuesRule.class);
 
+  /** Rule that converts a {@link org.apache.calcite.rel.logical.LogicalWindow}
+   * to an {@link org.apache.calcite.adapter.enumerable.EnumerableWindow}. */
   public static final EnumerableWindowRule ENUMERABLE_WINDOW_RULE =
-      new EnumerableWindowRule();
+      EnumerableWindowRule.DEFAULT_CONFIG.toRule(EnumerableWindowRule.class);
 
+  /** Rule that converts an {@link org.apache.calcite.rel.core.Collect}
+   * to an {@link EnumerableCollect}. */
   public static final EnumerableCollectRule ENUMERABLE_COLLECT_RULE =
-      new EnumerableCollectRule();
+      EnumerableCollectRule.DEFAULT_CONFIG.toRule(EnumerableCollectRule.class);
 
+  /** Rule that converts an {@link org.apache.calcite.rel.core.Uncollect}
+   * to an {@link EnumerableUncollect}. */
   public static final EnumerableUncollectRule ENUMERABLE_UNCOLLECT_RULE =
-      new EnumerableUncollectRule();
+      EnumerableUncollectRule.DEFAULT_CONFIG
+          .toRule(EnumerableUncollectRule.class);
 
   public static final EnumerableFilterToCalcRule ENUMERABLE_FILTER_TO_CALC_RULE =
-      new EnumerableFilterToCalcRule(RelFactories.LOGICAL_BUILDER);
+      EnumerableFilterToCalcRule.Config.DEFAULT.toRule();
 
+  /** Variant of {@link org.apache.calcite.rel.rules.ProjectToCalcRule} for
+   * {@link EnumerableConvention enumerable calling convention}. */
   public static final EnumerableProjectToCalcRule ENUMERABLE_PROJECT_TO_CALC_RULE =
-      new EnumerableProjectToCalcRule(RelFactories.LOGICAL_BUILDER);
+      EnumerableProjectToCalcRule.Config.DEFAULT.toRule();
 
+  /** Rule that converts a
+   *  {@link org.apache.calcite.rel.logical.LogicalTableScan} to
+   * {@link EnumerableConvention enumerable calling convention}. */
   public static final EnumerableTableScanRule ENUMERABLE_TABLE_SCAN_RULE =
-      new EnumerableTableScanRule(RelFactories.LOGICAL_BUILDER);
+      EnumerableTableScanRule.DEFAULT_CONFIG
+          .toRule(EnumerableTableScanRule.class);
 
+  /** Rule that converts a
+   * {@link org.apache.calcite.rel.logical.LogicalTableFunctionScan} to
+   * {@link EnumerableConvention enumerable calling convention}. */
   public static final EnumerableTableFunctionScanRule ENUMERABLE_TABLE_FUNCTION_SCAN_RULE =
-      new EnumerableTableFunctionScanRule(RelFactories.LOGICAL_BUILDER);
+      EnumerableTableFunctionScanRule.DEFAULT_CONFIG
+          .toRule(EnumerableTableFunctionScanRule.class);
 
+  /** Rule that converts a {@link LogicalMatch} to an
+   *  {@link EnumerableMatch}. */
   public static final EnumerableMatchRule ENUMERABLE_MATCH_RULE =
-      new EnumerableMatchRule();
+      EnumerableMatchRule.DEFAULT_CONFIG.toRule(EnumerableMatchRule.class);
 
+  /** Rule to convert a {@link LogicalAggregate}
+   * to an {@link EnumerableSortedAggregate}. */
   public static final EnumerableSortedAggregateRule ENUMERABLE_SORTED_AGGREGATE_RULE =
-      new EnumerableSortedAggregateRule();
+      EnumerableSortedAggregateRule.DEFAULT_CONFIG
+          .toRule(EnumerableSortedAggregateRule.class);
 
   /** Rule that converts any enumerable relational expression to bindable. */
   public static final EnumerableBindable.EnumerableToBindableConverterRule TO_BINDABLE =
-      new EnumerableBindable.EnumerableToBindableConverterRule(RelFactories.LOGICAL_BUILDER);
+      EnumerableBindable.EnumerableToBindableConverterRule.DEFAULT_CONFIG
+          .toRule(EnumerableBindable.EnumerableToBindableConverterRule.class);
 
   /**
    * Rule that converts {@link org.apache.calcite.interpreter.BindableRel}
    * to {@link org.apache.calcite.adapter.enumerable.EnumerableRel} by creating
    * an {@link org.apache.calcite.adapter.enumerable.EnumerableInterpreter}. */
   public static final EnumerableInterpreterRule TO_INTERPRETER =
-      new EnumerableInterpreterRule(RelFactories.LOGICAL_BUILDER);
+      EnumerableInterpreterRule.DEFAULT_CONFIG
+          .toRule(EnumerableInterpreterRule.class);
 
   public static final List<RelOptRule> ENUMERABLE_RULES = ImmutableList.of(
       EnumerableRules.ENUMERABLE_JOIN_RULE,
diff --git a/core/src/main/java/org/apache/calcite/adapter/enumerable/EnumerableSortRule.java b/core/src/main/java/org/apache/calcite/adapter/enumerable/EnumerableSortRule.java
index 4e9941b..84ef93c 100644
--- a/core/src/main/java/org/apache/calcite/adapter/enumerable/EnumerableSortRule.java
+++ b/core/src/main/java/org/apache/calcite/adapter/enumerable/EnumerableSortRule.java
@@ -24,11 +24,19 @@ import org.apache.calcite.rel.core.Sort;
 /**
  * Rule to convert an {@link org.apache.calcite.rel.core.Sort} to an
  * {@link EnumerableSort}.
+ *
+ * @see EnumerableRules#ENUMERABLE_SORT_RULE
  */
 class EnumerableSortRule extends ConverterRule {
-  EnumerableSortRule() {
-    super(Sort.class, Convention.NONE, EnumerableConvention.INSTANCE,
-        "EnumerableSortRule");
+  /** Default configuration. */
+  public static final Config DEFAULT_CONFIG = Config.INSTANCE
+      .withConversion(Sort.class, Convention.NONE,
+          EnumerableConvention.INSTANCE, "EnumerableSortRule")
+      .withRuleFactory(EnumerableSortRule::new);
+
+  /** Called from the Config. */
+  protected EnumerableSortRule(Config config) {
+    super(config);
   }
 
   public RelNode convert(RelNode rel) {
diff --git a/core/src/main/java/org/apache/calcite/adapter/enumerable/EnumerableSortedAggregateRule.java b/core/src/main/java/org/apache/calcite/adapter/enumerable/EnumerableSortedAggregateRule.java
index 2f6e810..06577c2 100644
--- a/core/src/main/java/org/apache/calcite/adapter/enumerable/EnumerableSortedAggregateRule.java
+++ b/core/src/main/java/org/apache/calcite/adapter/enumerable/EnumerableSortedAggregateRule.java
@@ -29,11 +29,19 @@ import org.apache.calcite.util.ImmutableIntList;
 /**
  * Rule to convert a {@link LogicalAggregate}
  * to an {@link EnumerableSortedAggregate}.
+ *
+ * @see EnumerableRules#ENUMERABLE_SORTED_AGGREGATE_RULE
  */
 class EnumerableSortedAggregateRule extends ConverterRule {
-  EnumerableSortedAggregateRule() {
-    super(LogicalAggregate.class, Convention.NONE,
-        EnumerableConvention.INSTANCE, "EnumerableSortedAggregateRule");
+  /** Default configuration. */
+  public static final Config DEFAULT_CONFIG = Config.INSTANCE
+      .withConversion(LogicalAggregate.class, Convention.NONE,
+          EnumerableConvention.INSTANCE, "EnumerableSortedAggregateRule")
+      .withRuleFactory(EnumerableSortedAggregateRule::new);
+
+  /** Called from the Config. */
+  protected EnumerableSortedAggregateRule(Config config) {
+    super(config);
   }
 
   public RelNode convert(RelNode rel) {
diff --git a/core/src/main/java/org/apache/calcite/adapter/enumerable/EnumerableTableFunctionScanRule.java b/core/src/main/java/org/apache/calcite/adapter/enumerable/EnumerableTableFunctionScanRule.java
index 744f95e..f684a00 100644
--- a/core/src/main/java/org/apache/calcite/adapter/enumerable/EnumerableTableFunctionScanRule.java
+++ b/core/src/main/java/org/apache/calcite/adapter/enumerable/EnumerableTableFunctionScanRule.java
@@ -20,39 +20,33 @@ import org.apache.calcite.plan.Convention;
 import org.apache.calcite.plan.RelTraitSet;
 import org.apache.calcite.rel.RelNode;
 import org.apache.calcite.rel.convert.ConverterRule;
-import org.apache.calcite.rel.core.RelFactories;
 import org.apache.calcite.rel.logical.LogicalTableFunctionScan;
-import org.apache.calcite.tools.RelBuilderFactory;
-
-import java.util.function.Predicate;
 
 /** Planner rule that converts a
- * {@link org.apache.calcite.rel.logical.LogicalTableFunctionScan}
- * relational expression
- * {@link org.apache.calcite.adapter.enumerable.EnumerableConvention enumerable calling convention}. */
+ * {@link org.apache.calcite.rel.logical.LogicalTableFunctionScan} to
+ * {@link org.apache.calcite.adapter.enumerable.EnumerableConvention enumerable calling convention}.
+ *
+ * @see EnumerableRules#ENUMERABLE_TABLE_FUNCTION_SCAN_RULE */
 public class EnumerableTableFunctionScanRule extends ConverterRule {
-  @Deprecated // to be removed before 2.0
-  public EnumerableTableFunctionScanRule() {
-    this(RelFactories.LOGICAL_BUILDER);
-  }
+  /** Default configuration. */
+  public static final Config DEFAULT_CONFIG = Config.EMPTY
+      .as(Config.class)
+      .withConversion(LogicalTableFunctionScan.class, Convention.NONE,
+          EnumerableConvention.INSTANCE, "EnumerableTableFunctionScanRule")
+      .withRuleFactory(EnumerableTableFunctionScanRule::new);
 
-  /**
-   * Creates an EnumerableTableFunctionScanRule.
-   *
-   * @param relBuilderFactory Builder for relational expressions
-   */
-  public EnumerableTableFunctionScanRule(RelBuilderFactory relBuilderFactory) {
-    super(LogicalTableFunctionScan.class, (Predicate<RelNode>) r -> true,
-        Convention.NONE, EnumerableConvention.INSTANCE, relBuilderFactory,
-        "EnumerableTableFunctionScanRule");
+  /** Creates an EnumerableTableFunctionScanRule. */
+  protected EnumerableTableFunctionScanRule(Config config) {
+    super(config);
   }
 
   @Override public RelNode convert(RelNode rel) {
     final RelTraitSet traitSet =
         rel.getTraitSet().replace(EnumerableConvention.INSTANCE);
-    LogicalTableFunctionScan tbl = (LogicalTableFunctionScan) rel;
+    LogicalTableFunctionScan scan = (LogicalTableFunctionScan) rel;
     return new EnumerableTableFunctionScan(rel.getCluster(), traitSet,
-        convertList(tbl.getInputs(), traitSet.getTrait(0)), tbl.getElementType(), tbl.getRowType(),
-        tbl.getCall(), tbl.getColumnMappings());
+        convertList(scan.getInputs(), traitSet.getTrait(0)),
+        scan.getElementType(), scan.getRowType(),
+        scan.getCall(), scan.getColumnMappings());
   }
 }
diff --git a/core/src/main/java/org/apache/calcite/adapter/enumerable/EnumerableTableModifyRule.java b/core/src/main/java/org/apache/calcite/adapter/enumerable/EnumerableTableModifyRule.java
index 3186c61..0019d3a 100644
--- a/core/src/main/java/org/apache/calcite/adapter/enumerable/EnumerableTableModifyRule.java
+++ b/core/src/main/java/org/apache/calcite/adapter/enumerable/EnumerableTableModifyRule.java
@@ -22,24 +22,22 @@ import org.apache.calcite.rel.RelNode;
 import org.apache.calcite.rel.convert.ConverterRule;
 import org.apache.calcite.rel.logical.LogicalTableModify;
 import org.apache.calcite.schema.ModifiableTable;
-import org.apache.calcite.tools.RelBuilderFactory;
-
-import java.util.function.Predicate;
 
 /** Planner rule that converts a
- * {@link org.apache.calcite.rel.logical.LogicalTableModify}
- * relational expression
- * {@link org.apache.calcite.adapter.enumerable.EnumerableConvention enumerable calling convention}. */
+ * {@link org.apache.calcite.rel.logical.LogicalTableModify} to
+ * {@link org.apache.calcite.adapter.enumerable.EnumerableConvention enumerable calling convention}.
+ *
+ * @see EnumerableRules#ENUMERABLE_TABLE_MODIFICATION_RULE */
 public class EnumerableTableModifyRule extends ConverterRule {
-  /**
-   * Creates an EnumerableTableModifyRule.
-   *
-   * @param relBuilderFactory Builder for relational expressions
-   */
-  public EnumerableTableModifyRule(RelBuilderFactory relBuilderFactory) {
-    super(LogicalTableModify.class, (Predicate<RelNode>) r -> true,
-        Convention.NONE, EnumerableConvention.INSTANCE, relBuilderFactory,
-        "EnumerableTableModificationRule");
+  /** Default configuration. */
+  public static final Config DEFAULT_CONFIG = Config.INSTANCE
+      .withConversion(LogicalTableModify.class, Convention.NONE,
+          EnumerableConvention.INSTANCE, "EnumerableTableModificationRule")
+      .withRuleFactory(EnumerableTableModifyRule::new);
+
+  /** Creates an EnumerableTableModifyRule. */
+  protected EnumerableTableModifyRule(Config config) {
+    super(config);
   }
 
   @Override public RelNode convert(RelNode rel) {
diff --git a/core/src/main/java/org/apache/calcite/adapter/enumerable/EnumerableTableScanRule.java b/core/src/main/java/org/apache/calcite/adapter/enumerable/EnumerableTableScanRule.java
index 341e79d..1ecb226 100644
--- a/core/src/main/java/org/apache/calcite/adapter/enumerable/EnumerableTableScanRule.java
+++ b/core/src/main/java/org/apache/calcite/adapter/enumerable/EnumerableTableScanRule.java
@@ -20,35 +20,27 @@ import org.apache.calcite.plan.Convention;
 import org.apache.calcite.plan.RelOptTable;
 import org.apache.calcite.rel.RelNode;
 import org.apache.calcite.rel.convert.ConverterRule;
-import org.apache.calcite.rel.core.RelFactories;
 import org.apache.calcite.rel.logical.LogicalTableScan;
 import org.apache.calcite.schema.QueryableTable;
 import org.apache.calcite.schema.Table;
-import org.apache.calcite.tools.RelBuilderFactory;
-
-import java.util.function.Predicate;
 
 /** Planner rule that converts a
- * {@link org.apache.calcite.rel.logical.LogicalTableFunctionScan}
- * relational expression
- * {@link EnumerableConvention enumerable calling convention}. */
+ * {@link org.apache.calcite.rel.logical.LogicalTableScan} to
+ * {@link EnumerableConvention enumerable calling convention}.
+ *
+ * @see EnumerableRules#ENUMERABLE_TABLE_SCAN_RULE */
 public class EnumerableTableScanRule extends ConverterRule {
+  /** Default configuration. */
+  public static final Config DEFAULT_CONFIG = Config.EMPTY
+      .as(Config.class)
+      .withConversion(LogicalTableScan.class,
+          r -> EnumerableTableScan.canHandle(r.getTable()),
+          Convention.NONE, EnumerableConvention.INSTANCE,
+          "EnumerableTableScanRule")
+      .withRuleFactory(EnumerableTableScanRule::new);
 
-  @Deprecated // to be removed before 2.0
-  public EnumerableTableScanRule() {
-    this(RelFactories.LOGICAL_BUILDER);
-  }
-
-  /**
-   * Creates an EnumerableTableScanRule.
-   *
-   * @param relBuilderFactory Builder for relational expressions
-   */
-  public EnumerableTableScanRule(RelBuilderFactory relBuilderFactory) {
-    super(LogicalTableScan.class,
-        (Predicate<LogicalTableScan>) r -> EnumerableTableScan.canHandle(r.getTable()),
-        Convention.NONE, EnumerableConvention.INSTANCE, relBuilderFactory,
-        "EnumerableTableScanRule");
+  protected EnumerableTableScanRule(Config config) {
+    super(config);
   }
 
   @Override public RelNode convert(RelNode rel) {
diff --git a/core/src/main/java/org/apache/calcite/adapter/enumerable/EnumerableTableSpoolRule.java b/core/src/main/java/org/apache/calcite/adapter/enumerable/EnumerableTableSpoolRule.java
index 12e3a01..dd546ff 100644
--- a/core/src/main/java/org/apache/calcite/adapter/enumerable/EnumerableTableSpoolRule.java
+++ b/core/src/main/java/org/apache/calcite/adapter/enumerable/EnumerableTableSpoolRule.java
@@ -28,15 +28,20 @@ import org.apache.calcite.rel.logical.LogicalTableSpool;
  *
  * <p>NOTE: The current API is experimental and subject to change without
  * notice.
+ *
+ * @see EnumerableRules#ENUMERABLE_TABLE_SPOOL_RULE
  */
 @Experimental
 public class EnumerableTableSpoolRule extends ConverterRule {
+  /** Default configuration. */
+  public static final Config DEFAULT_CONFIG = Config.INSTANCE
+      .withConversion(LogicalTableSpool.class, Convention.NONE,
+          EnumerableConvention.INSTANCE, "EnumerableTableSpoolRule")
+      .withRuleFactory(EnumerableTableSpoolRule::new);
 
-  EnumerableTableSpoolRule() {
-    super(LogicalTableSpool.class,
-        Convention.NONE,
-        EnumerableConvention.INSTANCE,
-        "EnumerableTableSpoolRule");
+  /** Called from the Config. */
+  protected EnumerableTableSpoolRule(Config config) {
+    super(config);
   }
 
   @Override public RelNode convert(RelNode rel) {
diff --git a/core/src/main/java/org/apache/calcite/adapter/enumerable/EnumerableUncollectRule.java b/core/src/main/java/org/apache/calcite/adapter/enumerable/EnumerableUncollectRule.java
index f3b3c75..9f33b71 100644
--- a/core/src/main/java/org/apache/calcite/adapter/enumerable/EnumerableUncollectRule.java
+++ b/core/src/main/java/org/apache/calcite/adapter/enumerable/EnumerableUncollectRule.java
@@ -25,14 +25,22 @@ import org.apache.calcite.rel.core.Uncollect;
 /**
  * Rule to convert an {@link org.apache.calcite.rel.core.Uncollect} to an
  * {@link EnumerableUncollect}.
+ *
+ * @see EnumerableRules#ENUMERABLE_UNCOLLECT_RULE
  */
 class EnumerableUncollectRule extends ConverterRule {
-  EnumerableUncollectRule() {
-    super(Uncollect.class, Convention.NONE, EnumerableConvention.INSTANCE,
-        "EnumerableUncollectRule");
+  /** Default configuration. */
+  static final Config DEFAULT_CONFIG = Config.INSTANCE
+      .withConversion(Uncollect.class, Convention.NONE,
+          EnumerableConvention.INSTANCE, "EnumerableUncollectRule")
+      .withRuleFactory(EnumerableUncollectRule::new);
+
+  /** Called from the Config. */
+  protected EnumerableUncollectRule(Config config) {
+    super(config);
   }
 
-  public RelNode convert(RelNode rel) {
+  @Override public RelNode convert(RelNode rel) {
     final Uncollect uncollect = (Uncollect) rel;
     final RelTraitSet traitSet =
         uncollect.getTraitSet().replace(EnumerableConvention.INSTANCE);
diff --git a/core/src/main/java/org/apache/calcite/adapter/enumerable/EnumerableUnionRule.java b/core/src/main/java/org/apache/calcite/adapter/enumerable/EnumerableUnionRule.java
index d611c51..08fc857 100644
--- a/core/src/main/java/org/apache/calcite/adapter/enumerable/EnumerableUnionRule.java
+++ b/core/src/main/java/org/apache/calcite/adapter/enumerable/EnumerableUnionRule.java
@@ -29,14 +29,22 @@ import java.util.List;
 /**
  * Rule to convert an {@link org.apache.calcite.rel.logical.LogicalUnion} to an
  * {@link EnumerableUnion}.
+ *
+ * @see EnumerableRules#ENUMERABLE_UNION_RULE
  */
 class EnumerableUnionRule extends ConverterRule {
-  EnumerableUnionRule() {
-    super(LogicalUnion.class, Convention.NONE, EnumerableConvention.INSTANCE,
-        "EnumerableUnionRule");
+  /** Default configuration. */
+  static final Config DEFAULT_CONFIG = Config.INSTANCE
+      .withConversion(LogicalUnion.class, Convention.NONE,
+          EnumerableConvention.INSTANCE, "EnumerableUnionRule")
+      .withRuleFactory(EnumerableUnionRule::new);
+
+  /** Called from the Config. */
+  protected EnumerableUnionRule(Config config) {
+    super(config);
   }
 
-  public RelNode convert(RelNode rel) {
+  @Override public RelNode convert(RelNode rel) {
     final LogicalUnion union = (LogicalUnion) rel;
     final EnumerableConvention out = EnumerableConvention.INSTANCE;
     final RelTraitSet traitSet = rel.getCluster().traitSet().replace(out);
diff --git a/core/src/main/java/org/apache/calcite/adapter/enumerable/EnumerableValuesRule.java b/core/src/main/java/org/apache/calcite/adapter/enumerable/EnumerableValuesRule.java
index c981a8b..605e922 100644
--- a/core/src/main/java/org/apache/calcite/adapter/enumerable/EnumerableValuesRule.java
+++ b/core/src/main/java/org/apache/calcite/adapter/enumerable/EnumerableValuesRule.java
@@ -20,24 +20,23 @@ import org.apache.calcite.plan.Convention;
 import org.apache.calcite.rel.RelNode;
 import org.apache.calcite.rel.convert.ConverterRule;
 import org.apache.calcite.rel.logical.LogicalValues;
-import org.apache.calcite.tools.RelBuilderFactory;
-
-import java.util.function.Predicate;
 
 /** Planner rule that converts a
  * {@link org.apache.calcite.rel.logical.LogicalValues}
  * relational expression
- * {@link org.apache.calcite.adapter.enumerable.EnumerableConvention enumerable calling convention}. */
+ * {@link org.apache.calcite.adapter.enumerable.EnumerableConvention enumerable calling convention}.
+ *
+ * @see EnumerableRules#ENUMERABLE_VALUES_RULE */
 public class EnumerableValuesRule extends ConverterRule {
-  /**
-   * Creates an EnumerableValuesRule.
-   *
-   * @param relBuilderFactory Builder for relational expressions
-   */
-  public EnumerableValuesRule(RelBuilderFactory relBuilderFactory) {
-    super(LogicalValues.class, (Predicate<RelNode>) r -> true,
-        Convention.NONE, EnumerableConvention.INSTANCE, relBuilderFactory,
-        "EnumerableValuesRule");
+  /** Default configuration. */
+  public static final Config DEFAULT_CONFIG = Config.INSTANCE
+      .withConversion(LogicalValues.class, Convention.NONE,
+          EnumerableConvention.INSTANCE, "EnumerableValuesRule")
+      .withRuleFactory(EnumerableValuesRule::new);
+
+  /** Creates an EnumerableValuesRule. */
+  protected EnumerableValuesRule(Config config) {
+    super(config);
   }
 
   @Override public RelNode convert(RelNode rel) {
diff --git a/core/src/main/java/org/apache/calcite/adapter/enumerable/EnumerableWindowRule.java b/core/src/main/java/org/apache/calcite/adapter/enumerable/EnumerableWindowRule.java
index bc3e1dd..837a664 100644
--- a/core/src/main/java/org/apache/calcite/adapter/enumerable/EnumerableWindowRule.java
+++ b/core/src/main/java/org/apache/calcite/adapter/enumerable/EnumerableWindowRule.java
@@ -25,14 +25,22 @@ import org.apache.calcite.rel.logical.LogicalWindow;
 /**
  * Rule to convert a {@link org.apache.calcite.rel.logical.LogicalWindow} to
  * an {@link org.apache.calcite.adapter.enumerable.EnumerableWindow}.
+ *
+ * @see EnumerableRules#ENUMERABLE_WINDOW_RULE
  */
 class EnumerableWindowRule extends ConverterRule {
-  EnumerableWindowRule() {
-    super(LogicalWindow.class, Convention.NONE, EnumerableConvention.INSTANCE,
-        "EnumerableWindowRule");
+  /** Default configuration. */
+  static final Config DEFAULT_CONFIG = Config.INSTANCE
+      .withConversion(LogicalWindow.class, Convention.NONE,
+          EnumerableConvention.INSTANCE, "EnumerableWindowRule")
+      .withRuleFactory(EnumerableWindowRule::new);
+
+  /** Called from the Config. */
+  protected EnumerableWindowRule(Config config) {
+    super(config);
   }
 
-  public RelNode convert(RelNode rel) {
+  @Override public RelNode convert(RelNode rel) {
     final LogicalWindow winAgg = (LogicalWindow) rel;
     final RelTraitSet traitSet =
         winAgg.getTraitSet().replace(EnumerableConvention.INSTANCE);
diff --git a/core/src/main/java/org/apache/calcite/adapter/jdbc/JdbcRules.java b/core/src/main/java/org/apache/calcite/adapter/jdbc/JdbcRules.java
index 438781b..f7f8ce8 100644
--- a/core/src/main/java/org/apache/calcite/adapter/jdbc/JdbcRules.java
+++ b/core/src/main/java/org/apache/calcite/adapter/jdbc/JdbcRules.java
@@ -25,7 +25,7 @@ import org.apache.calcite.plan.RelOptCost;
 import org.apache.calcite.plan.RelOptPlanner;
 import org.apache.calcite.plan.RelOptRule;
 import org.apache.calcite.plan.RelOptTable;
-import org.apache.calcite.plan.RelTrait;
+import org.apache.calcite.plan.RelRule;
 import org.apache.calcite.plan.RelTraitSet;
 import org.apache.calcite.prepare.Prepare;
 import org.apache.calcite.rel.InvalidRelException;
@@ -58,7 +58,6 @@ import org.apache.calcite.rex.RexInputRef;
 import org.apache.calcite.rex.RexLiteral;
 import org.apache.calcite.rex.RexMultisetUtil;
 import org.apache.calcite.rex.RexNode;
-import org.apache.calcite.rex.RexOver;
 import org.apache.calcite.rex.RexProgram;
 import org.apache.calcite.rex.RexUtil;
 import org.apache.calcite.rex.RexVisitorImpl;
@@ -82,7 +81,7 @@ import org.slf4j.Logger;
 import java.util.ArrayList;
 import java.util.List;
 import java.util.Set;
-import java.util.function.Predicate;
+import java.util.function.Consumer;
 
 /**
  * Rules and relational operators for
@@ -213,69 +212,59 @@ public class JdbcRules {
               TABLE_SCAN_FACTORY,
               SNAPSHOT_FACTORY));
 
+  /** Creates a list of rules with the given JDBC convention instance. */
   public static List<RelOptRule> rules(JdbcConvention out) {
-    return rules(out, RelFactories.LOGICAL_BUILDER);
+    final ImmutableList.Builder<RelOptRule> b = ImmutableList.builder();
+    foreachRule(out, b::add);
+    return b.build();
   }
 
+  /** Creates a list of rules with the given JDBC convention instance
+   * and builder factory. */
   public static List<RelOptRule> rules(JdbcConvention out,
       RelBuilderFactory relBuilderFactory) {
-    return ImmutableList.of(
-        new JdbcToEnumerableConverterRule(out, relBuilderFactory),
-        new JdbcJoinRule(out, relBuilderFactory),
-        new JdbcCalcRule(out, relBuilderFactory),
-        new JdbcProjectRule(out, relBuilderFactory),
-        new JdbcFilterRule(out, relBuilderFactory),
-        new JdbcAggregateRule(out, relBuilderFactory),
-        new JdbcSortRule(out, relBuilderFactory),
-        new JdbcUnionRule(out, relBuilderFactory),
-        new JdbcIntersectRule(out, relBuilderFactory),
-        new JdbcMinusRule(out, relBuilderFactory),
-        new JdbcTableModificationRule(out, relBuilderFactory),
-        new JdbcValuesRule(out, relBuilderFactory));
+    final ImmutableList.Builder<RelOptRule> b = ImmutableList.builder();
+    foreachRule(out, r ->
+        b.add(r.config.withRelBuilderFactory(relBuilderFactory).toRule()));
+    return b.build();
+  }
+
+  private static void foreachRule(JdbcConvention out,
+      Consumer<RelRule<?>> consumer) {
+    consumer.accept(JdbcToEnumerableConverterRule.create(out));
+    consumer.accept(JdbcJoinRule.create(out));
+    consumer.accept(JdbcCalcRule.create(out));
+    consumer.accept(JdbcProjectRule.create(out));
+    consumer.accept(JdbcFilterRule.create(out));
+    consumer.accept(JdbcAggregateRule.create(out));
+    consumer.accept(JdbcSortRule.create(out));
+    consumer.accept(JdbcUnionRule.create(out));
+    consumer.accept(JdbcIntersectRule.create(out));
+    consumer.accept(JdbcMinusRule.create(out));
+    consumer.accept(JdbcTableModificationRule.create(out));
+    consumer.accept(JdbcValuesRule.create(out));
   }
 
   /** Abstract base class for rule that converts to JDBC. */
   abstract static class JdbcConverterRule extends ConverterRule {
-    protected final JdbcConvention out;
-
-    @SuppressWarnings("unchecked")
-    @Deprecated // to be removed before 2.0
-    JdbcConverterRule(Class<? extends RelNode> clazz, RelTrait in,
-        JdbcConvention out, String description) {
-      this(clazz, (Predicate<RelNode>) r -> true, in, out,
-          RelFactories.LOGICAL_BUILDER, description);
-    }
-
-    <R extends RelNode> JdbcConverterRule(Class<R> clazz,
-        Predicate<? super R> predicate, RelTrait in, JdbcConvention out,
-        RelBuilderFactory relBuilderFactory, String description) {
-      super(clazz, predicate, in, out, relBuilderFactory, description);
-      this.out = out;
-    }
-
-    @SuppressWarnings({"Guava", "unchecked"})
-    @Deprecated // to be removed before 2.0
-    <R extends RelNode> JdbcConverterRule(Class<R> clazz,
-        com.google.common.base.Predicate<? super R> predicate,
-        RelTrait in, JdbcConvention out,
-        RelBuilderFactory relBuilderFactory, String description) {
-      this(clazz, (Predicate<R>) predicate, in, out, relBuilderFactory,
-          description);
+    protected JdbcConverterRule(Config config) {
+      super(config);
     }
   }
 
   /** Rule that converts a join to JDBC. */
   public static class JdbcJoinRule extends JdbcConverterRule {
-    @Deprecated // to be removed before 2.0
-    public JdbcJoinRule(JdbcConvention out) {
-      this(out, RelFactories.LOGICAL_BUILDER);
+    /** Creates a JdbcJoinRule. */
+    public static JdbcJoinRule create(JdbcConvention out) {
+      return Config.INSTANCE
+          .withConversion(Join.class, Convention.NONE, out, "JdbcJoinRule")
+          .withRuleFactory(JdbcJoinRule::new)
+          .toRule(JdbcJoinRule.class);
     }
 
-    /** Creates a JdbcJoinRule. */
-    public JdbcJoinRule(JdbcConvention out,
-        RelBuilderFactory relBuilderFactory) {
-      super(Join.class, (Predicate<RelNode>) r -> true, Convention.NONE,
-          out, relBuilderFactory, "JdbcJoinRule");
+    /** Called from the Config. */
+    protected JdbcJoinRule(Config config) {
+      super(config);
     }
 
     @Override public RelNode convert(RelNode rel) {
@@ -431,10 +420,16 @@ public class JdbcRules {
    */
   private static class JdbcCalcRule extends JdbcConverterRule {
     /** Creates a JdbcCalcRule. */
-    private JdbcCalcRule(JdbcConvention out,
-        RelBuilderFactory relBuilderFactory) {
-      super(Calc.class, (Predicate<RelNode>) r -> true, Convention.NONE,
-          out, relBuilderFactory, "JdbcCalcRule");
+    public static JdbcCalcRule create(JdbcConvention out) {
+      return Config.INSTANCE
+          .withConversion(Calc.class, Convention.NONE, out, "JdbcCalcRule")
+          .withRuleFactory(JdbcCalcRule::new)
+          .toRule(JdbcCalcRule.class);
+    }
+
+    /** Called from the Config. */
+    protected JdbcCalcRule(Config config) {
+      super(config);
     }
 
     public RelNode convert(RelNode rel) {
@@ -506,19 +501,21 @@ public class JdbcRules {
    * an {@link org.apache.calcite.adapter.jdbc.JdbcRules.JdbcProject}.
    */
   public static class JdbcProjectRule extends JdbcConverterRule {
-    @Deprecated // to be removed before 2.0
-    public JdbcProjectRule(final JdbcConvention out) {
-      this(out, RelFactories.LOGICAL_BUILDER);
+    /** Creates a JdbcProjectRule. */
+    public static JdbcProjectRule create(JdbcConvention out) {
+      return Config.INSTANCE
+          .withConversion(Project.class, project ->
+                  (out.dialect.supportsWindowFunctions()
+                      || !project.containsOver())
+                      && !userDefinedFunctionInProject(project),
+              Convention.NONE, out, "JdbcProjectRule")
+          .withRuleFactory(JdbcProjectRule::new)
+          .toRule(JdbcProjectRule.class);
     }
 
-    /** Creates a JdbcProjectRule. */
-    public JdbcProjectRule(final JdbcConvention out,
-        RelBuilderFactory relBuilderFactory) {
-      super(Project.class, (Predicate<Project>) project ->
-              (out.dialect.supportsWindowFunctions()
-                  || !RexOver.containsOver(project.getProjects(), null))
-                  && !userDefinedFunctionInProject(project),
-          Convention.NONE, out, relBuilderFactory, "JdbcProjectRule");
+    /** Called from the Config. */
+    protected JdbcProjectRule(Config config) {
+      super(config);
     }
 
     private static boolean userDefinedFunctionInProject(Project project) {
@@ -532,7 +529,7 @@ public class JdbcRules {
       return false;
     }
 
-    public RelNode convert(RelNode rel) {
+    @Override public RelNode convert(RelNode rel) {
       final Project project = (Project) rel;
 
       return new JdbcProject(
@@ -589,17 +586,18 @@ public class JdbcRules {
    * an {@link org.apache.calcite.adapter.jdbc.JdbcRules.JdbcFilter}.
    */
   public static class JdbcFilterRule extends JdbcConverterRule {
-    @Deprecated // to be removed before 2.0
-    public JdbcFilterRule(JdbcConvention out) {
-      this(out, RelFactories.LOGICAL_BUILDER);
+    /** Creates a JdbcFilterRule. */
+    public static JdbcFilterRule create(JdbcConvention out) {
+      return Config.INSTANCE
+          .withConversion(Filter.class, r -> !userDefinedFunctionInFilter(r),
+              Convention.NONE, out, "JdbcFilterRule")
+          .withRuleFactory(JdbcFilterRule::new)
+          .toRule(JdbcFilterRule.class);
     }
 
-    /** Creates a JdbcFilterRule. */
-    public JdbcFilterRule(JdbcConvention out,
-        RelBuilderFactory relBuilderFactory) {
-      super(Filter.class,
-          (Predicate<Filter>) r -> !userDefinedFunctionInFilter(r),
-          Convention.NONE, out, relBuilderFactory, "JdbcFilterRule");
+    /** Called from the Config. */
+    protected JdbcFilterRule(Config config) {
+      super(config);
     }
 
     private static boolean userDefinedFunctionInFilter(Filter filter) {
@@ -608,7 +606,7 @@ public class JdbcRules {
       return visitor.containsUserDefinedFunction();
     }
 
-    public RelNode convert(RelNode rel) {
+    @Override public RelNode convert(RelNode rel) {
       final Filter filter = (Filter) rel;
 
       return new JdbcFilter(
@@ -647,19 +645,21 @@ public class JdbcRules {
    * to a {@link org.apache.calcite.adapter.jdbc.JdbcRules.JdbcAggregate}.
    */
   public static class JdbcAggregateRule extends JdbcConverterRule {
-    @Deprecated // to be removed before 2.0
-    public JdbcAggregateRule(JdbcConvention out) {
-      this(out, RelFactories.LOGICAL_BUILDER);
+    /** Creates a JdbcAggregateRule. */
+    public static JdbcAggregateRule create(JdbcConvention out) {
+      return Config.INSTANCE
+          .withConversion(Aggregate.class, Convention.NONE, out,
+              "JdbcAggregateRule")
+          .withRuleFactory(JdbcAggregateRule::new)
+          .toRule(JdbcAggregateRule.class);
     }
 
-    /** Creates a JdbcAggregateRule. */
-    public JdbcAggregateRule(JdbcConvention out,
-        RelBuilderFactory relBuilderFactory) {
-      super(Aggregate.class, (Predicate<RelNode>) r -> true, Convention.NONE,
-          out, relBuilderFactory, "JdbcAggregateRule");
+    /** Called from the Config. */
+    protected JdbcAggregateRule(Config config) {
+      super(config);
     }
 
-    public RelNode convert(RelNode rel) {
+    @Override public RelNode convert(RelNode rel) {
       final Aggregate agg = (Aggregate) rel;
       if (agg.getGroupSets().size() != 1) {
         // GROUPING SETS not supported; see
@@ -670,7 +670,7 @@ public class JdbcRules {
           agg.getTraitSet().replace(out);
       try {
         return new JdbcAggregate(rel.getCluster(), traitSet,
-            convert(agg.getInput(), out), false, agg.getGroupSet(),
+            convert(agg.getInput(), out), agg.getGroupSet(),
             agg.getGroupSets(), agg.getAggCallList());
       } catch (InvalidRelException e) {
         LOGGER.debug(e.toString());
@@ -739,19 +739,20 @@ public class JdbcRules {
    * {@link org.apache.calcite.adapter.jdbc.JdbcRules.JdbcSort}.
    */
   public static class JdbcSortRule extends JdbcConverterRule {
-    @Deprecated // to be removed before 2.0
-    public JdbcSortRule(JdbcConvention out) {
-      this(out, RelFactories.LOGICAL_BUILDER);
+    /** Creates a JdbcSortRule. */
+    public static JdbcSortRule create(JdbcConvention out) {
+      return Config.INSTANCE
+          .withConversion(Sort.class, Convention.NONE, out, "JdbcSortRule")
+          .withRuleFactory(JdbcSortRule::new)
+          .toRule(JdbcSortRule.class);
     }
 
-    /** Creates a JdbcSortRule. */
-    public JdbcSortRule(JdbcConvention out,
-        RelBuilderFactory relBuilderFactory) {
-      super(Sort.class, (Predicate<RelNode>) r -> true, Convention.NONE, out,
-          relBuilderFactory, "JdbcSortRule");
+    /** Called from the Config. */
+    protected JdbcSortRule(Config config) {
+      super(config);
     }
 
-    public RelNode convert(RelNode rel) {
+    @Override public RelNode convert(RelNode rel) {
       return convert((Sort) rel, true);
     }
 
@@ -816,19 +817,20 @@ public class JdbcRules {
    * {@link org.apache.calcite.adapter.jdbc.JdbcRules.JdbcUnion}.
    */
   public static class JdbcUnionRule extends JdbcConverterRule {
-    @Deprecated // to be removed before 2.0
-    public JdbcUnionRule(JdbcConvention out) {
-      this(out, RelFactories.LOGICAL_BUILDER);
+    /** Creates a JdbcUnionRule. */
+    public static JdbcUnionRule create(JdbcConvention out) {
+      return Config.INSTANCE
+          .withConversion(Union.class, Convention.NONE, out, "JdbcUnionRule")
+          .withRuleFactory(JdbcUnionRule::new)
+          .toRule(JdbcUnionRule.class);
     }
 
-    /** Creates a JdbcUnionRule. */
-    public JdbcUnionRule(JdbcConvention out,
-        RelBuilderFactory relBuilderFactory) {
-      super(Union.class, (Predicate<RelNode>) r -> true, Convention.NONE, out,
-          relBuilderFactory, "JdbcUnionRule");
+    /** Called from the Config. */
+    protected JdbcUnionRule(Config config) {
+      super(config);
     }
 
-    public RelNode convert(RelNode rel) {
+    @Override public RelNode convert(RelNode rel) {
       final Union union = (Union) rel;
       final RelTraitSet traitSet =
           union.getTraitSet().replace(out);
@@ -868,13 +870,20 @@ public class JdbcRules {
    */
   public static class JdbcIntersectRule extends JdbcConverterRule {
     /** Creates a JdbcIntersectRule. */
-    private JdbcIntersectRule(JdbcConvention out,
-        RelBuilderFactory relBuilderFactory) {
-      super(Intersect.class, (Predicate<RelNode>) r -> true, Convention.NONE,
-          out, relBuilderFactory, "JdbcIntersectRule");
+    public static JdbcIntersectRule create(JdbcConvention out) {
+      return Config.INSTANCE
+          .withConversion(Intersect.class, Convention.NONE, out,
+              "JdbcIntersectRule")
+          .withRuleFactory(JdbcIntersectRule::new)
+          .toRule(JdbcIntersectRule.class);
     }
 
-    public RelNode convert(RelNode rel) {
+    /** Called from the Config. */
+    protected JdbcIntersectRule(Config config) {
+      super(config);
+    }
+
+    @Override public RelNode convert(RelNode rel) {
       final Intersect intersect = (Intersect) rel;
       if (intersect.all) {
         return null; // INTERSECT ALL not implemented
@@ -915,13 +924,19 @@ public class JdbcRules {
    */
   public static class JdbcMinusRule extends JdbcConverterRule {
     /** Creates a JdbcMinusRule. */
-    private JdbcMinusRule(JdbcConvention out,
-        RelBuilderFactory relBuilderFactory) {
-      super(Minus.class, (Predicate<RelNode>) r -> true, Convention.NONE, out,
-          relBuilderFactory, "JdbcMinusRule");
+    public static JdbcMinusRule create(JdbcConvention out) {
+      return Config.INSTANCE
+          .withConversion(Minus.class, Convention.NONE, out, "JdbcMinusRule")
+          .withRuleFactory(JdbcMinusRule::new)
+          .toRule(JdbcMinusRule.class);
     }
 
-    public RelNode convert(RelNode rel) {
+    /** Called from the Config. */
+    protected JdbcMinusRule(Config config) {
+      super(config);
+    }
+
+    @Override public RelNode convert(RelNode rel) {
       final Minus minus = (Minus) rel;
       if (minus.all) {
         return null; // EXCEPT ALL not implemented
@@ -953,11 +968,18 @@ public class JdbcRules {
 
   /** Rule that converts a table-modification to JDBC. */
   public static class JdbcTableModificationRule extends JdbcConverterRule {
-    /** Creates a JdbcTableModificationRule. */
-    private JdbcTableModificationRule(JdbcConvention out,
-        RelBuilderFactory relBuilderFactory) {
-      super(TableModify.class, (Predicate<RelNode>) r -> true,
-          Convention.NONE, out, relBuilderFactory, "JdbcTableModificationRule");
+    /** Creates a JdbcToEnumerableConverterRule. */
+    public static JdbcTableModificationRule create(JdbcConvention out) {
+      return Config.INSTANCE
+          .withConversion(TableModify.class, Convention.NONE, out,
+              "JdbcTableModificationRule")
+          .withRuleFactory(JdbcTableModificationRule::new)
+          .toRule(JdbcTableModificationRule.class);
+    }
+
+    /** Called from the Config. */
+    protected JdbcTableModificationRule(Config config) {
+      super(config);
     }
 
     @Override public RelNode convert(RelNode rel) {
@@ -1030,10 +1052,16 @@ public class JdbcRules {
   /** Rule that converts a values operator to JDBC. */
   public static class JdbcValuesRule extends JdbcConverterRule {
     /** Creates a JdbcValuesRule. */
-    private JdbcValuesRule(JdbcConvention out,
-        RelBuilderFactory relBuilderFactory) {
-      super(Values.class, (Predicate<RelNode>) r -> true, Convention.NONE,
-          out, relBuilderFactory, "JdbcValuesRule");
+    public static JdbcValuesRule create(JdbcConvention out) {
+      return Config.INSTANCE
+          .withConversion(Values.class, Convention.NONE, out, "JdbcValuesRule")
+          .withRuleFactory(JdbcValuesRule::new)
+          .toRule(JdbcValuesRule.class);
+    }
+
+    /** Called from the Config. */
+    protected JdbcValuesRule(Config config) {
+      super(config);
     }
 
     @Override public RelNode convert(RelNode rel) {
diff --git a/core/src/main/java/org/apache/calcite/adapter/jdbc/JdbcToEnumerableConverterRule.java b/core/src/main/java/org/apache/calcite/adapter/jdbc/JdbcToEnumerableConverterRule.java
index 86fbc3d..3446ba4 100644
--- a/core/src/main/java/org/apache/calcite/adapter/jdbc/JdbcToEnumerableConverterRule.java
+++ b/core/src/main/java/org/apache/calcite/adapter/jdbc/JdbcToEnumerableConverterRule.java
@@ -20,9 +20,6 @@ import org.apache.calcite.adapter.enumerable.EnumerableConvention;
 import org.apache.calcite.plan.RelTraitSet;
 import org.apache.calcite.rel.RelNode;
 import org.apache.calcite.rel.convert.ConverterRule;
-import org.apache.calcite.tools.RelBuilderFactory;
-
-import java.util.function.Predicate;
 
 /**
  * Rule to convert a relational expression from
@@ -31,11 +28,17 @@ import java.util.function.Predicate;
  */
 public class JdbcToEnumerableConverterRule extends ConverterRule {
   /** Creates a JdbcToEnumerableConverterRule. */
-  public JdbcToEnumerableConverterRule(JdbcConvention out,
-      RelBuilderFactory relBuilderFactory) {
-    super(RelNode.class, (Predicate<RelNode>) r -> true, out,
-        EnumerableConvention.INSTANCE, relBuilderFactory,
-        "JdbcToEnumerableConverterRule");
+  public static JdbcToEnumerableConverterRule create(JdbcConvention out) {
+    return Config.INSTANCE
+        .withConversion(RelNode.class, out, EnumerableConvention.INSTANCE,
+            "JdbcToEnumerableConverterRule")
+        .withRuleFactory(JdbcToEnumerableConverterRule::new)
+        .toRule(JdbcToEnumerableConverterRule.class);
+  }
+
+  /** Called from the Config. */
+  protected JdbcToEnumerableConverterRule(Config config) {
+    super(config);
   }
 
   @Override public RelNode convert(RelNode rel) {
diff --git a/core/src/main/java/org/apache/calcite/interpreter/Bindables.java b/core/src/main/java/org/apache/calcite/interpreter/Bindables.java
index a61d033..7d9adac 100644
--- a/core/src/main/java/org/apache/calcite/interpreter/Bindables.java
+++ b/core/src/main/java/org/apache/calcite/interpreter/Bindables.java
@@ -27,7 +27,7 @@ import org.apache.calcite.plan.RelOptPlanner;
 import org.apache.calcite.plan.RelOptRule;
 import org.apache.calcite.plan.RelOptRuleCall;
 import org.apache.calcite.plan.RelOptTable;
-import org.apache.calcite.plan.RelOptUtil;
+import org.apache.calcite.plan.RelRule;
 import org.apache.calcite.plan.RelTraitSet;
 import org.apache.calcite.rel.InvalidRelException;
 import org.apache.calcite.rel.RelCollation;
@@ -45,7 +45,6 @@ import org.apache.calcite.rel.core.JoinRelType;
 import org.apache.calcite.rel.core.Match;
 import org.apache.calcite.rel.core.Minus;
 import org.apache.calcite.rel.core.Project;
-import org.apache.calcite.rel.core.RelFactories;
 import org.apache.calcite.rel.core.SetOp;
 import org.apache.calcite.rel.core.Sort;
 import org.apache.calcite.rel.core.TableScan;
@@ -85,7 +84,6 @@ import java.util.Map;
 import java.util.Objects;
 import java.util.Set;
 import java.util.SortedSet;
-import java.util.function.Predicate;
 
 /**
  * Utilities pertaining to {@link BindableRel} and {@link BindableConvention}.
@@ -94,40 +92,45 @@ public class Bindables {
   private Bindables() {}
 
   public static final RelOptRule BINDABLE_TABLE_SCAN_RULE =
-      new BindableTableScanRule(RelFactories.LOGICAL_BUILDER);
+      BindableTableScanRule.Config.DEFAULT.toRule();
 
   public static final RelOptRule BINDABLE_FILTER_RULE =
-      new BindableFilterRule(RelFactories.LOGICAL_BUILDER);
+      BindableFilterRule.DEFAULT_CONFIG.toRule(BindableFilterRule.class);
 
   public static final RelOptRule BINDABLE_PROJECT_RULE =
-      new BindableProjectRule(RelFactories.LOGICAL_BUILDER);
+      BindableProjectRule.DEFAULT_CONFIG.toRule(BindableProjectRule.class);
 
   public static final RelOptRule BINDABLE_SORT_RULE =
-      new BindableSortRule(RelFactories.LOGICAL_BUILDER);
+      BindableSortRule.DEFAULT_CONFIG.toRule(BindableSortRule.class);
 
   public static final RelOptRule BINDABLE_JOIN_RULE =
-      new BindableJoinRule(RelFactories.LOGICAL_BUILDER);
+      BindableJoinRule.DEFAULT_CONFIG.toRule(BindableJoinRule.class);
 
+  public static final RelOptRule BINDABLE_SET_OP_RULE =
+      BindableSetOpRule.DEFAULT_CONFIG.toRule(BindableSetOpRule.class);
+
+  /** @deprecated Use {@link #BINDABLE_SET_OP_RULE}. */
   public static final RelOptRule BINDABLE_SETOP_RULE =
-      new BindableSetOpRule(RelFactories.LOGICAL_BUILDER);
+      BINDABLE_SET_OP_RULE;
 
   public static final RelOptRule BINDABLE_VALUES_RULE =
-      new BindableValuesRule(RelFactories.LOGICAL_BUILDER);
+      BindableValuesRule.DEFAULT_CONFIG.toRule(BindableValuesRule.class);
 
   public static final RelOptRule BINDABLE_AGGREGATE_RULE =
-      new BindableAggregateRule(RelFactories.LOGICAL_BUILDER);
+      BindableAggregateRule.DEFAULT_CONFIG.toRule(BindableAggregateRule.class);
 
   public static final RelOptRule BINDABLE_WINDOW_RULE =
-      new BindableWindowRule(RelFactories.LOGICAL_BUILDER);
+      BindableWindowRule.DEFAULT_CONFIG.toRule(BindableWindowRule.class);
 
   public static final RelOptRule BINDABLE_MATCH_RULE =
-      new BindableMatchRule(RelFactories.LOGICAL_BUILDER);
+      BindableMatchRule.DEFAULT_CONFIG.toRule(BindableMatchRule.class);
 
   /** Rule that converts a relational expression from
    * {@link org.apache.calcite.plan.Convention#NONE}
    * to {@link org.apache.calcite.interpreter.BindableConvention}. */
-  public static final ConverterRule FROM_NONE_RULE =
-      new NoneToBindableConverterRule(RelFactories.LOGICAL_BUILDER);
+  public static final NoneToBindableConverterRule FROM_NONE_RULE =
+      NoneToBindableConverterRule.DEFAULT_CONFIG
+          .toRule(NoneToBindableConverterRule.class);
 
   /** All rules that convert logical relational expression to bindable. */
   public static final ImmutableList<RelOptRule> RULES =
@@ -137,7 +140,7 @@ public class Bindables {
           BINDABLE_PROJECT_RULE,
           BINDABLE_SORT_RULE,
           BINDABLE_JOIN_RULE,
-          BINDABLE_SETOP_RULE,
+          BINDABLE_SET_OP_RULE,
           BINDABLE_VALUES_RULE,
           BINDABLE_AGGREGATE_RULE,
           BINDABLE_WINDOW_RULE,
@@ -155,16 +158,20 @@ public class Bindables {
   }
 
   /** Rule that converts a {@link org.apache.calcite.rel.core.TableScan}
-   * to bindable convention. */
-  public static class BindableTableScanRule extends RelOptRule {
+   * to bindable convention.
+   *
+   * @see #BINDABLE_TABLE_SCAN_RULE */
+  public static class BindableTableScanRule
+      extends RelRule<BindableTableScanRule.Config> {
+    /** Called from Config. */
+    protected BindableTableScanRule(Config config) {
+      super(config);
+    }
 
-    /**
-     * Creates a BindableTableScanRule.
-     *
-     * @param relBuilderFactory Builder for relational expressions
-     */
+    @Deprecated // to be removed before 2.0
     public BindableTableScanRule(RelBuilderFactory relBuilderFactory) {
-      super(operand(LogicalTableScan.class, none()), relBuilderFactory, null);
+      this(Config.DEFAULT.withRelBuilderFactory(relBuilderFactory)
+          .as(Config.class));
     }
 
     @Override public void onMatch(RelOptRuleCall call) {
@@ -175,6 +182,18 @@ public class Bindables {
             BindableTableScan.create(scan.getCluster(), table));
       }
     }
+
+    /** Rule configuration. */
+    public interface Config extends RelRule.Config {
+      Config DEFAULT = EMPTY
+          .withOperandSupplier(b ->
+              b.operand(LogicalTableScan.class).noInputs())
+          .as(Config.class);
+
+      @Override default BindableTableScanRule toRule() {
+        return new BindableTableScanRule(this);
+      }
+    }
   }
 
   /** Scan of a table that implements {@link ScannableTable} and therefore can
@@ -277,22 +296,23 @@ public class Bindables {
     }
   }
 
-  /** Rule that converts a {@link Filter} to bindable convention. */
+  /** Rule that converts a {@link Filter} to bindable convention.
+   *
+   * @see #BINDABLE_FILTER_RULE */
   public static class BindableFilterRule extends ConverterRule {
+    /** Default configuration. */
+    public static final Config DEFAULT_CONFIG = Config.INSTANCE
+        .withConversion(LogicalFilter.class, f -> !f.containsOver(),
+            Convention.NONE, BindableConvention.INSTANCE,
+            "BindableFilterRule")
+        .withRuleFactory(BindableFilterRule::new);
 
-    /**
-     * Creates a BindableFilterRule.
-     *
-     * @param relBuilderFactory Builder for relational expressions
-     */
-    public BindableFilterRule(RelBuilderFactory relBuilderFactory) {
-      super(LogicalFilter.class,
-          (Predicate<LogicalFilter>) RelOptUtil::notContainsWindowedAgg,
-          Convention.NONE, BindableConvention.INSTANCE, relBuilderFactory,
-          "BindableFilterRule");
+    /** Called from the Config. */
+    protected BindableFilterRule(Config config) {
+      super(config);
     }
 
-    public RelNode convert(RelNode rel) {
+    @Override public RelNode convert(RelNode rel) {
       final LogicalFilter filter = (LogicalFilter) rel;
       return BindableFilter.create(
           convert(filter.getInput(),
@@ -344,22 +364,23 @@ public class Bindables {
   /**
    * Rule to convert a {@link org.apache.calcite.rel.logical.LogicalProject}
    * to a {@link BindableProject}.
+   *
+   * @see #BINDABLE_PROJECT_RULE
    */
   public static class BindableProjectRule extends ConverterRule {
+    /** Default configuration. */
+    public static final Config DEFAULT_CONFIG = Config.INSTANCE
+        .withConversion(LogicalProject.class, p -> !p.containsOver(),
+            Convention.NONE, BindableConvention.INSTANCE,
+            "BindableProjectRule")
+        .withRuleFactory(BindableProjectRule::new);
 
-    /**
-     * Creates a BindableProjectRule.
-     *
-     * @param relBuilderFactory Builder for relational expressions
-     */
-    public BindableProjectRule(RelBuilderFactory relBuilderFactory) {
-      super(LogicalProject.class,
-          (Predicate<LogicalProject>) RelOptUtil::notContainsWindowedAgg,
-          Convention.NONE, BindableConvention.INSTANCE, relBuilderFactory,
-          "BindableProjectRule");
+    /** Called from the Config. */
+    protected BindableProjectRule(Config config) {
+      super(config);
     }
 
-    public RelNode convert(RelNode rel) {
+    @Override public RelNode convert(RelNode rel) {
       final LogicalProject project = (LogicalProject) rel;
       return new BindableProject(rel.getCluster(),
           rel.getTraitSet().replace(BindableConvention.INSTANCE),
@@ -402,20 +423,22 @@ public class Bindables {
   /**
    * Rule to convert an {@link org.apache.calcite.rel.core.Sort} to a
    * {@link org.apache.calcite.interpreter.Bindables.BindableSort}.
+   *
+   * @see #BINDABLE_SORT_RULE
    */
   public static class BindableSortRule extends ConverterRule {
+    /** Default configuration. */
+    public static final Config DEFAULT_CONFIG = Config.INSTANCE
+        .withConversion(Sort.class, Convention.NONE,
+            BindableConvention.INSTANCE, "BindableSortRule")
+        .withRuleFactory(BindableSortRule::new);
 
-    /**
-     * Creates a BindableSortRule.
-     *
-     * @param relBuilderFactory Builder for relational expressions
-     */
-    public BindableSortRule(RelBuilderFactory relBuilderFactory) {
-      super(Sort.class, (Predicate<RelNode>) r -> true, Convention.NONE,
-          BindableConvention.INSTANCE, relBuilderFactory, "BindableSortRule");
+    /** Called from the Config. */
+    protected BindableSortRule(Config config) {
+      super(config);
     }
 
-    public RelNode convert(RelNode rel) {
+    @Override public RelNode convert(RelNode rel) {
       final Sort sort = (Sort) rel;
       final RelTraitSet traitSet =
           sort.getTraitSet().replace(BindableConvention.INSTANCE);
@@ -458,21 +481,22 @@ public class Bindables {
   /**
    * Rule to convert a {@link org.apache.calcite.rel.logical.LogicalJoin}
    * to a {@link BindableJoin}.
+   *
+   * @see #BINDABLE_JOIN_RULE
    */
   public static class BindableJoinRule extends ConverterRule {
+    /** Default configuration. */
+    public static final Config DEFAULT_CONFIG = Config.INSTANCE
+        .withConversion(LogicalJoin.class, Convention.NONE,
+            BindableConvention.INSTANCE, "BindableJoinRule")
+        .withRuleFactory(BindableJoinRule::new);
 
-    /**
-     * Creates a BindableJoinRule.
-     *
-     * @param relBuilderFactory Builder for relational expressions
-     */
-    public BindableJoinRule(RelBuilderFactory relBuilderFactory) {
-      super(LogicalJoin.class, (Predicate<RelNode>) r -> true,
-          Convention.NONE, BindableConvention.INSTANCE, relBuilderFactory,
-          "BindableJoinRule");
+    /** Called from the Config. */
+    protected BindableJoinRule(Config config) {
+      super(config);
     }
 
-    public RelNode convert(RelNode rel) {
+    @Override public RelNode convert(RelNode rel) {
       final LogicalJoin join = (LogicalJoin) rel;
       final BindableConvention out = BindableConvention.INSTANCE;
       final RelTraitSet traitSet = join.getTraitSet().replace(out);
@@ -529,21 +553,22 @@ public class Bindables {
   /**
    * Rule to convert an {@link SetOp} to a {@link BindableUnion}
    * or {@link BindableIntersect} or {@link BindableMinus}.
+   *
+   * @see #BINDABLE_SET_OP_RULE
    */
   public static class BindableSetOpRule extends ConverterRule {
+    /** Default configuration. */
+    public static final Config DEFAULT_CONFIG = Config.INSTANCE
+        .withConversion(SetOp.class, Convention.NONE,
+            BindableConvention.INSTANCE, "BindableSetOpRule")
+        .withRuleFactory(BindableSetOpRule::new);
 
-    /**
-     * Creates a BindableSetOpRule.
-     *
-     * @param relBuilderFactory Builder for relational expressions
-     */
-    public BindableSetOpRule(RelBuilderFactory relBuilderFactory) {
-      super(SetOp.class, (Predicate<RelNode>) r -> true,
-          Convention.NONE, BindableConvention.INSTANCE, relBuilderFactory,
-          "BindableSetOpRule");
+    /** Called from the Config. */
+    protected BindableSetOpRule(Config config) {
+      super(config);
     }
 
-    public RelNode convert(RelNode rel) {
+    @Override public RelNode convert(RelNode rel) {
       final SetOp setOp = (SetOp) rel;
       final BindableConvention out = BindableConvention.INSTANCE;
       final RelTraitSet traitSet = setOp.getTraitSet().replace(out);
@@ -662,18 +687,19 @@ public class Bindables {
     }
   }
 
-  /** Rule that converts a {@link Values} to bindable convention. */
+  /** Rule that converts a {@link Values} to bindable convention.
+   *
+   * @see #BINDABLE_VALUES_RULE */
   public static class BindableValuesRule extends ConverterRule {
+    /** Default configuration. */
+    public static final Config DEFAULT_CONFIG = Config.INSTANCE
+        .withConversion(LogicalValues.class, Convention.NONE,
+            BindableConvention.INSTANCE, "BindableValuesRule")
+        .withRuleFactory(BindableValuesRule::new);
 
-    /**
-     * Creates a BindableValuesRule.
-     *
-     * @param relBuilderFactory Builder for relational expressions
-     */
-    public BindableValuesRule(RelBuilderFactory relBuilderFactory) {
-      super(LogicalValues.class, (Predicate<RelNode>) r -> true,
-          Convention.NONE, BindableConvention.INSTANCE, relBuilderFactory,
-          "BindableValuesRule");
+    /** Called from the Config. */
+    protected BindableValuesRule(Config config) {
+      super(config);
     }
 
     @Override public RelNode convert(RelNode rel) {
@@ -748,21 +774,22 @@ public class Bindables {
     }
   }
 
-  /** Rule that converts an {@link Aggregate} to bindable convention. */
+  /** Rule that converts an {@link Aggregate} to bindable convention.
+   *
+   * @see #BINDABLE_AGGREGATE_RULE */
   public static class BindableAggregateRule extends ConverterRule {
+    /** Default configuration. */
+    public static final Config DEFAULT_CONFIG = Config.INSTANCE
+        .withConversion(LogicalAggregate.class, Convention.NONE,
+            BindableConvention.INSTANCE, "BindableAggregateRule")
+        .withRuleFactory(BindableAggregateRule::new);
 
-    /**
-     * Creates a BindableAggregateRule.
-     *
-     * @param relBuilderFactory Builder for relational expressions
-     */
-    public BindableAggregateRule(RelBuilderFactory relBuilderFactory) {
-      super(LogicalAggregate.class, (Predicate<RelNode>) r -> true,
-          Convention.NONE, BindableConvention.INSTANCE, relBuilderFactory,
-          "BindableAggregateRule");
+    /** Called from the Config. */
+    protected BindableAggregateRule(Config config) {
+      super(config);
     }
 
-    public RelNode convert(RelNode rel) {
+    @Override public RelNode convert(RelNode rel) {
       final LogicalAggregate agg = (LogicalAggregate) rel;
       final RelTraitSet traitSet =
           agg.getTraitSet().replace(BindableConvention.INSTANCE);
@@ -810,24 +837,23 @@ public class Bindables {
     }
   }
 
-  /**
-   * Rule to convert a {@link org.apache.calcite.rel.logical.LogicalWindow}
+  /** Rule to convert a {@link org.apache.calcite.rel.logical.LogicalWindow}
    * to a {@link BindableWindow}.
-   */
+   *
+   * @see #BINDABLE_WINDOW_RULE */
   public static class BindableWindowRule extends ConverterRule {
+    /** Default configuration. */
+    public static final Config DEFAULT_CONFIG = Config.INSTANCE
+        .withConversion(LogicalWindow.class, Convention.NONE,
+            BindableConvention.INSTANCE, "BindableWindowRule")
+        .withRuleFactory(BindableWindowRule::new);
 
-    /**
-     * Creates a BindableWindowRule.
-     *
-     * @param relBuilderFactory Builder for relational expressions
-     */
-    public BindableWindowRule(RelBuilderFactory relBuilderFactory) {
-      super(LogicalWindow.class, (Predicate<RelNode>) r -> true,
-          Convention.NONE, BindableConvention.INSTANCE, relBuilderFactory,
-          "BindableWindowRule");
+    /** Called from the Config. */
+    protected BindableWindowRule(Config config) {
+      super(config);
     }
 
-    public RelNode convert(RelNode rel) {
+    @Override public RelNode convert(RelNode rel) {
       final LogicalWindow winAgg = (LogicalWindow) rel;
       final RelTraitSet traitSet =
           winAgg.getTraitSet().replace(BindableConvention.INSTANCE);
@@ -843,7 +869,7 @@ public class Bindables {
   /** Implementation of {@link org.apache.calcite.rel.core.Match}
    * in bindable convention. */
   public static class BindableMatch extends Match implements BindableRel {
-    /** Creates a BindableMatch. */
+    /** Singleton instance of BindableMatch. */
     BindableMatch(RelOptCluster cluster, RelTraitSet traitSet, RelNode input,
         RelDataType rowType, RexNode pattern, boolean strictStart,
         boolean strictEnd, Map<String, RexNode> patternDefinitions,
@@ -878,21 +904,22 @@ public class Bindables {
   /**
    * Rule to convert a {@link org.apache.calcite.rel.logical.LogicalMatch}
    * to a {@link BindableMatch}.
+   *
+   * @see #BINDABLE_MATCH_RULE
    */
   public static class BindableMatchRule extends ConverterRule {
+    /** Default configuration. */
+    public static final Config DEFAULT_CONFIG = Config.INSTANCE
+        .withConversion(LogicalMatch.class, Convention.NONE,
+            BindableConvention.INSTANCE, "BindableMatchRule")
+        .withRuleFactory(BindableMatchRule::new);
 
-    /**
-     * Creates a BindableMatchRule.
-     *
-     * @param relBuilderFactory Builder for relational expressions
-     */
-    public BindableMatchRule(RelBuilderFactory relBuilderFactory) {
-      super(LogicalMatch.class, (Predicate<RelNode>) r -> true,
-          Convention.NONE, BindableConvention.INSTANCE, relBuilderFactory,
-          "BindableMatchRule");
+    /** Called from the Config. */
+    protected BindableMatchRule(Config config) {
+      super(config);
     }
 
-    public RelNode convert(RelNode rel) {
+    @Override public RelNode convert(RelNode rel) {
       final LogicalMatch match = (LogicalMatch) rel;
       final RelTraitSet traitSet =
           match.getTraitSet().replace(BindableConvention.INSTANCE);
diff --git a/core/src/main/java/org/apache/calcite/interpreter/NoneToBindableConverterRule.java b/core/src/main/java/org/apache/calcite/interpreter/NoneToBindableConverterRule.java
index 17dc359..bdf78ab 100644
--- a/core/src/main/java/org/apache/calcite/interpreter/NoneToBindableConverterRule.java
+++ b/core/src/main/java/org/apache/calcite/interpreter/NoneToBindableConverterRule.java
@@ -20,30 +20,25 @@ import org.apache.calcite.plan.Convention;
 import org.apache.calcite.plan.RelTraitSet;
 import org.apache.calcite.rel.RelNode;
 import org.apache.calcite.rel.convert.ConverterRule;
-import org.apache.calcite.tools.RelBuilderFactory;
 
-import java.util.function.Predicate;
 
 /**
  * Rule to convert a relational expression from
  * {@link org.apache.calcite.plan.Convention#NONE}
  * to {@link org.apache.calcite.interpreter.BindableConvention}.
+ *
+ * @see Bindables#FROM_NONE_RULE
  */
 public class NoneToBindableConverterRule extends ConverterRule {
-  /** @deprecated Use {@link Bindables#FROM_NONE_RULE}. */
-  @Deprecated // to be removed before 1.25
-  public static final ConverterRule INSTANCE =
-      Bindables.FROM_NONE_RULE;
+  /** Default configuration. */
+  public static final Config DEFAULT_CONFIG = Config.INSTANCE
+      .withConversion(RelNode.class, Convention.NONE,
+          BindableConvention.INSTANCE, "NoneToBindableConverterRule")
+      .withRuleFactory(NoneToBindableConverterRule::new);
 
-  /**
-   * Creates a NoneToBindableConverterRule.
-   *
-   * @param relBuilderFactory Builder for relational expressions
-   */
-  public NoneToBindableConverterRule(RelBuilderFactory relBuilderFactory) {
-    super(RelNode.class, (Predicate<RelNode>) r -> true, Convention.NONE,
-        BindableConvention.INSTANCE, relBuilderFactory,
-        "NoneToBindableConverterRule");
+  /** Called from the Config. */
+  protected NoneToBindableConverterRule(Config config) {
+    super(config);
   }
 
   @Override public RelNode convert(RelNode rel) {
diff --git a/core/src/main/java/org/apache/calcite/plan/CommonRelSubExprRule.java b/core/src/main/java/org/apache/calcite/plan/CommonRelSubExprRule.java
index e5ffda5..256f25e 100644
--- a/core/src/main/java/org/apache/calcite/plan/CommonRelSubExprRule.java
+++ b/core/src/main/java/org/apache/calcite/plan/CommonRelSubExprRule.java
@@ -22,15 +22,24 @@ package org.apache.calcite.plan;
  * that are fired only on relational expressions that appear more than once
  * in a query tree.
  */
-public abstract class CommonRelSubExprRule extends RelOptRule {
+
+// TODO: obsolete this?
+public abstract class CommonRelSubExprRule
+    extends RelRule<CommonRelSubExprRule.Config> {
   //~ Constructors -----------------------------------------------------------
 
-  /**
-   * Creates a <code>CommonRelSubExprRule</code>.
-   *
-   * @param operand root operand, must not be null
-   */
+  /** Creates a CommonRelSubExprRule. */
+  protected CommonRelSubExprRule(Config config) {
+    super(config);
+  }
+
+  @Deprecated // to be removed before 2.0
   public CommonRelSubExprRule(RelOptRuleOperand operand) {
-    super(operand);
+    this(Config.EMPTY.withOperandSupplier(b -> b.exactly(operand))
+        .as(Config.class));
+  }
+
+  /** Rule configuration. */
+  public interface Config extends RelRule.Config {
   }
 }
diff --git a/core/src/main/java/org/apache/calcite/plan/RelOptMaterialization.java b/core/src/main/java/org/apache/calcite/plan/RelOptMaterialization.java
index b0d84fa..a8823e3 100644
--- a/core/src/main/java/org/apache/calcite/plan/RelOptMaterialization.java
+++ b/core/src/main/java/org/apache/calcite/plan/RelOptMaterialization.java
@@ -195,8 +195,8 @@ public class RelOptMaterialization {
       return null;
     }
     final Program program = Programs.hep(
-        ImmutableList.of(CoreRules.PROJECT_FILTER_TRANSPOSE,
-            CoreRules.AGGREGATE_PROJECT_MERGE,
+        ImmutableList.of(
+            CoreRules.PROJECT_FILTER_TRANSPOSE, CoreRules.AGGREGATE_PROJECT_MERGE,
             CoreRules.AGGREGATE_FILTER_TRANSPOSE),
         false,
         DefaultRelMetadataProvider.INSTANCE);
diff --git a/core/src/main/java/org/apache/calcite/plan/RelOptRule.java b/core/src/main/java/org/apache/calcite/plan/RelOptRule.java
index 1a066c0..6eea4b8 100644
--- a/core/src/main/java/org/apache/calcite/plan/RelOptRule.java
+++ b/core/src/main/java/org/apache/calcite/plan/RelOptRule.java
@@ -29,6 +29,7 @@ import java.util.ArrayList;
 import java.util.List;
 import java.util.Objects;
 import java.util.function.Predicate;
+import javax.annotation.Nonnull;
 
 /**
  * A <code>RelOptRule</code> transforms an expression into another. It has a
@@ -120,7 +121,10 @@ public abstract class RelOptRule {
    * @param <R> Class of relational expression to match
    * @return Operand that matches a relational expression that has no
    *   children
+   *
+   * @deprecated Use {@link RelRule.OperandBuilder#operand(Class)}
    */
+  @Deprecated // to be removed before 2.0
   public static <R extends RelNode> RelOptRuleOperand operand(
       Class<R> clazz,
       RelOptRuleOperandChildren operandList) {
@@ -138,7 +142,10 @@ public abstract class RelOptRule {
    * @param <R> Class of relational expression to match
    * @return Operand that matches a relational expression that has no
    *   children
+   *
+   * @deprecated Use {@link RelRule.OperandBuilder#operand(Class)}
    */
+  @Deprecated // to be removed before 2.0
   public static <R extends RelNode> RelOptRuleOperand operand(
       Class<R> clazz,
       RelTrait trait,
@@ -158,7 +165,10 @@ public abstract class RelOptRule {
    * @param <R> Class of relational expression to match
    * @return Operand that matches a relational expression that has a
    *   particular trait and predicate
+   *
+   * @deprecated Use {@link RelRule.OperandBuilder#operand(Class)}
    */
+  @Deprecated // to be removed before 2.0
   public static <R extends RelNode> RelOptRuleOperand operandJ(
       Class<R> clazz,
       RelTrait trait,
@@ -191,7 +201,10 @@ public abstract class RelOptRule {
    * @param rest Rest operands
    * @param <R> Class of relational expression to match
    * @return Operand
+   *
+   * @deprecated Use {@link RelRule.OperandBuilder#operand(Class)}
    */
+  @Deprecated // to be removed before 2.0
   public static <R extends RelNode> RelOptRuleOperand operandJ(
       Class<R> clazz,
       RelTrait trait,
@@ -231,7 +244,10 @@ public abstract class RelOptRule {
    * @param <R> Class of relational expression to match
    * @return Operand that matches a relational expression with a given
    *   list of children
+   *
+   * @deprecated Use {@link RelRule.OperandBuilder#operand(Class)}
    */
+  @Deprecated // to be removed before 2.0
   public static <R extends RelNode> RelOptRuleOperand operand(
       Class<R> clazz,
       RelOptRuleOperand first,
@@ -246,6 +262,7 @@ public abstract class RelOptRule {
    * @param trait    Trait to match, or null to match any trait
    * @param predicate Predicate to apply to relational expression
    */
+  @Deprecated // to be removed before 2.0
   protected static <R extends RelNode> ConverterRelOptRuleOperand
       convertOperand(Class<R> clazz, Predicate<? super R> predicate,
       RelTrait trait) {
@@ -272,7 +289,10 @@ public abstract class RelOptRule {
    * @param rest  Remaining child operands (may be empty)
    * @return List of child operands that matches child relational
    *   expressions in the order
+   *
+   * @deprecated Use {@link RelRule.OperandDetailBuilder#inputs}
    */
+  @Deprecated // to be removed before 2.0
   public static RelOptRuleOperandChildren some(
       RelOptRuleOperand first,
       RelOptRuleOperand... rest) {
@@ -307,6 +327,7 @@ public abstract class RelOptRule {
    * @return List of child operands that matches child relational
    *   expressions in any order
    */
+  @Deprecated // to be removed before 2.0
   public static RelOptRuleOperandChildren unordered(
       RelOptRuleOperand first,
       RelOptRuleOperand... rest) {
@@ -319,7 +340,10 @@ public abstract class RelOptRule {
    * Creates an empty list of child operands.
    *
    * @return Empty list of child operands
+   *
+   * @deprecated Use {@link RelRule.OperandDetailBuilder#noInputs()}
    */
+  @Deprecated // to be removed before 2.0
   public static RelOptRuleOperandChildren none() {
     return RelOptRuleOperandChildren.LEAF_CHILDREN;
   }
@@ -330,7 +354,10 @@ public abstract class RelOptRule {
    *
    * @return List of child operands that signifies that the operand matches
    *   any number of child relational expressions
+   *
+   * @deprecated Use {@link RelRule.OperandDetailBuilder#anyInputs()}
    */
+  @Deprecated // to be removed before 2.0
   public static RelOptRuleOperandChildren any() {
     return RelOptRuleOperandChildren.ANY_CHILDREN;
   }
@@ -447,10 +474,11 @@ public abstract class RelOptRule {
    * @param that Another rule
    * @return Whether this rule is equal to another rule
    */
-  protected boolean equals(RelOptRule that) {
+  protected boolean equals(@Nonnull RelOptRule that) {
     // Include operands and class in the equality criteria just in case
     // they have chosen a poor description.
-    return this == that || this.getClass() == that.getClass()
+    return this == that
+        || this.getClass() == that.getClass()
         && this.description.equals(that.description)
         && this.operand.equals(that.operand);
   }
@@ -640,7 +668,7 @@ public abstract class RelOptRule {
   /**
    * Operand to an instance of the converter rule.
    */
-  private static class ConverterRelOptRuleOperand extends RelOptRuleOperand {
+  protected static class ConverterRelOptRuleOperand extends RelOptRuleOperand {
     <R extends RelNode> ConverterRelOptRuleOperand(Class<R> clazz, RelTrait in,
         Predicate<? super R> predicate) {
       super(clazz, in, predicate, RelOptRuleOperandChildPolicy.ANY,
diff --git a/core/src/main/java/org/apache/calcite/plan/RelOptRuleCall.java b/core/src/main/java/org/apache/calcite/plan/RelOptRuleCall.java
index 861b658..fd5f50f 100644
--- a/core/src/main/java/org/apache/calcite/plan/RelOptRuleCall.java
+++ b/core/src/main/java/org/apache/calcite/plan/RelOptRuleCall.java
@@ -177,7 +177,7 @@ public abstract class RelOptRuleCall {
 
   /** Assigns the input relational expressions of a given relational expression,
    * as seen by this particular call. Is only called when the operand is
-   * {@link RelOptRule#any()}. */
+   * {@link RelRule.OperandDetailBuilder#anyInputs() any}. */
   protected void setChildRels(RelNode rel, List<RelNode> inputs) {
     if (nodeInputs.isEmpty()) {
       nodeInputs = new HashMap<>();
diff --git a/core/src/main/java/org/apache/calcite/plan/RelOptRuleOperandChildren.java b/core/src/main/java/org/apache/calcite/plan/RelOptRuleOperandChildren.java
index 98539a8..2ef1465 100644
--- a/core/src/main/java/org/apache/calcite/plan/RelOptRuleOperandChildren.java
+++ b/core/src/main/java/org/apache/calcite/plan/RelOptRuleOperandChildren.java
@@ -28,8 +28,11 @@ import java.util.List;
  * {@link RelOptRule#some},
  * {@link RelOptRule#none},
  * {@link RelOptRule#any},
- * {@link RelOptRule#unordered},</p>
+ * {@link RelOptRule#unordered}.
+ *
+ * @deprecated Use {@link RelRule.OperandBuilder}
  */
+@Deprecated // to be removed before 2.0
 public class RelOptRuleOperandChildren {
   static final RelOptRuleOperandChildren ANY_CHILDREN =
       new RelOptRuleOperandChildren(
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 2e634f8..f7c1ca7 100644
--- a/core/src/main/java/org/apache/calcite/plan/RelOptUtil.java
+++ b/core/src/main/java/org/apache/calcite/plan/RelOptUtil.java
@@ -144,8 +144,7 @@ public abstract class RelOptUtil {
   @SuppressWarnings("Guava")
   @Deprecated // to be removed before 2.0
   public static final com.google.common.base.Predicate<Filter>
-      FILTER_PREDICATE =
-      RelOptUtil::notContainsWindowedAgg;
+      FILTER_PREDICATE = f -> !f.containsOver();
 
   @SuppressWarnings("Guava")
   @Deprecated // to be removed before 2.0
@@ -2060,7 +2059,6 @@ public abstract class RelOptUtil {
     }
 
     planner.addRule(CoreRules.FILTER_REDUCE_EXPRESSIONS);
-
   }
 
   @Deprecated // to be removed before 1.25
@@ -2979,7 +2977,7 @@ public abstract class RelOptUtil {
       return null;
     }
     if (RexOver.containsOver(nodes, null)
-        && RexOver.containsOver(project.getProjects(), null)) {
+        && project.containsOver()) {
       // Is it valid relational algebra to apply windowed function to a windowed
       // function? Possibly. But it's invalid SQL, so don't go there.
       return null;
@@ -3427,17 +3425,17 @@ public abstract class RelOptUtil {
 
   /** Predicate for if a {@link Calc} does not contain windowed aggregates. */
   public static boolean notContainsWindowedAgg(Calc calc) {
-    return !calc.getProgram().containsAggs();
+    return !calc.containsOver();
   }
 
   /** Predicate for if a {@link Filter} does not windowed aggregates. */
   public static boolean notContainsWindowedAgg(Filter filter) {
-    return !RexOver.containsOver(filter.getCondition());
+    return !filter.containsOver();
   }
 
   /** Predicate for if a {@link Project} does not contain windowed aggregates. */
   public static boolean notContainsWindowedAgg(Project project) {
-    return !RexOver.containsOver(project.getProjects(), null);
+    return !project.containsOver();
   }
 
   /** Policies for handling two- and three-valued boolean logic. */
diff --git a/core/src/main/java/org/apache/calcite/plan/RelRule.java b/core/src/main/java/org/apache/calcite/plan/RelRule.java
new file mode 100644
index 0000000..edf62de
--- /dev/null
+++ b/core/src/main/java/org/apache/calcite/plan/RelRule.java
@@ -0,0 +1,324 @@
+/*
+ * 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.rel.RelNode;
+import org.apache.calcite.rel.core.RelFactories;
+import org.apache.calcite.tools.RelBuilderFactory;
+import org.apache.calcite.util.ImmutableBeans;
+
+import com.google.common.collect.ImmutableList;
+
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Objects;
+import java.util.function.BiConsumer;
+import java.util.function.Function;
+import java.util.function.Predicate;
+import javax.annotation.Nonnull;
+
+/**
+ * Rule that is parameterized via a configuration.
+ *
+ * <p>Eventually (before Calcite version 2.0), this class will replace
+ * {@link RelOptRule}. Constructors of {@code RelOptRule} are deprecated, so new
+ * rule classes should extend {@code RelRule}, not {@code RelOptRule}.
+ * Next, we will deprecate {@code RelOptRule}, so that variables that reference
+ * rules will be of type {@code RelRule}.
+ *
+ * <p><b>Guidelines for writing rules</b>
+ *
+ * <p>1. If your rule is a sub-class of
+ * {@link org.apache.calcite.rel.convert.ConverterRule}
+ * and does not need any extra properties,
+ * there's no need to create an {@code interface Config} inside your class.
+ * In your class, create a constant
+ * {@code public static final Config DEFAULT_CONFIG}. Goto step 5.
+ *
+ * <p>2. If your rule is not a sub-class of
+ * {@link org.apache.calcite.rel.convert.ConverterRule},
+ * create an inner {@code interface Config extends RelRule.Config}.
+ * Implement {@link Config#toRule() toRule} using a {@code default} method:
+ *
+ * <blockquote>
+ * <code>
+ * &#x40;Override default CsvProjectTableScanRule toRule() {<br>
+ * &nbsp;&nbsp;return new CsvProjectTableScanRule(this);<br>
+ * }
+ * </code>
+ * </blockquote>
+ *
+ * <p>3. For each configuration property, create a pair of methods in your
+ * {@code Config} interface. For example, for a property {@code foo} of type
+ * {@code int}, create methods {@code foo} and {@code withFoo}:
+ *
+ * <blockquote><pre><code>
+ * &#x2f;** Returns foo. *&#x2f;
+ * &#x40;ImmutableBeans.Property
+ * int foo();
+ *
+ * &#x2f;** Sets {&#x40;link #foo}. *&#x2f;
+ * Config withFoo(int x);
+ * </code></pre></blockquote>
+ *
+ * <p>4. In your {@code Config} interface, create a {@code DEFAULT} constant
+ * that represents the most typical configuration of your rule. For example,
+ * {@code CsvProjectTableScanRule.Config} has the following:
+ *
+ * <blockquote><pre><code>
+ * Config DEFAULT = EMPTY
+ *     .withOperandSupplier(b0 -&gt;
+ *         b0.operand(LogicalProject.class).oneInput(b1 -&gt;
+ *             b1.operand(CsvTableScan.class).noInputs()))
+ *      .as(Config.class);
+ * </code></pre></blockquote>
+ *
+ * <p>5. Do not create an {@code INSTANCE} constant inside your rule.
+ * Instead, create a named instance of your rule, with default configuration,
+ * in a holder class. The holder class must not be a sub-class of
+ * {@code RelOptRule} (otherwise cyclic class-loading issues may arise).
+ * Generally it will be called <code><i>Xxx</i>Rules</code>, for example
+ * {@code CsvRules}. The rule instance is named after your rule, and is based
+ * on the default config ({@code Config.DEFAULT}, or {@code DEFAULT_CONFIG} for
+ * converter rules):
+ *
+ * <blockquote><pre><code>
+ * &#x2f;** Rule that matches a {&#x40;code Project} on a
+ *  * {&#x40;code CsvTableScan} and pushes down projects if possible. *&#x2f;
+ * public static final CsvProjectTableScanRule PROJECT_SCAN =
+ *     CsvProjectTableScanRule.Config.DEFAULT.toRule();
+ * </code></pre></blockquote>
+ *
+ * @param <C> Configuration type
+ */
+public abstract class RelRule<C extends RelRule.Config> extends RelOptRule {
+  public final C config;
+
+  /** Creates a RelRule. */
+  public RelRule(C config) {
+    super(OperandBuilderImpl.operand(config.operandSupplier()),
+        config.relBuilderFactory(), config.description());
+    this.config = config;
+  }
+
+  /** Rule configuration. */
+  public interface Config {
+    /** Empty configuration. */
+    RelRule.Config EMPTY = ImmutableBeans.create(Config.class)
+        .withRelBuilderFactory(RelFactories.LOGICAL_BUILDER)
+        .withOperandSupplier(b -> {
+          throw new IllegalArgumentException("Rules must have at least one "
+              + "operand. Call Config.withOperandSupplier to specify them.");
+        });
+
+    /** Creates a rule that uses this configuration. Sub-class must override. */
+    RelOptRule toRule();
+
+    /** Casts this configuration to another type, usually a sub-class. */
+    default <T> T as(Class<T> class_) {
+      return ImmutableBeans.copy(class_, this);
+    }
+
+    /** The factory that is used to create a
+     * {@link org.apache.calcite.tools.RelBuilder} during rule invocations. */
+    @ImmutableBeans.Property
+    RelBuilderFactory relBuilderFactory();
+
+    /** Sets {@link #relBuilderFactory()}. */
+    Config withRelBuilderFactory(RelBuilderFactory factory);
+
+    /** Description of the rule instance. */
+    @ImmutableBeans.Property
+    String description();
+
+    /** Sets {@link #description()}. */
+    Config withDescription(String description);
+
+    /** Creates the operands for the rule instance. */
+    @ImmutableBeans.Property
+    OperandTransform operandSupplier();
+
+    /** Sets {@link #operandSupplier()}. */
+    Config withOperandSupplier(OperandTransform transform);
+  }
+
+  /** Function that creates an operand.
+   *
+   * @see Config#withOperandSupplier(OperandTransform) */
+  @FunctionalInterface
+  public interface OperandTransform extends Function<OperandBuilder, Done> {
+  }
+
+  /** Callback to create an operand.
+   *
+   * @see OperandTransform */
+  public interface OperandBuilder {
+    /** Starts building an operand by specifying its class.
+     * Call further methods on the returned {@link OperandDetailBuilder} to
+     * complete the operand. */
+    <R extends RelNode> OperandDetailBuilder<R> operand(Class<R> relClass);
+
+    /** Supplies an operand that has been built manually. */
+    Done exactly(RelOptRuleOperand operand);
+  }
+
+  /** Indicates that an operand is complete.
+   *
+   * @see OperandTransform */
+  public interface Done {
+  }
+
+  /** Add details about an operand, such as its inputs.
+   *
+   * @param <R> Type of relational expression */
+  public interface OperandDetailBuilder<R extends RelNode> {
+    /** Sets a trait of this operand. */
+    OperandDetailBuilder<R> trait(@Nonnull RelTrait trait);
+
+    /** Sets the predicate of this operand. */
+    OperandDetailBuilder<R> predicate(Predicate<? super R> predicate);
+
+    /** Indicates that this operand has a single input. */
+    Done oneInput(OperandTransform transform);
+
+    /** Indicates that this operand has several inputs. */
+    Done inputs(OperandTransform... transforms);
+
+    /** Indicates that this operand has several inputs, unordered. */
+    Done unorderedInputs(OperandTransform... transforms);
+
+    /** Indicates that this operand takes any number or type of inputs. */
+    Done anyInputs();
+
+    /** Indicates that this operand takes no inputs. */
+    Done noInputs();
+
+    /** Indicates that this operand converts a relational expression to
+     * another trait. */
+    Done convert(RelTrait in);
+  }
+
+  /** Implementation of {@link OperandBuilder}. */
+  private static class OperandBuilderImpl implements OperandBuilder {
+    final List<RelOptRuleOperand> operands = new ArrayList<>();
+
+    static RelOptRuleOperand operand(OperandTransform transform) {
+      final OperandBuilderImpl b = new OperandBuilderImpl();
+      final Done done = transform.apply(b);
+      Objects.requireNonNull(done);
+      if (b.operands.size() != 1) {
+        throw new IllegalArgumentException("operand supplier must call one of "
+            + "the following methods: operand or exactly");
+      }
+      return b.operands.get(0);
+    }
+
+    public <R extends RelNode> OperandDetailBuilder<R> operand(Class<R> relClass) {
+      return new OperandDetailBuilderImpl<>(this, relClass);
+    }
+
+    public Done exactly(RelOptRuleOperand operand) {
+      operands.add(operand);
+      return DoneImpl.INSTANCE;
+    }
+  }
+
+  /** Implementation of {@link OperandDetailBuilder}.
+   *
+   * @param <R> Type of relational expression */
+  private static class OperandDetailBuilderImpl<R extends RelNode>
+      implements OperandDetailBuilder<R> {
+    private final OperandBuilderImpl parent;
+    private final Class<R> relClass;
+    final OperandBuilderImpl inputBuilder = new OperandBuilderImpl();
+    private RelTrait trait;
+    private Predicate<? super R> predicate = r -> true;
+
+    OperandDetailBuilderImpl(OperandBuilderImpl parent, Class<R> relClass) {
+      this.parent = Objects.requireNonNull(parent);
+      this.relClass = Objects.requireNonNull(relClass);
+    }
+
+    public OperandDetailBuilderImpl<R> trait(@Nonnull RelTrait trait) {
+      this.trait = Objects.requireNonNull(trait);
+      return this;
+    }
+
+    public OperandDetailBuilderImpl<R> predicate(Predicate<? super R> predicate) {
+      this.predicate = predicate;
+      return this;
+    }
+
+    /** Indicates that there are no more inputs. */
+    Done done(RelOptRuleOperandChildPolicy childPolicy) {
+      parent.operands.add(
+          new RelOptRuleOperand(relClass, trait, predicate, childPolicy,
+              ImmutableList.copyOf(inputBuilder.operands)));
+      return DoneImpl.INSTANCE;
+    }
+
+    public Done convert(RelTrait in) {
+      parent.operands.add(
+          new ConverterRelOptRuleOperand(relClass, in, predicate));
+      return DoneImpl.INSTANCE;
+    }
+
+    public Done noInputs() {
+      return done(RelOptRuleOperandChildPolicy.LEAF);
+    }
+
+    public Done anyInputs() {
+      return done(RelOptRuleOperandChildPolicy.ANY);
+    }
+
+    public Done oneInput(OperandTransform transform) {
+      final Done done = transform.apply(inputBuilder);
+      Objects.requireNonNull(done);
+      return done(RelOptRuleOperandChildPolicy.SOME);
+    }
+
+    public Done inputs(OperandTransform... transforms) {
+      for (OperandTransform transform : transforms) {
+        final Done done = transform.apply(inputBuilder);
+        Objects.requireNonNull(done);
+      }
+      return done(RelOptRuleOperandChildPolicy.SOME);
+    }
+
+    public Done unorderedInputs(OperandTransform... transforms) {
+      for (OperandTransform transform : transforms) {
+        final Done done = transform.apply(inputBuilder);
+        Objects.requireNonNull(done);
+      }
+      return done(RelOptRuleOperandChildPolicy.UNORDERED);
+    }
+  }
+
+  /** Singleton instance of {@link Done}. */
+  private enum DoneImpl implements Done {
+    INSTANCE
+  }
+
+  /** Callback interface that helps you avoid creating sub-classes of
+   * {@link RelRule} that differ only in implementations of
+   * {@link #onMatch(RelOptRuleCall)} method.
+   *
+   * @param <R> Rule type */
+  public interface MatchHandler<R extends RelOptRule>
+      extends BiConsumer<R, RelOptRuleCall> {
+  }
+}
diff --git a/core/src/main/java/org/apache/calcite/plan/hep/HepPlanner.java b/core/src/main/java/org/apache/calcite/plan/hep/HepPlanner.java
index 97cf205..77a1155 100644
--- a/core/src/main/java/org/apache/calcite/plan/hep/HepPlanner.java
+++ b/core/src/main/java/org/apache/calcite/plan/hep/HepPlanner.java
@@ -300,7 +300,8 @@ public class HepPlanner extends AbstractRelOptPlanner {
         if (!instruction.guaranteed) {
           // Add a TraitMatchingRule to work bottom-up
           instruction.ruleSet.add(
-              new TraitMatchingRule(converter, RelFactories.LOGICAL_BUILDER));
+              TraitMatchingRule.config(converter, RelFactories.LOGICAL_BUILDER)
+                  .toRule());
         }
       }
     }
diff --git a/core/src/main/java/org/apache/calcite/plan/hep/HepProgramBuilder.java b/core/src/main/java/org/apache/calcite/plan/hep/HepProgramBuilder.java
index f3ed082..f1bd874 100644
--- a/core/src/main/java/org/apache/calcite/plan/hep/HepProgramBuilder.java
+++ b/core/src/main/java/org/apache/calcite/plan/hep/HepProgramBuilder.java
@@ -23,6 +23,7 @@ import org.apache.calcite.plan.RelOptRule;
 import java.util.ArrayList;
 import java.util.Collection;
 import java.util.List;
+import java.util.Objects;
 
 /**
  * HepProgramBuilder creates instances of {@link HepProgram}.
@@ -111,7 +112,7 @@ public class HepProgramBuilder {
   public HepProgramBuilder addRuleInstance(RelOptRule rule) {
     HepInstruction.RuleInstance instruction =
         new HepInstruction.RuleInstance();
-    instruction.rule = rule;
+    instruction.rule = Objects.requireNonNull(rule);
     instructions.add(instruction);
     return this;
   }
diff --git a/core/src/main/java/org/apache/calcite/plan/hep/HepRuleCall.java b/core/src/main/java/org/apache/calcite/plan/hep/HepRuleCall.java
index 629c791..e02d595 100644
--- a/core/src/main/java/org/apache/calcite/plan/hep/HepRuleCall.java
+++ b/core/src/main/java/org/apache/calcite/plan/hep/HepRuleCall.java
@@ -52,8 +52,7 @@ public class HepRuleCall extends RelOptRuleCall {
 
   //~ Methods ----------------------------------------------------------------
 
-  // implement RelOptRuleCall
-  public void transformTo(RelNode rel, Map<RelNode, RelNode> equiv,
+  @Override public void transformTo(RelNode rel, Map<RelNode, RelNode> equiv,
       RelHintsPropagator handler) {
     final RelNode rel0 = rels[0];
     RelOptUtil.verifyTypeEquivalence(rel0, rel, rel0);
diff --git a/core/src/main/java/org/apache/calcite/plan/volcano/AbstractConverter.java b/core/src/main/java/org/apache/calcite/plan/volcano/AbstractConverter.java
index f1ef0b9..40a50ac 100644
--- a/core/src/main/java/org/apache/calcite/plan/volcano/AbstractConverter.java
+++ b/core/src/main/java/org/apache/calcite/plan/volcano/AbstractConverter.java
@@ -19,15 +19,14 @@ package org.apache.calcite.plan.volcano;
 import org.apache.calcite.plan.RelOptCluster;
 import org.apache.calcite.plan.RelOptCost;
 import org.apache.calcite.plan.RelOptPlanner;
-import org.apache.calcite.plan.RelOptRule;
 import org.apache.calcite.plan.RelOptRuleCall;
+import org.apache.calcite.plan.RelRule;
 import org.apache.calcite.plan.RelTrait;
 import org.apache.calcite.plan.RelTraitDef;
 import org.apache.calcite.plan.RelTraitSet;
 import org.apache.calcite.rel.RelNode;
 import org.apache.calcite.rel.RelWriter;
 import org.apache.calcite.rel.convert.ConverterImpl;
-import org.apache.calcite.rel.core.RelFactories;
 import org.apache.calcite.rel.metadata.RelMetadataQuery;
 import org.apache.calcite.tools.RelBuilderFactory;
 
@@ -88,36 +87,39 @@ public class AbstractConverter extends ConverterImpl {
   //~ Inner Classes ----------------------------------------------------------
 
   /**
-   * Rule which converts an {@link AbstractConverter} into a chain of
+   * Rule that converts an {@link AbstractConverter} into a chain of
    * converters from the source relation to the target traits.
    *
    * <p>The chain produced is minimal: we have previously built the transitive
-   * closure of the graph of conversions, so we choose the shortest chain.</p>
+   * closure of the graph of conversions, so we choose the shortest chain.
    *
    * <p>Unlike the {@link AbstractConverter} they are replacing, these
    * converters are guaranteed to be able to convert any relation of their
    * calling convention. Furthermore, because they introduce subsets of other
    * calling conventions along the way, these subsets may spawn more efficient
-   * conversions which are not generally applicable.</p>
+   * conversions which are not generally applicable.
    *
    * <p>AbstractConverters can be messy, so they restrain themselves: they
    * don't fire if the target subset already has an implementation (with less
-   * than infinite cost).</p>
+   * than infinite cost).
    */
-  public static class ExpandConversionRule extends RelOptRule {
+  public static class ExpandConversionRule
+      extends RelRule<ExpandConversionRule.Config> {
     public static final ExpandConversionRule INSTANCE =
-        new ExpandConversionRule(RelFactories.LOGICAL_BUILDER);
+        Config.DEFAULT.toRule();
 
-    /**
-     * Creates an ExpandConversionRule.
-     *
-     * @param relBuilderFactory Builder for relational expressions
-     */
+    /** Creates an ExpandConversionRule. */
+    protected ExpandConversionRule(Config config) {
+      super(config);
+    }
+
+    @Deprecated // to be removed before 2.0
     public ExpandConversionRule(RelBuilderFactory relBuilderFactory) {
-      super(operand(AbstractConverter.class, any()), relBuilderFactory, null);
+      this(Config.DEFAULT.withRelBuilderFactory(relBuilderFactory)
+          .as(Config.class));
     }
 
-    public void onMatch(RelOptRuleCall call) {
+    @Override public void onMatch(RelOptRuleCall call) {
       final VolcanoPlanner planner = (VolcanoPlanner) call.getPlanner();
       AbstractConverter converter = call.rel(0);
       final RelNode child = converter.getInput();
@@ -129,5 +131,17 @@ public class AbstractConverter extends ConverterImpl {
         call.transformTo(converted);
       }
     }
+
+    /** Rule configuration. */
+    public interface Config extends RelRule.Config {
+      Config DEFAULT = EMPTY
+          .withOperandSupplier(b ->
+              b.operand(AbstractConverter.class).anyInputs())
+          .as(Config.class);
+
+      @Override default ExpandConversionRule toRule() {
+        return new ExpandConversionRule(this);
+      }
+    }
   }
 }
diff --git a/core/src/main/java/org/apache/calcite/plan/volcano/VolcanoRuleCall.java b/core/src/main/java/org/apache/calcite/plan/volcano/VolcanoRuleCall.java
index d4f3f03..b5e61e3 100644
--- a/core/src/main/java/org/apache/calcite/plan/volcano/VolcanoRuleCall.java
+++ b/core/src/main/java/org/apache/calcite/plan/volcano/VolcanoRuleCall.java
@@ -91,8 +91,7 @@ public class VolcanoRuleCall extends RelOptRuleCall {
 
   //~ Methods ----------------------------------------------------------------
 
-  // implement RelOptRuleCall
-  public void transformTo(RelNode rel, Map<RelNode, RelNode> equiv,
+  @Override public void transformTo(RelNode rel, Map<RelNode, RelNode> equiv,
       RelHintsPropagator handler) {
     if (rel instanceof PhysicalNode
         && rule instanceof TransformationRule) {
diff --git a/core/src/main/java/org/apache/calcite/rel/convert/ConverterRule.java b/core/src/main/java/org/apache/calcite/rel/convert/ConverterRule.java
index 2da27fd..0d78259 100644
--- a/core/src/main/java/org/apache/calcite/rel/convert/ConverterRule.java
+++ b/core/src/main/java/org/apache/calcite/rel/convert/ConverterRule.java
@@ -19,28 +19,47 @@ package org.apache.calcite.rel.convert;
 import org.apache.calcite.plan.Convention;
 import org.apache.calcite.plan.RelOptRule;
 import org.apache.calcite.plan.RelOptRuleCall;
+import org.apache.calcite.plan.RelRule;
 import org.apache.calcite.plan.RelTrait;
 import org.apache.calcite.plan.RelTraitDef;
 import org.apache.calcite.rel.RelNode;
-import org.apache.calcite.rel.core.RelFactories;
 import org.apache.calcite.tools.RelBuilderFactory;
+import org.apache.calcite.util.ImmutableBeans;
 
 import java.util.Locale;
 import java.util.Objects;
+import java.util.function.Function;
 import java.util.function.Predicate;
 
 /**
  * Abstract base class for a rule which converts from one calling convention to
  * another without changing semantics.
  */
-public abstract class ConverterRule extends RelOptRule {
+public abstract class ConverterRule
+    extends RelRule<ConverterRule.Config> {
   //~ Instance fields --------------------------------------------------------
 
   private final RelTrait inTrait;
   private final RelTrait outTrait;
+  protected final Convention out;
 
   //~ Constructors -----------------------------------------------------------
 
+  /** Creates a <code>ConverterRule</code>. */
+  protected ConverterRule(Config config) {
+    super(config);
+    this.inTrait = Objects.requireNonNull(config.inTrait());
+    this.outTrait = Objects.requireNonNull(config.outTrait());
+
+    // Source and target traits must have same type
+    assert inTrait.getTraitDef() == outTrait.getTraitDef();
+
+    // Most sub-classes are concerned with converting one convention to
+    // another, and for them, the "out" field is a convenient short-cut.
+    this.out = outTrait instanceof Convention ? (Convention) outTrait
+        : null;
+  }
+
   /**
    * Creates a <code>ConverterRule</code>.
    *
@@ -48,11 +67,14 @@ public abstract class ConverterRule extends RelOptRule {
    * @param in          Trait of relational expression to consider converting
    * @param out         Trait which is converted to
    * @param descriptionPrefix Description prefix of rule
+   *
+   * @deprecated Use {@link #ConverterRule(Config)}
    */
+  @Deprecated // to be removed before 2.0
   public ConverterRule(Class<? extends RelNode> clazz, RelTrait in,
       RelTrait out, String descriptionPrefix) {
-    this(clazz, (Predicate<RelNode>) r -> true, in, out,
-        RelFactories.LOGICAL_BUILDER, descriptionPrefix);
+    this(Config.INSTANCE
+        .withConversion(clazz, in, out, descriptionPrefix));
   }
 
   @SuppressWarnings("Guava")
@@ -60,7 +82,9 @@ public abstract class ConverterRule extends RelOptRule {
   public <R extends RelNode> ConverterRule(Class<R> clazz,
       com.google.common.base.Predicate<? super R> predicate,
       RelTrait in, RelTrait out, String descriptionPrefix) {
-    this(clazz, predicate, in, out, RelFactories.LOGICAL_BUILDER, descriptionPrefix);
+    this(Config.INSTANCE
+        .withConversion(clazz, (Predicate<? super R>) predicate::apply,
+            in, out, descriptionPrefix));
   }
 
   /**
@@ -72,18 +96,17 @@ public abstract class ConverterRule extends RelOptRule {
    * @param out         Trait which is converted to
    * @param relBuilderFactory Builder for relational expressions
    * @param descriptionPrefix Description prefix of rule
+   *
+   * @deprecated Use {@link #ConverterRule(Config)}
    */
+  @Deprecated // to be removed before 2.0
   public <R extends RelNode> ConverterRule(Class<R> clazz,
       Predicate<? super R> predicate, RelTrait in, RelTrait out,
       RelBuilderFactory relBuilderFactory, String descriptionPrefix) {
-    super(convertOperand(clazz, predicate, in),
-        relBuilderFactory,
-        createDescription(descriptionPrefix, in, out));
-    this.inTrait = Objects.requireNonNull(in);
-    this.outTrait = Objects.requireNonNull(out);
-
-    // Source and target traits must have same type
-    assert in.getTraitDef() == out.getTraitDef();
+    this(Config.EMPTY
+        .withRelBuilderFactory(relBuilderFactory)
+        .as(Config.class)
+        .withConversion(clazz, predicate, in, out, descriptionPrefix));
   }
 
   @SuppressWarnings("Guava")
@@ -97,7 +120,7 @@ public abstract class ConverterRule extends RelOptRule {
 
   //~ Methods ----------------------------------------------------------------
 
-  public Convention getOutConvention() {
+  @Override public Convention getOutConvention() {
     return (Convention) outTrait;
   }
 
@@ -138,7 +161,7 @@ public abstract class ConverterRule extends RelOptRule {
     return false;
   }
 
-  public void onMatch(RelOptRuleCall call) {
+  @Override public void onMatch(RelOptRuleCall call) {
     RelNode rel = call.rel(0);
     if (rel.getTraitSet().contains(inTrait)) {
       final RelNode converted = convert(rel);
@@ -150,4 +173,51 @@ public abstract class ConverterRule extends RelOptRule {
 
   //~ Inner Classes ----------------------------------------------------------
 
+  /** Rule configuration. */
+  public interface Config extends RelRule.Config {
+    Config INSTANCE = EMPTY.as(Config.class);
+
+    @ImmutableBeans.Property
+    RelTrait inTrait();
+
+    /** Sets {@link #inTrait}. */
+    Config withInTrait(RelTrait trait);
+
+    @ImmutableBeans.Property
+    RelTrait outTrait();
+
+    /** Sets {@link #outTrait}. */
+    Config withOutTrait(RelTrait trait);
+
+    @ImmutableBeans.Property
+    Function<Config, ConverterRule> ruleFactory();
+
+    /** Sets {@link #outTrait}. */
+    Config withRuleFactory(Function<Config, ConverterRule> factory);
+
+    default <R extends RelNode> Config withConversion(Class<R> clazz,
+        Predicate<? super R> predicate, RelTrait in, RelTrait out,
+        String descriptionPrefix) {
+      return withInTrait(in)
+          .withOutTrait(out)
+          .withOperandSupplier(b ->
+              b.operand(clazz).predicate(predicate).convert(in))
+          .withDescription(createDescription(descriptionPrefix, in, out))
+          .as(Config.class);
+    }
+
+    default Config withConversion(Class<? extends RelNode> clazz, RelTrait in,
+        RelTrait out, String descriptionPrefix) {
+      return withConversion(clazz, r -> true, in, out, descriptionPrefix);
+    }
+
+    @Override default RelOptRule toRule() {
+      return toRule(ConverterRule.class);
+    }
+
+    default <R extends ConverterRule> R toRule(Class<R> ruleClass) {
+      return ruleClass.cast(ruleFactory().apply(this));
+    }
+  }
+
 }
diff --git a/core/src/main/java/org/apache/calcite/rel/convert/TraitMatchingRule.java b/core/src/main/java/org/apache/calcite/rel/convert/TraitMatchingRule.java
index c857693..affb186 100644
--- a/core/src/main/java/org/apache/calcite/rel/convert/TraitMatchingRule.java
+++ b/core/src/main/java/org/apache/calcite/rel/convert/TraitMatchingRule.java
@@ -17,12 +17,14 @@
 package org.apache.calcite.rel.convert;
 
 import org.apache.calcite.plan.Convention;
-import org.apache.calcite.plan.RelOptRule;
 import org.apache.calcite.plan.RelOptRuleCall;
+import org.apache.calcite.plan.RelOptRuleOperand;
 import org.apache.calcite.plan.RelOptRuleOperandChildPolicy;
+import org.apache.calcite.plan.RelRule;
 import org.apache.calcite.rel.RelNode;
 import org.apache.calcite.rel.core.RelFactories;
 import org.apache.calcite.tools.RelBuilderFactory;
+import org.apache.calcite.util.ImmutableBeans;
 
 /**
  * TraitMatchingRule adapts a converter rule, restricting it to fire only when
@@ -30,48 +32,71 @@ import org.apache.calcite.tools.RelBuilderFactory;
  * {@link org.apache.calcite.plan.hep.HepPlanner} in cases where alternate
  * implementations are available and it is desirable to minimize converters.
  */
-public class TraitMatchingRule extends RelOptRule {
-  //~ Instance fields --------------------------------------------------------
-
-  private final ConverterRule converter;
+public class TraitMatchingRule extends RelRule<TraitMatchingRule.Config> {
+  /**
+   * Creates a configuration for a TraitMatchingRule.
+   *
+   * @param converterRule     Rule to be restricted; rule must take a single
+   *                          operand expecting a single input
+   * @param relBuilderFactory Builder for relational expressions
+   */
+  public static Config config(ConverterRule converterRule,
+      RelBuilderFactory relBuilderFactory) {
+    final RelOptRuleOperand operand = converterRule.getOperand();
+    assert operand.childPolicy == RelOptRuleOperandChildPolicy.ANY;
+    return Config.EMPTY.withRelBuilderFactory(relBuilderFactory)
+        .withDescription("TraitMatchingRule: " + converterRule)
+        .withOperandSupplier(b0 ->
+            b0.operand(operand.getMatchedClass()).oneInput(b1 ->
+                b1.operand(RelNode.class).anyInputs()))
+        .as(Config.class)
+        .withConverterRule(converterRule);
+  }
 
   //~ Constructors -----------------------------------------------------------
 
+  /** Creates a TraitMatchingRule. */
+  protected TraitMatchingRule(Config config) {
+    super(config);
+  }
+
   @Deprecated // to be removed before 2.0
   public TraitMatchingRule(ConverterRule converterRule) {
-    this(converterRule, RelFactories.LOGICAL_BUILDER);
+    this(config(converterRule, RelFactories.LOGICAL_BUILDER));
   }
 
-  /**
-   * Creates a TraitMatchingRule.
-   *
-   * @param converterRule     Rule to be restricted; rule must take a single
-   *                          operand expecting a single input
-   * @param relBuilderFactory Builder for relational expressions
-   */
+  @Deprecated // to be removed before 2.0
   public TraitMatchingRule(ConverterRule converterRule,
       RelBuilderFactory relBuilderFactory) {
-    super(
-        operand(
-            converterRule.getOperand().getMatchedClass(),
-            operand(RelNode.class, any())),
-        relBuilderFactory,
-        "TraitMatchingRule: " + converterRule);
-    assert converterRule.getOperand().childPolicy
-        == RelOptRuleOperandChildPolicy.ANY;
-    this.converter = converterRule;
+    this(config(converterRule, relBuilderFactory));
   }
 
   //~ Methods ----------------------------------------------------------------
 
   @Override public Convention getOutConvention() {
-    return converter.getOutConvention();
+    return config.converterRule().getOutConvention();
   }
 
-  public void onMatch(RelOptRuleCall call) {
+  @Override public void onMatch(RelOptRuleCall call) {
     RelNode input = call.rel(1);
-    if (input.getTraitSet().contains(converter.getOutTrait())) {
-      converter.onMatch(call);
+    final ConverterRule converterRule = config.converterRule();
+    if (input.getTraitSet().contains(converterRule.getOutTrait())) {
+      converterRule.onMatch(call);
     }
   }
+
+  /** Rule configuration. */
+  public interface Config extends RelRule.Config {
+    @Override default TraitMatchingRule toRule() {
+      return new TraitMatchingRule(this);
+    }
+
+    /** Returns the rule to be restricted; rule must take a single
+     * operand expecting a single input. */
+    @ImmutableBeans.Property
+    ConverterRule converterRule();
+
+    /** Sets {@link #converterRule()}. */
+    Config withConverterRule(ConverterRule converterRule);
+  }
 }
diff --git a/core/src/main/java/org/apache/calcite/rel/core/Calc.java b/core/src/main/java/org/apache/calcite/rel/core/Calc.java
index 8b2b738..4f1f208 100644
--- a/core/src/main/java/org/apache/calcite/rel/core/Calc.java
+++ b/core/src/main/java/org/apache/calcite/rel/core/Calc.java
@@ -33,6 +33,7 @@ import org.apache.calcite.rel.type.RelDataType;
 import org.apache.calcite.rex.RexBuilder;
 import org.apache.calcite.rex.RexLocalRef;
 import org.apache.calcite.rex.RexNode;
+import org.apache.calcite.rex.RexOver;
 import org.apache.calcite.rex.RexProgram;
 import org.apache.calcite.rex.RexProgramBuilder;
 import org.apache.calcite.rex.RexShuttle;
@@ -130,6 +131,11 @@ public abstract class Calc extends SingleRel implements Hintable {
     return copy(traitSet, child, program);
   }
 
+  /** Returns whether this Calc contains any windowed-aggregate functions. */
+  public final boolean containsOver() {
+    return RexOver.containsOver(program);
+  }
+
   public boolean isValid(Litmus litmus, Context context) {
     if (!RelOptUtil.equal(
         "program's input type",
diff --git a/core/src/main/java/org/apache/calcite/rel/core/Filter.java b/core/src/main/java/org/apache/calcite/rel/core/Filter.java
index 7394ddf..91c435e 100644
--- a/core/src/main/java/org/apache/calcite/rel/core/Filter.java
+++ b/core/src/main/java/org/apache/calcite/rel/core/Filter.java
@@ -29,6 +29,7 @@ import org.apache.calcite.rel.metadata.RelMdUtil;
 import org.apache.calcite.rel.metadata.RelMetadataQuery;
 import org.apache.calcite.rex.RexChecker;
 import org.apache.calcite.rex.RexNode;
+import org.apache.calcite.rex.RexOver;
 import org.apache.calcite.rex.RexProgram;
 import org.apache.calcite.rex.RexShuttle;
 import org.apache.calcite.rex.RexUtil;
@@ -114,6 +115,11 @@ public abstract class Filter extends SingleRel {
     return condition;
   }
 
+  /** Returns whether this Filter contains any windowed-aggregate functions. */
+  public final boolean containsOver() {
+    return RexOver.containsOver(condition);
+  }
+
   @Override public boolean isValid(Litmus litmus, Context context) {
     if (RexUtil.isNullabilityCast(getCluster().getTypeFactory(), condition)) {
       return litmus.fail("Cast for just nullability not allowed");
diff --git a/core/src/main/java/org/apache/calcite/rel/core/Project.java b/core/src/main/java/org/apache/calcite/rel/core/Project.java
index c24b7af..92c7d4f 100644
--- a/core/src/main/java/org/apache/calcite/rel/core/Project.java
+++ b/core/src/main/java/org/apache/calcite/rel/core/Project.java
@@ -33,6 +33,7 @@ import org.apache.calcite.rel.type.RelDataTypeField;
 import org.apache.calcite.rex.RexChecker;
 import org.apache.calcite.rex.RexInputRef;
 import org.apache.calcite.rex.RexNode;
+import org.apache.calcite.rex.RexOver;
 import org.apache.calcite.rex.RexShuttle;
 import org.apache.calcite.rex.RexUtil;
 import org.apache.calcite.sql.SqlExplainLevel;
@@ -199,6 +200,11 @@ public abstract class Project extends SingleRel implements Hintable {
     return 1;
   }
 
+  /** Returns whether this Project contains any windowed-aggregate functions. */
+  public final boolean containsOver() {
+    return RexOver.containsOver(getProjects(), null);
+  }
+
   public boolean isValid(Litmus litmus, Context context) {
     if (!super.isValid(litmus, context)) {
       return litmus.fail(null);
diff --git a/core/src/main/java/org/apache/calcite/rel/core/RelFactories.java b/core/src/main/java/org/apache/calcite/rel/core/RelFactories.java
index 4d838e0..7b49647 100644
--- a/core/src/main/java/org/apache/calcite/rel/core/RelFactories.java
+++ b/core/src/main/java/org/apache/calcite/rel/core/RelFactories.java
@@ -205,7 +205,7 @@ public class RelFactories {
    * of the appropriate type for a rule's calling convention.
    */
   public interface ExchangeFactory {
-    /** Creates a Exchange. */
+    /** Creates an Exchange. */
     RelNode createExchange(RelNode input, RelDistribution distribution);
   }
 
diff --git a/core/src/main/java/org/apache/calcite/rel/rel2sql/SqlImplementor.java b/core/src/main/java/org/apache/calcite/rel/rel2sql/SqlImplementor.java
index c02e8b0..8fe1fad 100644
--- a/core/src/main/java/org/apache/calcite/rel/rel2sql/SqlImplementor.java
+++ b/core/src/main/java/org/apache/calcite/rel/rel2sql/SqlImplementor.java
@@ -1537,7 +1537,7 @@ public abstract class SqlImplementor {
       }
 
       if (rel instanceof Project
-          && RexOver.containsOver(((Project) rel).getProjects(), null)
+          && ((Project) rel).containsOver()
           && maxClause == Clause.SELECT) {
         // Cannot merge a Project that contains windowed functions onto an
         // underlying Project
diff --git a/core/src/main/java/org/apache/calcite/rel/rules/AbstractJoinExtractFilterRule.java b/core/src/main/java/org/apache/calcite/rel/rules/AbstractJoinExtractFilterRule.java
index 4582f63..7f40d28 100644
--- a/core/src/main/java/org/apache/calcite/rel/rules/AbstractJoinExtractFilterRule.java
+++ b/core/src/main/java/org/apache/calcite/rel/rules/AbstractJoinExtractFilterRule.java
@@ -16,9 +16,9 @@
  */
 package org.apache.calcite.rel.rules;
 
-import org.apache.calcite.plan.RelOptRule;
 import org.apache.calcite.plan.RelOptRuleCall;
 import org.apache.calcite.plan.RelOptRuleOperand;
+import org.apache.calcite.plan.RelRule;
 import org.apache.calcite.rel.RelNode;
 import org.apache.calcite.rel.core.Join;
 import org.apache.calcite.rel.core.JoinRelType;
@@ -38,15 +38,25 @@ import org.apache.calcite.tools.RelBuilderFactory;
  * <p>The constructor is parameterized to allow any sub-class of
  * {@link org.apache.calcite.rel.core.Join}.</p>
  */
-public abstract class AbstractJoinExtractFilterRule extends RelOptRule
+public abstract class AbstractJoinExtractFilterRule
+    extends RelRule<AbstractJoinExtractFilterRule.Config>
     implements TransformationRule {
   /** Creates an AbstractJoinExtractFilterRule. */
+  protected AbstractJoinExtractFilterRule(Config config) {
+    super(config);
+  }
+
+  @Deprecated // to be removed before 2.0
   protected AbstractJoinExtractFilterRule(RelOptRuleOperand operand,
       RelBuilderFactory relBuilderFactory, String description) {
-    super(operand, relBuilderFactory, description);
+    this(Config.EMPTY
+        .withOperandSupplier(b -> b.exactly(operand))
+        .withRelBuilderFactory(relBuilderFactory)
+        .withDescription(description)
+        .as(Config.class));
   }
 
-  public void onMatch(RelOptRuleCall call) {
+  @Override public void onMatch(RelOptRuleCall call) {
     final Join join = call.rel(0);
 
     if (join.getJoinType() != JoinRelType.INNER) {
@@ -81,4 +91,8 @@ public abstract class AbstractJoinExtractFilterRule extends RelOptRule
 
     call.transformTo(builder.build());
   }
+
+  /** Rule configuration. */
+  public interface Config extends RelRule.Config {
+  }
 }
diff --git a/core/src/main/java/org/apache/calcite/rel/rules/AbstractMaterializedViewRule.java b/core/src/main/java/org/apache/calcite/rel/rules/AbstractMaterializedViewRule.java
deleted file mode 100644
index c70ea16..0000000
--- a/core/src/main/java/org/apache/calcite/rel/rules/AbstractMaterializedViewRule.java
+++ /dev/null
@@ -1,48 +0,0 @@
-/*
- * 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.rel.rules;
-
-import org.apache.calcite.plan.RelOptRule;
-import org.apache.calcite.rel.rules.materialize.MaterializedViewRules;
-
-/** For backwards compatibility.
- *
- * @deprecated Use equivalent fields in {@link MaterializedViewRules}.
- */
-@Deprecated // to be removed before 1.25
-public abstract class AbstractMaterializedViewRule {
-  private AbstractMaterializedViewRule() {
-  }
-
-  public static final RelOptRule INSTANCE_PROJECT_AGGREGATE =
-      MaterializedViewRules.PROJECT_AGGREGATE;
-
-  public static final RelOptRule INSTANCE_AGGREGATE =
-      MaterializedViewRules.AGGREGATE;
-
-  public static final RelOptRule INSTANCE_FILTER =
-      MaterializedViewRules.FILTER;
-
-  public static final RelOptRule INSTANCE_JOIN =
-      MaterializedViewRules.JOIN;
-
-  public static final RelOptRule INSTANCE_PROJECT_FILTER =
-      MaterializedViewRules.PROJECT_FILTER;
-
-  public static final RelOptRule INSTANCE_PROJECT_JOIN =
-      MaterializedViewRules.PROJECT_JOIN;
-}
diff --git a/core/src/main/java/org/apache/calcite/rel/rules/AggregateCaseToFilterRule.java b/core/src/main/java/org/apache/calcite/rel/rules/AggregateCaseToFilterRule.java
index 60fe9dc..2c52c00 100644
--- a/core/src/main/java/org/apache/calcite/rel/rules/AggregateCaseToFilterRule.java
+++ b/core/src/main/java/org/apache/calcite/rel/rules/AggregateCaseToFilterRule.java
@@ -17,8 +17,8 @@
 package org.apache.calcite.rel.rules;
 
 import org.apache.calcite.plan.RelOptCluster;
-import org.apache.calcite.plan.RelOptRule;
 import org.apache.calcite.plan.RelOptRuleCall;
+import org.apache.calcite.plan.RelRule;
 import org.apache.calcite.rel.RelCollations;
 import org.apache.calcite.rel.core.Aggregate;
 import org.apache.calcite.rel.core.AggregateCall;
@@ -60,19 +60,24 @@ import javax.annotation.Nullable;
  *   <code>SELECT SUM(salary) FILTER (WHERE gender = 'F')<br>
  *   FROM Emp</code>
  * </blockquote>
+ *
+ * @see CoreRules#AGGREGATE_CASE_TO_FILTER
  */
-public class AggregateCaseToFilterRule extends RelOptRule
+public class AggregateCaseToFilterRule
+    extends RelRule<AggregateCaseToFilterRule.Config>
     implements TransformationRule {
-  /** @deprecated Use {@link CoreRules#AGGREGATE_CASE_TO_FILTER}. */
-  @Deprecated // to be removed before 1.25
-  public static final AggregateCaseToFilterRule INSTANCE =
-      CoreRules.AGGREGATE_CASE_TO_FILTER;
 
   /** Creates an AggregateCaseToFilterRule. */
+  protected AggregateCaseToFilterRule(Config config) {
+    super(config);
+  }
+
+  @Deprecated // to be removed before 2.0
   protected AggregateCaseToFilterRule(RelBuilderFactory relBuilderFactory,
       String description) {
-    super(operand(Aggregate.class, operand(Project.class, any())),
-        relBuilderFactory, description);
+    this(Config.DEFAULT.withRelBuilderFactory(relBuilderFactory)
+        .withDescription(description)
+        .as(Config.class));
   }
 
   @Override public boolean matches(final RelOptRuleCall call) {
@@ -266,4 +271,17 @@ public class AggregateCaseToFilterRule extends RelOptRule
     return rexNode instanceof RexLiteral
         && SqlTypeName.INT_TYPES.contains(rexNode.getType().getSqlTypeName());
   }
+
+  /** Rule configuration. */
+  public interface Config extends RelRule.Config {
+    Config DEFAULT = EMPTY
+        .withOperandSupplier(b0 ->
+            b0.operand(Aggregate.class).oneInput(b1 ->
+                b1.operand(Project.class).anyInputs()))
+        .as(Config.class);
+
+    @Override default AggregateCaseToFilterRule toRule() {
+      return new AggregateCaseToFilterRule(this);
+    }
+  }
 }
diff --git a/core/src/main/java/org/apache/calcite/rel/rules/AggregateExpandDistinctAggregatesRule.java b/core/src/main/java/org/apache/calcite/rel/rules/AggregateExpandDistinctAggregatesRule.java
index 06ad022..b152c83 100644
--- a/core/src/main/java/org/apache/calcite/rel/rules/AggregateExpandDistinctAggregatesRule.java
+++ b/core/src/main/java/org/apache/calcite/rel/rules/AggregateExpandDistinctAggregatesRule.java
@@ -17,8 +17,8 @@
 package org.apache.calcite.rel.rules;
 
 import org.apache.calcite.plan.Contexts;
-import org.apache.calcite.plan.RelOptRule;
 import org.apache.calcite.plan.RelOptRuleCall;
+import org.apache.calcite.plan.RelRule;
 import org.apache.calcite.rel.RelCollations;
 import org.apache.calcite.rel.core.Aggregate;
 import org.apache.calcite.rel.core.Aggregate.Group;
@@ -36,6 +36,7 @@ import org.apache.calcite.sql.fun.SqlStdOperatorTable;
 import org.apache.calcite.sql.fun.SqlSumEmptyIsZeroAggFunction;
 import org.apache.calcite.tools.RelBuilder;
 import org.apache.calcite.tools.RelBuilderFactory;
+import org.apache.calcite.util.ImmutableBeans;
 import org.apache.calcite.util.ImmutableBitSet;
 import org.apache.calcite.util.ImmutableIntList;
 import org.apache.calcite.util.Optionality;
@@ -76,32 +77,29 @@ import java.util.stream.Stream;
  * (e.g. {@code COUNT(DISTINCT x), COUNT(DISTINCT y)})
  * the rule creates separate {@code Aggregate}s and combines using a
  * {@link org.apache.calcite.rel.core.Join}.
+ *
+ * @see CoreRules#AGGREGATE_EXPAND_DISTINCT_AGGREGATES
+ * @see CoreRules#AGGREGATE_EXPAND_DISTINCT_AGGREGATES_TO_JOIN
  */
-public final class AggregateExpandDistinctAggregatesRule extends RelOptRule
+public final class AggregateExpandDistinctAggregatesRule
+    extends RelRule<AggregateExpandDistinctAggregatesRule.Config>
     implements TransformationRule {
-  //~ Static fields/initializers ---------------------------------------------
-
-  /** @deprecated Use {@link CoreRules#AGGREGATE_EXPAND_DISTINCT_AGGREGATES}. */
-  @Deprecated // to be removed before 1.25
-  public static final AggregateExpandDistinctAggregatesRule INSTANCE =
-      CoreRules.AGGREGATE_EXPAND_DISTINCT_AGGREGATES;
-
-  /** @deprecated Use
-   * {@link CoreRules#AGGREGATE_EXPAND_DISTINCT_AGGREGATES_TO_JOIN}. */
-  @Deprecated // to be removed before 1.25
-  public static final AggregateExpandDistinctAggregatesRule JOIN =
-      CoreRules.AGGREGATE_EXPAND_DISTINCT_AGGREGATES_TO_JOIN;
-
-  public final boolean useGroupingSets;
 
-  //~ Constructors -----------------------------------------------------------
+  /** Creates an AggregateExpandDistinctAggregatesRule. */
+  protected AggregateExpandDistinctAggregatesRule(Config config) {
+    super(config);
+  }
 
+  @Deprecated // to be removed before 2.0
   public AggregateExpandDistinctAggregatesRule(
       Class<? extends Aggregate> clazz,
       boolean useGroupingSets,
       RelBuilderFactory relBuilderFactory) {
-    super(operand(clazz, any()), relBuilderFactory, null);
-    this.useGroupingSets = useGroupingSets;
+    this(Config.DEFAULT.withRelBuilderFactory(relBuilderFactory)
+        .withOperandSupplier(b ->
+            b.operand(clazz).anyInputs())
+        .as(Config.class)
+        .withUsingGroupingSets(useGroupingSets));
   }
 
   @Deprecated // to be removed before 2.0
@@ -121,7 +119,7 @@ public final class AggregateExpandDistinctAggregatesRule extends RelOptRule
 
   //~ Methods ----------------------------------------------------------------
 
-  public void onMatch(RelOptRuleCall call) {
+  @Override public void onMatch(RelOptRuleCall call) {
     final Aggregate aggregate = call.rel(0);
     if (!aggregate.containsDistinctCall()) {
       return;
@@ -193,7 +191,7 @@ public final class AggregateExpandDistinctAggregatesRule extends RelOptRule
       return;
     }
 
-    if (useGroupingSets) {
+    if (((Config) config).isUsingGroupingSets()) {
       rewriteUsingGroupingSets(call, aggregate);
       return;
     }
@@ -899,4 +897,26 @@ public final class AggregateExpandDistinctAggregatesRule extends RelOptRule
             ImmutableBitSet.range(projects.size()), null, ImmutableList.of()));
     return relBuilder;
   }
+
+       /** Rule configuration. */
+  public interface Config extends RelRule.Config {
+    Config DEFAULT = EMPTY
+        .withOperandSupplier(b ->
+            b.operand(LogicalAggregate.class).anyInputs())
+        .as(Config.class);
+
+    Config JOIN = DEFAULT.withUsingGroupingSets(false);
+
+    @Override default AggregateExpandDistinctAggregatesRule toRule() {
+      return new AggregateExpandDistinctAggregatesRule(this);
+    }
+
+    /** Whether to use GROUPING SETS, default true. */
+    @ImmutableBeans.Property
+    @ImmutableBeans.BooleanDefault(true)
+    boolean isUsingGroupingSets();
+
+    /** Sets {@link #isUsingGroupingSets()}. */
+    Config withUsingGroupingSets(boolean usingGroupingSets);
+  }
 }
diff --git a/core/src/main/java/org/apache/calcite/rel/rules/AggregateExtractProjectRule.java b/core/src/main/java/org/apache/calcite/rel/rules/AggregateExtractProjectRule.java
index 1a81683..0452e73 100644
--- a/core/src/main/java/org/apache/calcite/rel/rules/AggregateExtractProjectRule.java
+++ b/core/src/main/java/org/apache/calcite/rel/rules/AggregateExtractProjectRule.java
@@ -16,13 +16,14 @@
  */
 package org.apache.calcite.rel.rules;
 
-import org.apache.calcite.plan.RelOptRule;
 import org.apache.calcite.plan.RelOptRuleCall;
 import org.apache.calcite.plan.RelOptRuleOperand;
+import org.apache.calcite.plan.RelRule;
 import org.apache.calcite.rel.RelNode;
 import org.apache.calcite.rel.core.Aggregate;
 import org.apache.calcite.rel.core.AggregateCall;
 import org.apache.calcite.rel.core.Project;
+import org.apache.calcite.rel.logical.LogicalTableScan;
 import org.apache.calcite.rex.RexNode;
 import org.apache.calcite.tools.RelBuilder;
 import org.apache.calcite.tools.RelBuilderFactory;
@@ -48,32 +49,38 @@ import java.util.List;
  * <p>To prevent cycles, this rule will not extract a {@code Project} if the
  * {@code Aggregate}s input is already a {@code Project}.
  */
-public class AggregateExtractProjectRule extends RelOptRule
+public class AggregateExtractProjectRule
+    extends RelRule<AggregateExtractProjectRule.Config>
     implements TransformationRule {
+  public static final AggregateExtractProjectRule SCAN =
+      Config.DEFAULT.toRule();
 
-  /**
-   * Creates an AggregateExtractProjectRule.
-   *
-   * @param relBuilderFactory Builder for relational expressions
-   */
+  /** Creates an AggregateExtractProjectRule. */
+  protected AggregateExtractProjectRule(Config config) {
+    super(config);
+  }
+
+  @Deprecated // to be removed before 2.0
   public AggregateExtractProjectRule(
       Class<? extends Aggregate> aggregateClass,
       Class<? extends RelNode> inputClass,
       RelBuilderFactory relBuilderFactory) {
-    // Predicate prevents matching against an Aggregate whose input
-    // is already a Project. Prevents this rule firing repeatedly.
-    this(
-        operand(aggregateClass,
-            operandJ(inputClass, null, r -> !(r instanceof Project), any())),
-        relBuilderFactory);
+    this(Config.DEFAULT
+        .withRelBuilderFactory(relBuilderFactory)
+        .as(Config.class)
+        .withOperandFor(aggregateClass, inputClass));
   }
 
+  @Deprecated // to be removed before 2.0
   public AggregateExtractProjectRule(RelOptRuleOperand operand,
       RelBuilderFactory builderFactory) {
-    super(operand, builderFactory, null);
+    this(Config.DEFAULT
+        .withRelBuilderFactory(builderFactory)
+        .withOperandSupplier(b -> b.exactly(operand))
+        .as(Config.class));
   }
 
-  public void onMatch(RelOptRuleCall call) {
+  @Override public void onMatch(RelOptRuleCall call) {
     final Aggregate aggregate = call.rel(0);
     final RelNode input = call.rel(1);
     // Compute which input fields are used.
@@ -130,4 +137,28 @@ public class AggregateExtractProjectRule extends RelOptRule
     relBuilder.aggregate(groupKey, newAggCallList);
     call.transformTo(relBuilder.build());
   }
+
+  /** Rule configuration. */
+  public interface Config extends RelRule.Config {
+    Config DEFAULT = EMPTY
+        .as(Config.class)
+        .withOperandFor(Aggregate.class, LogicalTableScan.class);
+
+    @Override default AggregateExtractProjectRule toRule() {
+      return new AggregateExtractProjectRule(this);
+    }
+
+    /** Defines an operand tree for the given classes. */
+    default Config withOperandFor(Class<? extends Aggregate> aggregateClass,
+        Class<? extends RelNode> inputClass) {
+      return withOperandSupplier(b0 ->
+          b0.operand(aggregateClass).oneInput(b1 ->
+              b1.operand(inputClass)
+                  // Predicate prevents matching against an Aggregate whose
+                  // input is already a Project. Prevents this rule firing
+                  // repeatedly.
+                  .predicate(r -> !(r instanceof Project)).anyInputs()))
+          .as(Config.class);
+    }
+  }
 }
diff --git a/core/src/main/java/org/apache/calcite/rel/rules/AggregateFilterTransposeRule.java b/core/src/main/java/org/apache/calcite/rel/rules/AggregateFilterTransposeRule.java
index ae0b75a..56974af 100644
--- a/core/src/main/java/org/apache/calcite/rel/rules/AggregateFilterTransposeRule.java
+++ b/core/src/main/java/org/apache/calcite/rel/rules/AggregateFilterTransposeRule.java
@@ -16,17 +16,16 @@
  */
 package org.apache.calcite.rel.rules;
 
-import org.apache.calcite.plan.RelOptRule;
 import org.apache.calcite.plan.RelOptRuleCall;
 import org.apache.calcite.plan.RelOptRuleOperand;
 import org.apache.calcite.plan.RelOptUtil;
+import org.apache.calcite.plan.RelRule;
 import org.apache.calcite.plan.SubstitutionVisitor;
 import org.apache.calcite.rel.RelNode;
 import org.apache.calcite.rel.core.Aggregate;
 import org.apache.calcite.rel.core.Aggregate.Group;
 import org.apache.calcite.rel.core.AggregateCall;
 import org.apache.calcite.rel.core.Filter;
-import org.apache.calcite.rel.core.RelFactories;
 import org.apache.calcite.rel.metadata.RelMetadataQuery;
 import org.apache.calcite.rex.RexNode;
 import org.apache.calcite.rex.RexUtil;
@@ -54,28 +53,27 @@ import java.util.List;
  * under an aggregate to an existing aggregate table.
  *
  * @see org.apache.calcite.rel.rules.FilterAggregateTransposeRule
+ * @see CoreRules#AGGREGATE_FILTER_TRANSPOSE
  */
-public class AggregateFilterTransposeRule extends RelOptRule
+public class AggregateFilterTransposeRule
+    extends RelRule<AggregateFilterTransposeRule.Config>
     implements TransformationRule {
-  /** @deprecated Use {@link CoreRules#AGGREGATE_FILTER_TRANSPOSE}. */
-  @Deprecated // to be removed before 1.25
-  public static final AggregateFilterTransposeRule INSTANCE =
-      CoreRules.AGGREGATE_FILTER_TRANSPOSE;
 
-  AggregateFilterTransposeRule() {
-    this(
-        operand(Aggregate.class,
-            operand(Filter.class, any())),
-        RelFactories.LOGICAL_BUILDER);
+  /** Creates an AggregateFilterTransposeRule. */
+  protected AggregateFilterTransposeRule(Config config) {
+    super(config);
   }
 
-  /** Creates an AggregateFilterTransposeRule. */
+  @Deprecated // to be removed before 2.0
   public AggregateFilterTransposeRule(RelOptRuleOperand operand,
       RelBuilderFactory relBuilderFactory) {
-    super(operand, relBuilderFactory, null);
+    this(Config.DEFAULT
+        .withRelBuilderFactory(relBuilderFactory)
+        .withOperandSupplier(b -> b.exactly(operand))
+        .as(Config.class));
   }
 
-  public void onMatch(RelOptRuleCall call) {
+  @Override public void onMatch(RelOptRuleCall call) {
     final Aggregate aggregate = call.rel(0);
     final Filter filter = call.rel(1);
 
@@ -157,4 +155,34 @@ public class AggregateFilterTransposeRule extends RelOptRule
       call.transformTo(topAggregate);
     }
   }
+
+  /** Rule configuration. */
+  public interface Config extends RelRule.Config {
+    Config DEFAULT = EMPTY.as(Config.class)
+        .withOperandFor(Aggregate.class, Filter.class);
+
+    @Override default AggregateFilterTransposeRule toRule() {
+      return new AggregateFilterTransposeRule(this);
+    }
+
+    /** Defines an operand tree for the given 2 classes. */
+    default Config withOperandFor(Class<? extends Aggregate> aggregateClass,
+        Class<? extends Filter> filterClass) {
+      return withOperandSupplier(b0 ->
+          b0.operand(aggregateClass).oneInput(b1 ->
+              b1.operand(filterClass).anyInputs()))
+          .as(Config.class);
+    }
+
+    /** Defines an operand tree for the given 3 classes. */
+    default Config withOperandFor(Class<? extends Aggregate> aggregateClass,
+        Class<? extends Filter> filterClass,
+        Class<? extends RelNode> relClass) {
+      return withOperandSupplier(b0 ->
+          b0.operand(aggregateClass).oneInput(b1 ->
+              b1.operand(filterClass).oneInput(b2 ->
+                  b2.operand(relClass).anyInputs())))
+          .as(Config.class);
+    }
+  }
 }
diff --git a/core/src/main/java/org/apache/calcite/rel/rules/AggregateJoinJoinRemoveRule.java b/core/src/main/java/org/apache/calcite/rel/rules/AggregateJoinJoinRemoveRule.java
index 6a5735d..45034bc 100644
--- a/core/src/main/java/org/apache/calcite/rel/rules/AggregateJoinJoinRemoveRule.java
+++ b/core/src/main/java/org/apache/calcite/rel/rules/AggregateJoinJoinRemoveRule.java
@@ -16,14 +16,16 @@
  */
 package org.apache.calcite.rel.rules;
 
-import org.apache.calcite.plan.RelOptRule;
 import org.apache.calcite.plan.RelOptRuleCall;
 import org.apache.calcite.plan.RelOptUtil;
+import org.apache.calcite.plan.RelRule;
 import org.apache.calcite.rel.RelNode;
 import org.apache.calcite.rel.core.Aggregate;
 import org.apache.calcite.rel.core.AggregateCall;
 import org.apache.calcite.rel.core.Join;
 import org.apache.calcite.rel.core.JoinRelType;
+import org.apache.calcite.rel.logical.LogicalAggregate;
+import org.apache.calcite.rel.logical.LogicalJoin;
 import org.apache.calcite.rex.RexNode;
 import org.apache.calcite.rex.RexUtil;
 import org.apache.calcite.tools.RelBuilder;
@@ -44,43 +46,43 @@ import java.util.Set;
  * on a {@link org.apache.calcite.rel.core.Join} and removes the left input
  * of the join provided that the left input is also a left join if possible.
  *
- * <p>For instance,</p>
+ * <p>For instance,
  *
  * <blockquote>
- * <pre>select distinct s.product_id, pc.product_id from
- * sales as s
+ * <pre>select distinct s.product_id, pc.product_id
+ * from sales as s
  * left join product as p
- * on s.product_id = p.product_id
+ *   on s.product_id = p.product_id
  * left join product_class pc
- * on s.product_id = pc.product_id</pre></blockquote>
+ *   on s.product_id = pc.product_id</pre></blockquote>
  *
  * <p>becomes
  *
  * <blockquote>
- * <pre>select distinct s.product_id, pc.product_id from
- * sales as s
+ * <pre>select distinct s.product_id, pc.product_id
+ * from sales as s
  * left join product_class pc
- * on s.product_id = pc.product_id</pre></blockquote>
+ *   on s.product_id = pc.product_id</pre></blockquote>
  *
+ * @see CoreRules#AGGREGATE_JOIN_JOIN_REMOVE
  */
-public class AggregateJoinJoinRemoveRule extends RelOptRule
+public class AggregateJoinJoinRemoveRule
+    extends RelRule<AggregateJoinJoinRemoveRule.Config>
     implements TransformationRule {
-  /** @deprecated Use {@link CoreRules#AGGREGATE_JOIN_JOIN_REMOVE}. */
-  @Deprecated // to be removed before 1.25
-  public static final AggregateJoinJoinRemoveRule INSTANCE =
-      CoreRules.AGGREGATE_JOIN_JOIN_REMOVE;
 
   /** Creates an AggregateJoinJoinRemoveRule. */
+  protected AggregateJoinJoinRemoveRule(Config config) {
+    super(config);
+  }
+
+  @Deprecated // to be removed before 2.0
   public AggregateJoinJoinRemoveRule(
       Class<? extends Aggregate> aggregateClass,
       Class<? extends Join> joinClass, RelBuilderFactory relBuilderFactory) {
-    super(
-        operand(aggregateClass,
-            operandJ(joinClass, null,
-                join -> join.getJoinType() == JoinRelType.LEFT,
-                operandJ(joinClass, null,
-                    join -> join.getJoinType() == JoinRelType.LEFT, any()))),
-        relBuilderFactory, null);
+    this(Config.DEFAULT
+        .withRelBuilderFactory(relBuilderFactory)
+        .as(Config.class)
+        .withOperandFor(aggregateClass, joinClass));
   }
 
   @Override public void onMatch(RelOptRuleCall call) {
@@ -152,4 +154,26 @@ public class AggregateJoinJoinRemoveRule extends RelOptRule
 
     call.transformTo(newAggregate);
   }
+
+  /** Rule configuration. */
+  public interface Config extends RelRule.Config {
+    Config DEFAULT = EMPTY
+        .as(Config.class)
+        .withOperandFor(LogicalAggregate.class, LogicalJoin.class);
+
+    @Override default AggregateJoinJoinRemoveRule toRule() {
+      return new AggregateJoinJoinRemoveRule(this);
+    }
+
+    /** Defines an operand tree for the given classes. */
+    default Config withOperandFor(Class<? extends Aggregate> aggregateClass,
+        Class<? extends Join> joinClass) {
+      return withOperandSupplier(b0 -> b0.operand(aggregateClass)
+          .oneInput(b1 -> b1.operand(joinClass)
+              .predicate(join -> join.getJoinType() == JoinRelType.LEFT)
+              .inputs(b2 -> b2.operand(joinClass)
+                  .predicate(join -> join.getJoinType() == JoinRelType.LEFT)
+                  .anyInputs()))).as(Config.class);
+    }
+  }
 }
diff --git a/core/src/main/java/org/apache/calcite/rel/rules/AggregateJoinRemoveRule.java b/core/src/main/java/org/apache/calcite/rel/rules/AggregateJoinRemoveRule.java
index 032dfdb..41ca66a 100644
--- a/core/src/main/java/org/apache/calcite/rel/rules/AggregateJoinRemoveRule.java
+++ b/core/src/main/java/org/apache/calcite/rel/rules/AggregateJoinRemoveRule.java
@@ -16,14 +16,16 @@
  */
 package org.apache.calcite.rel.rules;
 
-import org.apache.calcite.plan.RelOptRule;
 import org.apache.calcite.plan.RelOptRuleCall;
 import org.apache.calcite.plan.RelOptUtil;
+import org.apache.calcite.plan.RelRule;
 import org.apache.calcite.rel.RelNode;
 import org.apache.calcite.rel.core.Aggregate;
 import org.apache.calcite.rel.core.AggregateCall;
 import org.apache.calcite.rel.core.Join;
 import org.apache.calcite.rel.core.JoinRelType;
+import org.apache.calcite.rel.logical.LogicalAggregate;
+import org.apache.calcite.rel.logical.LogicalJoin;
 import org.apache.calcite.tools.RelBuilder;
 import org.apache.calcite.tools.RelBuilderFactory;
 import org.apache.calcite.util.ImmutableBitSet;
@@ -54,23 +56,25 @@ import java.util.Set;
  * <blockquote>
  * <pre>select distinct s.product_id from sales as s</pre></blockquote>
  *
+ * @see CoreRules#AGGREGATE_JOIN_REMOVE
  */
-public class AggregateJoinRemoveRule extends RelOptRule implements TransformationRule {
-  /** @deprecated Use {@link CoreRules#AGGREGATE_JOIN_REMOVE}. */
-  @Deprecated // to be removed before 1.25
-  public static final AggregateJoinRemoveRule INSTANCE =
-      CoreRules.AGGREGATE_JOIN_REMOVE;
+public class AggregateJoinRemoveRule
+    extends RelRule<AggregateJoinRemoveRule.Config>
+    implements TransformationRule {
 
   /** Creates an AggregateJoinRemoveRule. */
+  protected AggregateJoinRemoveRule(Config config) {
+    super(config);
+  }
+
+  @Deprecated // to be removed before 2.0
   public AggregateJoinRemoveRule(
       Class<? extends Aggregate> aggregateClass,
       Class<? extends Join> joinClass, RelBuilderFactory relBuilderFactory) {
-    super(
-        operand(aggregateClass,
-            operandJ(joinClass, null,
-                join -> join.getJoinType() == JoinRelType.LEFT
-                    || join.getJoinType() == JoinRelType.RIGHT, any())),
-        relBuilderFactory, null);
+    this(Config.DEFAULT
+        .withRelBuilderFactory(relBuilderFactory)
+        .as(Config.class)
+        .withOperandFor(aggregateClass, joinClass));
   }
 
   @Override public void onMatch(RelOptRuleCall call) {
@@ -119,4 +123,27 @@ public class AggregateJoinRemoveRule extends RelOptRule implements Transformatio
     }
     call.transformTo(node);
   }
+
+  /** Rule configuration. */
+  public interface Config extends RelRule.Config {
+    Config DEFAULT = EMPTY.as(Config.class)
+        .withOperandFor(LogicalAggregate.class, LogicalJoin.class);
+
+    @Override default AggregateJoinRemoveRule toRule() {
+      return new AggregateJoinRemoveRule(this);
+    }
+
+    /** Defines an operand tree for the given classes. */
+    default Config withOperandFor(Class<? extends Aggregate> aggregateClass,
+        Class<? extends Join> joinClass) {
+      return withOperandSupplier(b0 ->
+          b0.operand(aggregateClass).oneInput(b1 ->
+              b1.operand(joinClass)
+                  .predicate(join ->
+                      join.getJoinType() == JoinRelType.LEFT
+                          || join.getJoinType() == JoinRelType.RIGHT)
+                  .anyInputs()))
+          .as(Config.class);
+    }
+  }
 }
diff --git a/core/src/main/java/org/apache/calcite/rel/rules/AggregateJoinTransposeRule.java b/core/src/main/java/org/apache/calcite/rel/rules/AggregateJoinTransposeRule.java
index d5b6062..2e46153 100644
--- a/core/src/main/java/org/apache/calcite/rel/rules/AggregateJoinTransposeRule.java
+++ b/core/src/main/java/org/apache/calcite/rel/rules/AggregateJoinTransposeRule.java
@@ -17,15 +17,17 @@
 package org.apache.calcite.rel.rules;
 
 import org.apache.calcite.linq4j.Ord;
-import org.apache.calcite.plan.RelOptRule;
 import org.apache.calcite.plan.RelOptRuleCall;
 import org.apache.calcite.plan.RelOptUtil;
+import org.apache.calcite.plan.RelRule;
 import org.apache.calcite.rel.RelNode;
 import org.apache.calcite.rel.core.Aggregate;
 import org.apache.calcite.rel.core.AggregateCall;
 import org.apache.calcite.rel.core.Join;
 import org.apache.calcite.rel.core.JoinRelType;
 import org.apache.calcite.rel.core.RelFactories;
+import org.apache.calcite.rel.logical.LogicalAggregate;
+import org.apache.calcite.rel.logical.LogicalJoin;
 import org.apache.calcite.rel.metadata.RelMetadataQuery;
 import org.apache.calcite.rel.type.RelDataType;
 import org.apache.calcite.rex.RexBuilder;
@@ -38,6 +40,7 @@ import org.apache.calcite.sql.SqlSplittableAggFunction;
 import org.apache.calcite.tools.RelBuilder;
 import org.apache.calcite.tools.RelBuilderFactory;
 import org.apache.calcite.util.Bug;
+import org.apache.calcite.util.ImmutableBeans;
 import org.apache.calcite.util.ImmutableBitSet;
 import org.apache.calcite.util.Util;
 import org.apache.calcite.util.mapping.Mapping;
@@ -58,29 +61,27 @@ import java.util.TreeMap;
  * Planner rule that pushes an
  * {@link org.apache.calcite.rel.core.Aggregate}
  * past a {@link org.apache.calcite.rel.core.Join}.
+ *
+ * @see CoreRules#AGGREGATE_JOIN_TRANSPOSE
+ * @see CoreRules#AGGREGATE_JOIN_TRANSPOSE_EXTENDED
  */
-public class AggregateJoinTransposeRule extends RelOptRule implements TransformationRule {
-  /** @deprecated Use {@link CoreRules#AGGREGATE_JOIN_TRANSPOSE}. */
-  @Deprecated // to be removed before 1.25
-  public static final AggregateJoinTransposeRule INSTANCE =
-      CoreRules.AGGREGATE_JOIN_TRANSPOSE;
-
-  /** @deprecated Use {@link CoreRules#AGGREGATE_JOIN_TRANSPOSE_EXTENDED}. */
-  @Deprecated // to be removed before 1.25
-  public static final AggregateJoinTransposeRule EXTENDED =
-      CoreRules.AGGREGATE_JOIN_TRANSPOSE_EXTENDED;
-
-  private final boolean allowFunctions;
+public class AggregateJoinTransposeRule
+    extends RelRule<AggregateJoinTransposeRule.Config>
+    implements TransformationRule {
 
   /** Creates an AggregateJoinTransposeRule. */
+  protected AggregateJoinTransposeRule(Config config) {
+    super(config);
+  }
+
+  @Deprecated // to be removed before 2.0
   public AggregateJoinTransposeRule(Class<? extends Aggregate> aggregateClass,
       Class<? extends Join> joinClass, RelBuilderFactory relBuilderFactory,
       boolean allowFunctions) {
-    super(
-        operandJ(aggregateClass, null, agg -> isAggregateSupported(agg, allowFunctions),
-            operand(joinClass, null, any())),
-        relBuilderFactory, null);
-    this.allowFunctions = allowFunctions;
+    this(Config.DEFAULT
+        .withRelBuilderFactory(relBuilderFactory)
+        .as(Config.class)
+        .withOperandFor(aggregateClass, joinClass, allowFunctions));
   }
 
   @Deprecated // to be removed before 2.0
@@ -124,7 +125,8 @@ public class AggregateJoinTransposeRule extends RelOptRule implements Transforma
         allowFunctions);
   }
 
-  private static boolean isAggregateSupported(Aggregate aggregate, boolean allowFunctions) {
+  private static boolean isAggregateSupported(Aggregate aggregate,
+      boolean allowFunctions) {
     if (!allowFunctions && !aggregate.getAggCallList().isEmpty()) {
       return false;
     }
@@ -152,7 +154,7 @@ public class AggregateJoinTransposeRule extends RelOptRule implements Transforma
     return join.getJoinType() == JoinRelType.INNER || aggregate.getAggCallList().isEmpty();
   }
 
-  public void onMatch(RelOptRuleCall call) {
+  @Override public void onMatch(RelOptRuleCall call) {
     final Aggregate aggregate = call.rel(0);
     final Join join = call.rel(1);
     final RexBuilder rexBuilder = aggregate.getCluster().getRexBuilder();
@@ -213,7 +215,7 @@ public class AggregateJoinTransposeRule extends RelOptRule implements Transforma
       final ImmutableBitSet belowAggregateKey =
           belowAggregateKeyNotShifted.shift(-offset);
       final boolean unique;
-      if (!allowFunctions) {
+      if (!config.isAllowFunctions()) {
         assert aggregate.getAggCallList().isEmpty();
         // If there are no functions, it doesn't matter as much whether we
         // aggregate the inputs before the join, because there will not be
@@ -447,4 +449,39 @@ public class AggregateJoinTransposeRule extends RelOptRule implements Transforma
     RelNode newInput;
     boolean aggregate;
   }
+
+  /** Rule configuration. */
+  public interface Config extends RelRule.Config {
+    Config DEFAULT = EMPTY.as(Config.class)
+        .withOperandFor(LogicalAggregate.class, LogicalJoin.class, false);
+
+    /** Extended instance that can push down aggregate functions. */
+    Config EXTENDED = EMPTY.as(Config.class)
+        .withOperandFor(LogicalAggregate.class, LogicalJoin.class, true);
+
+    @Override default AggregateJoinTransposeRule toRule() {
+      return new AggregateJoinTransposeRule(this);
+    }
+
+    /** Whether to push down aggregate functions, default false. */
+    @ImmutableBeans.Property
+    @ImmutableBeans.BooleanDefault(false)
+    boolean isAllowFunctions();
+
+    /** Sets {@link #isAllowFunctions()}. */
+    Config withAllowFunctions(boolean allowFunctions);
+
+    /** Defines an operand tree for the given classes, and also sets
+     * {@link #isAllowFunctions()}. */
+    default Config withOperandFor(Class<? extends Aggregate> aggregateClass,
+        Class<? extends Join> joinClass, boolean allowFunctions) {
+      return withAllowFunctions(allowFunctions)
+          .withOperandSupplier(b0 ->
+              b0.operand(aggregateClass)
+                  .predicate(agg -> isAggregateSupported(agg, allowFunctions))
+              .oneInput(b1 ->
+                  b1.operand(joinClass).anyInputs()))
+          .as(Config.class);
+    }
+  }
 }
diff --git a/core/src/main/java/org/apache/calcite/rel/rules/AggregateMergeRule.java b/core/src/main/java/org/apache/calcite/rel/rules/AggregateMergeRule.java
index f9bdc52..fa063fd 100644
--- a/core/src/main/java/org/apache/calcite/rel/rules/AggregateMergeRule.java
+++ b/core/src/main/java/org/apache/calcite/rel/rules/AggregateMergeRule.java
@@ -16,13 +16,12 @@
  */
 package org.apache.calcite.rel.rules;
 
-import org.apache.calcite.plan.RelOptRule;
 import org.apache.calcite.plan.RelOptRuleCall;
 import org.apache.calcite.plan.RelOptRuleOperand;
+import org.apache.calcite.plan.RelRule;
 import org.apache.calcite.rel.core.Aggregate;
 import org.apache.calcite.rel.core.Aggregate.Group;
 import org.apache.calcite.rel.core.AggregateCall;
-import org.apache.calcite.rel.core.RelFactories;
 import org.apache.calcite.sql.SqlSplittableAggFunction;
 import org.apache.calcite.sql.fun.SqlStdOperatorTable;
 import org.apache.calcite.tools.RelBuilderFactory;
@@ -45,25 +44,24 @@ import java.util.Objects;
  * <p>For example, SUM of SUM becomes SUM; SUM of COUNT becomes COUNT;
  * MAX of MAX becomes MAX; MIN of MIN becomes MIN. AVG of AVG would not
  * match, nor would COUNT of COUNT.
+ *
+ * @see CoreRules#AGGREGATE_MERGE
  */
-public class AggregateMergeRule extends RelOptRule implements TransformationRule {
-  /** @deprecated Use {@link CoreRules#AGGREGATE_MERGE}. */
-  @Deprecated // to be removed before 1.25
-  public static final AggregateMergeRule INSTANCE =
-      CoreRules.AGGREGATE_MERGE;
-
-  AggregateMergeRule() {
-    this(
-        operand(Aggregate.class,
-            operandJ(Aggregate.class, null,
-                agg -> Aggregate.isSimple(agg), any())),
-        RelFactories.LOGICAL_BUILDER);
-  }
+public class AggregateMergeRule
+    extends RelRule<AggregateMergeRule.Config>
+    implements TransformationRule {
 
   /** Creates an AggregateMergeRule. */
+  protected AggregateMergeRule(Config config) {
+    super(config);
+  }
+
+  @Deprecated // to be removed before 2.0
   public AggregateMergeRule(RelOptRuleOperand operand,
       RelBuilderFactory relBuilderFactory) {
-    super(operand, relBuilderFactory, null);
+    this(Config.DEFAULT.withRelBuilderFactory(relBuilderFactory)
+        .withOperandSupplier(b -> b.exactly(operand))
+        .as(Config.class));
   }
 
   private boolean isAggregateSupported(AggregateCall aggCall) {
@@ -78,7 +76,7 @@ public class AggregateMergeRule extends RelOptRule implements TransformationRule
     return splitter != null;
   }
 
-  public void onMatch(RelOptRuleCall call) {
+  @Override public void onMatch(RelOptRuleCall call) {
     final Aggregate topAgg = call.rel(0);
     final Aggregate bottomAgg = call.rel(1);
     if (topAgg.getGroupCount() > bottomAgg.getGroupCount()) {
@@ -148,4 +146,20 @@ public class AggregateMergeRule extends RelOptRule implements TransformationRule
             newGroupingSets, finalCalls);
     call.transformTo(finalAgg);
   }
+
+  /** Rule configuration. */
+  public interface Config extends RelRule.Config {
+    Config DEFAULT = EMPTY
+        .withOperandSupplier(b0 ->
+            b0.operand(Aggregate.class)
+                .oneInput(b1 ->
+                    b1.operand(Aggregate.class)
+                        .predicate(Aggregate::isSimple)
+                        .anyInputs()))
+        .as(Config.class);
+
+    @Override default AggregateMergeRule toRule() {
+      return new AggregateMergeRule(this);
+    }
+  }
 }
diff --git a/core/src/main/java/org/apache/calcite/rel/rules/AggregateProjectMergeRule.java b/core/src/main/java/org/apache/calcite/rel/rules/AggregateProjectMergeRule.java
index 6ca4260..49d56e5 100644
--- a/core/src/main/java/org/apache/calcite/rel/rules/AggregateProjectMergeRule.java
+++ b/core/src/main/java/org/apache/calcite/rel/rules/AggregateProjectMergeRule.java
@@ -16,9 +16,9 @@
  */
 package org.apache.calcite.rel.rules;
 
-import org.apache.calcite.plan.RelOptRule;
 import org.apache.calcite.plan.RelOptRuleCall;
 import org.apache.calcite.plan.RelOptUtil;
+import org.apache.calcite.plan.RelRule;
 import org.apache.calcite.rel.RelNode;
 import org.apache.calcite.rel.core.Aggregate;
 import org.apache.calcite.rel.core.Aggregate.Group;
@@ -50,24 +50,30 @@ import java.util.Set;
  *
  * <p>In some cases, this rule has the effect of trimming: the aggregate will
  * use fewer columns than the project did.
+ *
+ * @see CoreRules#AGGREGATE_PROJECT_MERGE
  */
-public class AggregateProjectMergeRule extends RelOptRule implements TransformationRule {
-  /** @deprecated Use {@link CoreRules#AGGREGATE_PROJECT_MERGE}. */
-  @Deprecated // to be removed before 1.25
-  public static final AggregateProjectMergeRule INSTANCE =
-      CoreRules.AGGREGATE_PROJECT_MERGE;
+public class AggregateProjectMergeRule
+    extends RelRule<AggregateProjectMergeRule.Config>
+    implements TransformationRule {
+
+  /** Creates an AggregateProjectMergeRule. */
+  protected AggregateProjectMergeRule(Config config) {
+    super(config);
+  }
 
+  @Deprecated // to be removed before 2.0
   public AggregateProjectMergeRule(
       Class<? extends Aggregate> aggregateClass,
       Class<? extends Project> projectClass,
       RelBuilderFactory relBuilderFactory) {
-    super(
-        operand(aggregateClass,
-            operand(projectClass, any())),
-        relBuilderFactory, null);
+    this(CoreRules.AGGREGATE_PROJECT_MERGE.config
+        .withRelBuilderFactory(relBuilderFactory)
+        .as(Config.class)
+        .withOperandFor(aggregateClass, projectClass));
   }
 
-  public void onMatch(RelOptRuleCall call) {
+  @Override public void onMatch(RelOptRuleCall call) {
     final Aggregate aggregate = call.rel(0);
     final Project project = call.rel(1);
     RelNode x = apply(call, aggregate, project);
@@ -134,4 +140,22 @@ public class AggregateProjectMergeRule extends RelOptRule implements Transformat
 
     return relBuilder.build();
   }
+
+  /** Rule configuration. */
+  public interface Config extends RelRule.Config {
+    Config DEFAULT = EMPTY.as(Config.class)
+        .withOperandFor(Aggregate.class, Project.class);
+
+    @Override default AggregateProjectMergeRule toRule() {
+      return new AggregateProjectMergeRule(this);
+    }
+
+    /** Defines an operand tree for the given classes. */
+    default Config withOperandFor(Class<? extends Aggregate> aggregateClass,
+        Class<? extends Project> projectClass) {
+      return withOperandSupplier(b0 ->
+          b0.operand(aggregateClass).oneInput(b1 ->
+              b1.operand(projectClass).anyInputs())).as(Config.class);
+    }
+  }
 }
diff --git a/core/src/main/java/org/apache/calcite/rel/rules/AggregateProjectPullUpConstantsRule.java b/core/src/main/java/org/apache/calcite/rel/rules/AggregateProjectPullUpConstantsRule.java
index 9484a0c..b044c6d 100644
--- a/core/src/main/java/org/apache/calcite/rel/rules/AggregateProjectPullUpConstantsRule.java
+++ b/core/src/main/java/org/apache/calcite/rel/rules/AggregateProjectPullUpConstantsRule.java
@@ -17,11 +17,12 @@
 package org.apache.calcite.rel.rules;
 
 import org.apache.calcite.plan.RelOptPredicateList;
-import org.apache.calcite.plan.RelOptRule;
 import org.apache.calcite.plan.RelOptRuleCall;
+import org.apache.calcite.plan.RelRule;
 import org.apache.calcite.rel.RelNode;
 import org.apache.calcite.rel.core.Aggregate;
 import org.apache.calcite.rel.core.AggregateCall;
+import org.apache.calcite.rel.logical.LogicalAggregate;
 import org.apache.calcite.rel.logical.LogicalProject;
 import org.apache.calcite.rel.metadata.RelMetadataQuery;
 import org.apache.calcite.rel.type.RelDataType;
@@ -55,43 +56,30 @@ import java.util.TreeMap;
  * reduced aggregate. If those constants are not used, another rule will remove
  * them from the project.
  */
-public class AggregateProjectPullUpConstantsRule extends RelOptRule
+public class AggregateProjectPullUpConstantsRule
+    extends RelRule<AggregateProjectPullUpConstantsRule.Config>
     implements TransformationRule {
-  //~ Static fields/initializers ---------------------------------------------
-
-  /** @deprecated Use {@link CoreRules#AGGREGATE_PROJECT_PULL_UP_CONSTANTS}. */
-  @Deprecated // to be removed before 1.25
-  public static final AggregateProjectPullUpConstantsRule INSTANCE =
-      CoreRules.AGGREGATE_PROJECT_PULL_UP_CONSTANTS;
-
-  /** @deprecated Use {@link CoreRules#AGGREGATE_ANY_PULL_UP_CONSTANTS}. */
-  @Deprecated // to be removed before 1.25
-  public static final AggregateProjectPullUpConstantsRule INSTANCE2 =
-      CoreRules.AGGREGATE_ANY_PULL_UP_CONSTANTS;
-
-  //~ Constructors -----------------------------------------------------------
-
-  /**
-   * Creates an AggregateProjectPullUpConstantsRule.
-   *
-   * @param aggregateClass Aggregate class
-   * @param inputClass Input class, such as {@link LogicalProject}
-   * @param relBuilderFactory Builder for relational expressions
-   * @param description Description, or null to guess description
-   */
+
+  /** Creates an AggregateProjectPullUpConstantsRule. */
+  protected AggregateProjectPullUpConstantsRule(Config config) {
+    super(config);
+  }
+
+  @Deprecated // to be removed before 2.0
   public AggregateProjectPullUpConstantsRule(
       Class<? extends Aggregate> aggregateClass,
       Class<? extends RelNode> inputClass,
       RelBuilderFactory relBuilderFactory, String description) {
-    super(
-        operandJ(aggregateClass, null, Aggregate::isSimple,
-            operand(inputClass, any())),
-        relBuilderFactory, description);
+    this(Config.DEFAULT
+        .withRelBuilderFactory(relBuilderFactory)
+        .withDescription(description)
+        .as(Config.class)
+        .withOperandFor(aggregateClass, inputClass));
   }
 
   //~ Methods ----------------------------------------------------------------
 
-  public void onMatch(RelOptRuleCall call) {
+  @Override public void onMatch(RelOptRuleCall call) {
     final Aggregate aggregate = call.rel(0);
     final RelNode input = call.rel(1);
 
@@ -185,4 +173,29 @@ public class AggregateProjectPullUpConstantsRule extends RelOptRule
     call.transformTo(relBuilder.build());
   }
 
+
+  /** Rule configuration. */
+  public interface Config extends RelRule.Config {
+    Config DEFAULT = EMPTY.as(Config.class)
+        .withOperandFor(LogicalAggregate.class, LogicalProject.class);
+
+    @Override default AggregateProjectPullUpConstantsRule toRule() {
+      return new AggregateProjectPullUpConstantsRule(this);
+    }
+
+    /** Defines an operand tree for the given classes.
+     *
+     * @param aggregateClass Aggregate class
+     * @param inputClass Input class, such as {@link LogicalProject}
+     */
+    default Config withOperandFor(Class<? extends Aggregate> aggregateClass,
+        Class<? extends RelNode> inputClass) {
+      return withOperandSupplier(b0 ->
+          b0.operand(aggregateClass)
+              .predicate(Aggregate::isSimple)
+              .oneInput(b1 ->
+                  b1.operand(inputClass).anyInputs()))
+          .as(Config.class);
+    }
+  }
 }
diff --git a/core/src/main/java/org/apache/calcite/rel/rules/AggregateProjectStarTableRule.java b/core/src/main/java/org/apache/calcite/rel/rules/AggregateProjectStarTableRule.java
new file mode 100644
index 0000000..f641521
--- /dev/null
+++ b/core/src/main/java/org/apache/calcite/rel/rules/AggregateProjectStarTableRule.java
@@ -0,0 +1,78 @@
+/*
+ * 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.rel.rules;
+
+import org.apache.calcite.plan.RelOptRuleCall;
+import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rel.core.Aggregate;
+import org.apache.calcite.rel.core.Project;
+import org.apache.calcite.schema.impl.StarTable;
+
+/** Variant of {@link AggregateStarTableRule} that accepts a {@link Project}
+ * between the {@link Aggregate} and its {@link StarTable.StarTableScan}
+ * input. */
+public class AggregateProjectStarTableRule extends AggregateStarTableRule {
+  /** Creates an AggregateProjectStarTableRule. */
+  protected AggregateProjectStarTableRule(Config config) {
+    super(config);
+  }
+
+  @Override public void onMatch(RelOptRuleCall call) {
+    final Aggregate aggregate = call.rel(0);
+    final Project project = call.rel(1);
+    final StarTable.StarTableScan scan = call.rel(2);
+    final RelNode rel =
+        AggregateProjectMergeRule.apply(call, aggregate, project);
+    final Aggregate aggregate2;
+    final Project project2;
+    if (rel instanceof Aggregate) {
+      project2 = null;
+      aggregate2 = (Aggregate) rel;
+    } else if (rel instanceof Project) {
+      project2 = (Project) rel;
+      aggregate2 = (Aggregate) project2.getInput();
+    } else {
+      return;
+    }
+    apply(call, project2, aggregate2, scan);
+  }
+
+  /** Rule configuration. */
+  public interface Config extends AggregateStarTableRule.Config {
+    Config DEFAULT = EMPTY.as(Config.class)
+        .withOperandFor(Aggregate.class, Project.class,
+            StarTable.StarTableScan.class);
+
+    @Override default AggregateProjectStarTableRule toRule() {
+      return new AggregateProjectStarTableRule(this);
+    }
+
+    /** Defines an operand tree for the given classes. */
+    default Config withOperandFor(Class<? extends Aggregate> aggregateClass,
+        Class<? extends Project> projectClass,
+        Class<StarTable.StarTableScan> scanClass) {
+      return withOperandSupplier(b0 ->
+          b0.operand(aggregateClass)
+              .predicate(Aggregate::isSimple)
+              .oneInput(b1 ->
+                  b1.operand(projectClass)
+                      .oneInput(b2 ->
+                          b2.operand(scanClass).noInputs())))
+          .as(Config.class);
+    }
+  }
+}
diff --git a/core/src/main/java/org/apache/calcite/rel/rules/AggregateReduceFunctionsRule.java b/core/src/main/java/org/apache/calcite/rel/rules/AggregateReduceFunctionsRule.java
index d5ce21d..0ffbbcb 100644
--- a/core/src/main/java/org/apache/calcite/rel/rules/AggregateReduceFunctionsRule.java
+++ b/core/src/main/java/org/apache/calcite/rel/rules/AggregateReduceFunctionsRule.java
@@ -17,12 +17,13 @@
 package org.apache.calcite.rel.rules;
 
 import org.apache.calcite.plan.RelOptCluster;
-import org.apache.calcite.plan.RelOptRule;
 import org.apache.calcite.plan.RelOptRuleCall;
 import org.apache.calcite.plan.RelOptRuleOperand;
+import org.apache.calcite.plan.RelRule;
 import org.apache.calcite.rel.RelNode;
 import org.apache.calcite.rel.core.Aggregate;
 import org.apache.calcite.rel.core.AggregateCall;
+import org.apache.calcite.rel.logical.LogicalAggregate;
 import org.apache.calcite.rel.type.RelDataType;
 import org.apache.calcite.rel.type.RelDataTypeFactory;
 import org.apache.calcite.rel.type.RelDataTypeField;
@@ -36,11 +37,13 @@ import org.apache.calcite.sql.type.SqlTypeUtil;
 import org.apache.calcite.tools.RelBuilder;
 import org.apache.calcite.tools.RelBuilderFactory;
 import org.apache.calcite.util.CompositeList;
+import org.apache.calcite.util.ImmutableBeans;
 import org.apache.calcite.util.ImmutableBitSet;
 import org.apache.calcite.util.ImmutableIntList;
 import org.apache.calcite.util.Util;
 
 import com.google.common.collect.ImmutableList;
+import com.google.common.collect.ImmutableSet;
 
 import java.math.BigDecimal;
 import java.util.ArrayList;
@@ -50,6 +53,8 @@ import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
 import java.util.Objects;
+import java.util.Set;
+import javax.annotation.Nonnull;
 
 /**
  * Planner rule that reduces aggregate functions in
@@ -89,69 +94,62 @@ import java.util.Objects;
  * <p>Since many of these rewrites introduce multiple occurrences of simpler
  * forms like {@code COUNT(x)}, the rule gathers common sub-expressions as it
  * goes.
+ *
+ * @see CoreRules#AGGREGATE_REDUCE_FUNCTIONS
  */
-public class AggregateReduceFunctionsRule extends RelOptRule
+public class AggregateReduceFunctionsRule
+    extends RelRule<AggregateReduceFunctionsRule.Config>
     implements TransformationRule {
   //~ Static fields/initializers ---------------------------------------------
 
-  /** @deprecated Use {@link CoreRules#AGGREGATE_REDUCE_FUNCTIONS}. */
-  @Deprecated // to be removed before 1.25
-  public static final AggregateReduceFunctionsRule INSTANCE =
-      CoreRules.AGGREGATE_REDUCE_FUNCTIONS;
+  private static void validateFunction(SqlKind function) {
+    if (!isValid(function)) {
+      throw new IllegalArgumentException("AggregateReduceFunctionsRule doesn't "
+          + "support function: " + function.sql);
+    }
+  }
 
-  private final EnumSet<SqlKind> functionsToReduce;
+  private static boolean isValid(SqlKind function) {
+    return SqlKind.AVG_AGG_FUNCTIONS.contains(function)
+        || SqlKind.COVAR_AVG_AGG_FUNCTIONS.contains(function)
+        || function == SqlKind.SUM;
+  }
+
+  private final Set<SqlKind> functionsToReduce;
 
   //~ Constructors -----------------------------------------------------------
 
-  /**
-   * Creates an AggregateReduceFunctionsRule to reduce all functions
-   * handled by this rule
-   * @param operand operand to determine if rule can be applied
-   * @param relBuilderFactory builder for relational expressions
-   */
+  /** Creates an AggregateReduceFunctionsRule. */
+  protected AggregateReduceFunctionsRule(Config config) {
+    super(config);
+    this.functionsToReduce =
+        ImmutableSet.copyOf(config.actualFunctionsToReduce());
+  }
+
+  @Deprecated // to be removed before 2.0
   public AggregateReduceFunctionsRule(RelOptRuleOperand operand,
       RelBuilderFactory relBuilderFactory) {
-    super(operand, relBuilderFactory, null);
-    functionsToReduce = EnumSet.noneOf(SqlKind.class);
-    addDefaultSetOfFunctionsToReduce();
+    this(Config.DEFAULT
+        .withRelBuilderFactory(relBuilderFactory)
+        .withOperandSupplier(b -> b.exactly(operand))
+        .as(Config.class)
+        // reduce all functions handled by this rule
+        .withFunctionsToReduce(null));
   }
 
-  /**
-   * Creates an AggregateReduceFunctionsRule with client
-   * provided information on which specific functions will
-   * be reduced by this rule
-   * @param aggregateClass aggregate class
-   * @param relBuilderFactory builder for relational expressions
-   * @param functionsToReduce client provided information
-   *                          on which specific functions
-   *                          will be reduced by this rule
-   */
+  @Deprecated // to be removed before 2.0
   public AggregateReduceFunctionsRule(Class<? extends Aggregate> aggregateClass,
       RelBuilderFactory relBuilderFactory, EnumSet<SqlKind> functionsToReduce) {
-    super(operand(aggregateClass, any()), relBuilderFactory, null);
-    Objects.requireNonNull(functionsToReduce,
-        "Expecting a valid handle for AggregateFunctionsToReduce");
-    this.functionsToReduce = EnumSet.noneOf(SqlKind.class);
-    for (SqlKind function : functionsToReduce) {
-      if (SqlKind.AVG_AGG_FUNCTIONS.contains(function)
-          || SqlKind.COVAR_AVG_AGG_FUNCTIONS.contains(function)
-          || function == SqlKind.SUM) {
-        this.functionsToReduce.add(function);
-      } else {
-        throw new IllegalArgumentException(
-          "AggregateReduceFunctionsRule doesn't support function: " + function.sql);
-      }
-    }
+    this(Config.DEFAULT
+        .withRelBuilderFactory(relBuilderFactory)
+        .as(Config.class)
+        .withOperandFor(aggregateClass)
+        // reduce specific functions provided by the client
+        .withFunctionsToReduce(Objects.requireNonNull(functionsToReduce)));
   }
 
   //~ Methods ----------------------------------------------------------------
 
-  private void addDefaultSetOfFunctionsToReduce() {
-    functionsToReduce.addAll(SqlKind.AVG_AGG_FUNCTIONS);
-    functionsToReduce.addAll(SqlKind.COVAR_AVG_AGG_FUNCTIONS);
-    functionsToReduce.add(SqlKind.SUM);
-  }
-
   @Override public boolean matches(RelOptRuleCall call) {
     if (!super.matches(call)) {
       return false;
@@ -160,7 +158,7 @@ public class AggregateReduceFunctionsRule extends RelOptRule
     return containsAvgStddevVarCall(oldAggRel.getAggCallList());
   }
 
-  public void onMatch(RelOptRuleCall ruleCall) {
+  @Override public void onMatch(RelOptRuleCall ruleCall) {
     Aggregate oldAggRel = (Aggregate) ruleCall.rels[0];
     reduceAggs(ruleCall, oldAggRel);
   }
@@ -857,4 +855,42 @@ public class AggregateReduceFunctionsRule extends RelOptRule
         relNode.getRowType().getFieldList().get(i);
     return inputField.getType();
   }
+
+  /** Rule configuration. */
+  public interface Config extends RelRule.Config {
+    Config DEFAULT = EMPTY.as(Config.class)
+        .withOperandFor(LogicalAggregate.class);
+
+    Set<SqlKind> DEFAULT_FUNCTIONS_TO_REDUCE =
+        ImmutableSet.<SqlKind>builder()
+            .addAll(SqlKind.AVG_AGG_FUNCTIONS)
+            .addAll(SqlKind.COVAR_AVG_AGG_FUNCTIONS)
+            .add(SqlKind.SUM)
+            .build();
+
+    @Override default AggregateReduceFunctionsRule toRule() {
+      return new AggregateReduceFunctionsRule(this);
+    }
+
+    @ImmutableBeans.Property
+    Set<SqlKind> functionsToReduce();
+
+    /** Sets {@link #functionsToReduce}. */
+    Config withFunctionsToReduce(Set<SqlKind> functionSet);
+
+    /** Returns the validated set of functions to reduce, or the default set
+     * if not specified. */
+    @Nonnull default Set<SqlKind> actualFunctionsToReduce() {
+      final Set<SqlKind> set =
+          Util.first(functionsToReduce(), DEFAULT_FUNCTIONS_TO_REDUCE);
+      set.forEach(AggregateReduceFunctionsRule::validateFunction);
+      return set;
+    }
+
+    /** Defines an operand tree for the given classes. */
+    default Config withOperandFor(Class<? extends Aggregate> aggregateClass) {
+      return withOperandSupplier(b -> b.operand(aggregateClass).anyInputs())
+          .as(Config.class);
+    }
+  }
 }
diff --git a/core/src/main/java/org/apache/calcite/rel/rules/AggregateRemoveRule.java b/core/src/main/java/org/apache/calcite/rel/rules/AggregateRemoveRule.java
index 7ac5382..96017c6 100644
--- a/core/src/main/java/org/apache/calcite/rel/rules/AggregateRemoveRule.java
+++ b/core/src/main/java/org/apache/calcite/rel/rules/AggregateRemoveRule.java
@@ -16,12 +16,13 @@
  */
 package org.apache.calcite.rel.rules;
 
-import org.apache.calcite.plan.RelOptRule;
 import org.apache.calcite.plan.RelOptRuleCall;
+import org.apache.calcite.plan.RelRule;
 import org.apache.calcite.rel.RelNode;
 import org.apache.calcite.rel.core.Aggregate;
 import org.apache.calcite.rel.core.AggregateCall;
 import org.apache.calcite.rel.core.RelFactories;
+import org.apache.calcite.rel.logical.LogicalAggregate;
 import org.apache.calcite.rel.metadata.RelMetadataQuery;
 import org.apache.calcite.rex.RexBuilder;
 import org.apache.calcite.rex.RexNode;
@@ -43,28 +44,30 @@ import java.util.Objects;
  * (that is, it is implementing {@code SELECT DISTINCT}),
  * or all the aggregate functions are splittable,
  * and the underlying relational expression is already distinct.
+ *
+ * @see CoreRules#AGGREGATE_REMOVE
  */
-public class AggregateRemoveRule extends RelOptRule implements SubstitutionRule {
-  /** @deprecated Use {@link CoreRules#AGGREGATE_REMOVE}. */
-  @Deprecated // to be removed before 1.25
-  public static final AggregateRemoveRule INSTANCE =
-      CoreRules.AGGREGATE_REMOVE;
+public class AggregateRemoveRule
+    extends RelRule<AggregateRemoveRule.Config>
+    implements SubstitutionRule {
 
-  //~ Constructors -----------------------------------------------------------
+  /** Creates an AggregateRemoveRule. */
+  protected AggregateRemoveRule(Config config) {
+    super(config);
+  }
 
   @Deprecated // to be removed before 2.0
   public AggregateRemoveRule(Class<? extends Aggregate> aggregateClass) {
     this(aggregateClass, RelFactories.LOGICAL_BUILDER);
   }
 
-  /**
-   * Creates an AggregateRemoveRule.
-   */
+  @Deprecated // to be removed before 2.0
   public AggregateRemoveRule(Class<? extends Aggregate> aggregateClass,
       RelBuilderFactory relBuilderFactory) {
-    super(
-        operandJ(aggregateClass, null, agg -> isAggregateSupported(agg),
-            any()), relBuilderFactory, null);
+    this(Config.DEFAULT
+        .withRelBuilderFactory(relBuilderFactory)
+        .as(Config.class)
+        .withOperandFor(aggregateClass));
   }
 
   private static boolean isAggregateSupported(Aggregate aggregate) {
@@ -85,7 +88,7 @@ public class AggregateRemoveRule extends RelOptRule implements SubstitutionRule
 
   //~ Methods ----------------------------------------------------------------
 
-  public void onMatch(RelOptRuleCall call) {
+  @Override public void onMatch(RelOptRuleCall call) {
     final Aggregate aggregate = call.rel(0);
     final RelNode input = aggregate.getInput();
     final RelMetadataQuery mq = call.getMetadataQuery();
@@ -126,4 +129,25 @@ public class AggregateRemoveRule extends RelOptRule implements SubstitutionRule
     call.getPlanner().prune(aggregate);
     call.transformTo(relBuilder.build());
   }
+
+  /** Rule configuration. */
+  public interface Config extends RelRule.Config {
+    Config DEFAULT = EMPTY
+        .withRelBuilderFactory(RelFactories.LOGICAL_BUILDER)
+        .as(Config.class)
+        .withOperandFor(LogicalAggregate.class);
+
+    @Override default AggregateRemoveRule toRule() {
+      return new AggregateRemoveRule(this);
+    }
+
+    /** Defines an operand tree for the given classes. */
+    default Config withOperandFor(Class<? extends Aggregate> aggregateClass) {
+      return withOperandSupplier(b ->
+          b.operand(aggregateClass)
+              .predicate(AggregateRemoveRule::isAggregateSupported)
+              .anyInputs())
+          .as(Config.class);
+    }
+  }
 }
diff --git a/core/src/main/java/org/apache/calcite/rel/rules/AggregateStarTableRule.java b/core/src/main/java/org/apache/calcite/rel/rules/AggregateStarTableRule.java
index 2941d3a..859150c 100644
--- a/core/src/main/java/org/apache/calcite/rel/rules/AggregateStarTableRule.java
+++ b/core/src/main/java/org/apache/calcite/rel/rules/AggregateStarTableRule.java
@@ -24,10 +24,10 @@ import org.apache.calcite.materialize.TileKey;
 import org.apache.calcite.plan.RelOptCluster;
 import org.apache.calcite.plan.RelOptLattice;
 import org.apache.calcite.plan.RelOptPlanner;
-import org.apache.calcite.plan.RelOptRule;
 import org.apache.calcite.plan.RelOptRuleCall;
 import org.apache.calcite.plan.RelOptRuleOperand;
 import org.apache.calcite.plan.RelOptTable;
+import org.apache.calcite.plan.RelRule;
 import org.apache.calcite.plan.SubstitutionVisitor;
 import org.apache.calcite.plan.ViewExpanders;
 import org.apache.calcite.prepare.RelOptTableImpl;
@@ -56,29 +56,28 @@ import java.util.List;
  *
  * <p>This pattern indicates that an aggregate table may exist. The rule asks
  * the star table for an aggregate table at the required level of aggregation.
+ *
+ * @see AggregateProjectStarTableRule
+ * @see CoreRules#AGGREGATE_STAR_TABLE
+ * @see CoreRules#AGGREGATE_PROJECT_STAR_TABLE
  */
-public class AggregateStarTableRule extends RelOptRule implements TransformationRule {
-  /** @deprecated Use {@link CoreRules#AGGREGATE_STAR_TABLE}. */
-  @Deprecated // to be removed before 1.25
-  public static final AggregateStarTableRule INSTANCE =
-      CoreRules.AGGREGATE_STAR_TABLE;
-
-  /** @deprecated This field is prone to issues during class-loading;
-   * use {@link CoreRules#AGGREGATE_PROJECT_STAR_TABLE} instead. */
-  @Deprecated // to be removed before 1.25
-  public static final AggregateStarTableRule INSTANCE2 =
-      CoreRules.AGGREGATE_PROJECT_STAR_TABLE;
-
-  /**
-   * Creates an AggregateStarTableRule.
-   *
-   * @param operand           root operand, must not be null
-   * @param description       Description, or null to guess description
-   * @param relBuilderFactory Builder for relational expressions
-   */
+public class AggregateStarTableRule
+    extends RelRule<AggregateStarTableRule.Config>
+    implements TransformationRule {
+
+  /** Creates an AggregateStarTableRule. */
+  protected AggregateStarTableRule(Config config) {
+    super(config);
+  }
+
+  @Deprecated // to be removed before 2.0
   public AggregateStarTableRule(RelOptRuleOperand operand,
       RelBuilderFactory relBuilderFactory, String description) {
-    super(operand, relBuilderFactory, description);
+    this(Config.DEFAULT
+        .withRelBuilderFactory(relBuilderFactory)
+        .withDescription(description)
+        .withOperandSupplier(b -> b.exactly(operand))
+        .as(Config.class));
   }
 
   @Override public void onMatch(RelOptRuleCall call) {
@@ -245,4 +244,25 @@ public class AggregateStarTableRule extends RelOptRule implements Transformation
     }
     return -1;
   }
+
+  /** Rule configuration. */
+  public interface Config extends RelRule.Config {
+    Config DEFAULT = EMPTY.as(Config.class)
+        .withOperandFor(Aggregate.class, StarTable.StarTableScan.class);
+
+    @Override default AggregateStarTableRule toRule() {
+      return new AggregateStarTableRule(this);
+    }
+
+    /** Defines an operand tree for the given classes. */
+    default Config withOperandFor(Class<? extends Aggregate> aggregateClass,
+        Class<StarTable.StarTableScan> scanClass) {
+      return withOperandSupplier(b0 ->
+          b0.operand(aggregateClass)
+              .predicate(Aggregate::isSimple)
+              .oneInput(b1 ->
+                  b1.operand(scanClass).noInputs()))
+          .as(Config.class);
+    }
+  }
 }
diff --git a/core/src/main/java/org/apache/calcite/rel/rules/AggregateUnionAggregateRule.java b/core/src/main/java/org/apache/calcite/rel/rules/AggregateUnionAggregateRule.java
index 977d6d3..8453d39 100644
--- a/core/src/main/java/org/apache/calcite/rel/rules/AggregateUnionAggregateRule.java
+++ b/core/src/main/java/org/apache/calcite/rel/rules/AggregateUnionAggregateRule.java
@@ -16,13 +16,15 @@
  */
 package org.apache.calcite.rel.rules;
 
-import org.apache.calcite.plan.RelOptRule;
 import org.apache.calcite.plan.RelOptRuleCall;
 import org.apache.calcite.plan.RelOptUtil;
+import org.apache.calcite.plan.RelRule;
 import org.apache.calcite.rel.RelNode;
 import org.apache.calcite.rel.core.Aggregate;
 import org.apache.calcite.rel.core.RelFactories;
 import org.apache.calcite.rel.core.Union;
+import org.apache.calcite.rel.logical.LogicalAggregate;
+import org.apache.calcite.rel.logical.LogicalUnion;
 import org.apache.calcite.tools.RelBuilder;
 import org.apache.calcite.tools.RelBuilderFactory;
 
@@ -36,40 +38,33 @@ import org.apache.calcite.tools.RelBuilderFactory;
  * <p>This rule only handles cases where the
  * {@link org.apache.calcite.rel.core.Union}s
  * still have only two inputs.
+ *
+ * @see CoreRules#AGGREGATE_UNION_AGGREGATE
+ * @see CoreRules#AGGREGATE_UNION_AGGREGATE_FIRST
+ * @see CoreRules#AGGREGATE_UNION_AGGREGATE_SECOND
  */
-public class AggregateUnionAggregateRule extends RelOptRule implements TransformationRule {
-  /** @deprecated Use {@link CoreRules#AGGREGATE_UNION_AGGREGATE_FIRST}. */
-  @Deprecated // to be removed before 1.25
-  public static final AggregateUnionAggregateRule AGG_ON_FIRST_INPUT =
-      CoreRules.AGGREGATE_UNION_AGGREGATE_FIRST;
-
-  /** @deprecated Use {@link CoreRules#AGGREGATE_UNION_AGGREGATE_SECOND}. */
-  @Deprecated // to be removed before 1.25
-  public static final AggregateUnionAggregateRule AGG_ON_SECOND_INPUT =
-      CoreRules.AGGREGATE_UNION_AGGREGATE_SECOND;
+public class AggregateUnionAggregateRule
+    extends RelRule<AggregateUnionAggregateRule.Config>
+    implements TransformationRule {
 
-  /** @deprecated Use {@link CoreRules#AGGREGATE_UNION_AGGREGATE}. */
-  @Deprecated // to be removed before 1.25
-  public static final AggregateUnionAggregateRule INSTANCE =
-      CoreRules.AGGREGATE_UNION_AGGREGATE;
-
-  //~ Constructors -----------------------------------------------------------
+  /** Creates an AggregateUnionAggregateRule. */
+  protected AggregateUnionAggregateRule(Config config) {
+    super(config);
+  }
 
-  /**
-   * Creates a AggregateUnionAggregateRule.
-   */
+  @Deprecated // to be removed before 2.0
   public AggregateUnionAggregateRule(Class<? extends Aggregate> aggregateClass,
       Class<? extends Union> unionClass,
       Class<? extends RelNode> firstUnionInputClass,
       Class<? extends RelNode> secondUnionInputClass,
       RelBuilderFactory relBuilderFactory,
       String desc) {
-    super(
-        operandJ(aggregateClass, null, Aggregate::isSimple,
-            operand(unionClass,
-                operand(firstUnionInputClass, any()),
-                operand(secondUnionInputClass, any()))),
-        relBuilderFactory, desc);
+    this(Config.DEFAULT
+        .withRelBuilderFactory(relBuilderFactory)
+        .withDescription(desc)
+        .as(Config.class)
+        .withOperandFor(aggregateClass, unionClass, firstUnionInputClass,
+            secondUnionInputClass));
   }
 
   @Deprecated // to be removed before 2.0
@@ -101,7 +96,7 @@ public class AggregateUnionAggregateRule extends RelOptRule implements Transform
     }
   }
 
-  public void onMatch(RelOptRuleCall call) {
+  @Override public void onMatch(RelOptRuleCall call) {
     final Aggregate topAggRel = call.rel(0);
     final Union union = call.rel(1);
 
@@ -138,4 +133,44 @@ public class AggregateUnionAggregateRule extends RelOptRule implements Transform
         topAggRel.getAggCallList());
     call.transformTo(relBuilder.build());
   }
+
+  /** Rule configuration. */
+  public interface Config extends RelRule.Config {
+    Config DEFAULT = EMPTY
+        .withDescription("AggregateUnionAggregateRule")
+        .as(Config.class)
+        .withOperandFor(LogicalAggregate.class, LogicalUnion.class,
+            RelNode.class, RelNode.class);
+
+    Config AGG_FIRST = DEFAULT
+        .withDescription("AggregateUnionAggregateRule:first-input-agg")
+        .as(Config.class)
+        .withOperandFor(LogicalAggregate.class, LogicalUnion.class,
+            LogicalAggregate.class, RelNode.class);
+
+    Config AGG_SECOND = DEFAULT
+        .withDescription("AggregateUnionAggregateRule:second-input-agg")
+        .as(Config.class)
+        .withOperandFor(LogicalAggregate.class, LogicalUnion.class,
+            RelNode.class, LogicalAggregate.class);
+
+    @Override default AggregateUnionAggregateRule toRule() {
+      return new AggregateUnionAggregateRule(this);
+    }
+
+    /** Defines an operand tree for the given classes. */
+    default Config withOperandFor(Class<? extends Aggregate> aggregateClass,
+        Class<? extends Union> unionClass,
+        Class<? extends RelNode> firstUnionInputClass,
+        Class<? extends RelNode> secondUnionInputClass) {
+      return withOperandSupplier(b0 ->
+          b0.operand(aggregateClass)
+              .predicate(Aggregate::isSimple)
+              .oneInput(b1 ->
+                  b1.operand(unionClass).inputs(
+                      b2 -> b2.operand(firstUnionInputClass).anyInputs(),
+                      b3 -> b3.operand(secondUnionInputClass).anyInputs())))
+          .as(Config.class);
+    }
+  }
 }
diff --git a/core/src/main/java/org/apache/calcite/rel/rules/AggregateUnionTransposeRule.java b/core/src/main/java/org/apache/calcite/rel/rules/AggregateUnionTransposeRule.java
index 16c91d2..5bd165e 100644
--- a/core/src/main/java/org/apache/calcite/rel/rules/AggregateUnionTransposeRule.java
+++ b/core/src/main/java/org/apache/calcite/rel/rules/AggregateUnionTransposeRule.java
@@ -17,13 +17,15 @@
 package org.apache.calcite.rel.rules;
 
 import org.apache.calcite.linq4j.Ord;
-import org.apache.calcite.plan.RelOptRule;
 import org.apache.calcite.plan.RelOptRuleCall;
+import org.apache.calcite.plan.RelRule;
 import org.apache.calcite.rel.RelNode;
 import org.apache.calcite.rel.core.Aggregate;
 import org.apache.calcite.rel.core.AggregateCall;
 import org.apache.calcite.rel.core.RelFactories;
 import org.apache.calcite.rel.core.Union;
+import org.apache.calcite.rel.logical.LogicalAggregate;
+import org.apache.calcite.rel.logical.LogicalUnion;
 import org.apache.calcite.rel.metadata.RelMdUtil;
 import org.apache.calcite.rel.metadata.RelMetadataQuery;
 import org.apache.calcite.rel.type.RelDataType;
@@ -50,12 +52,12 @@ import java.util.Map;
  * Planner rule that pushes an
  * {@link org.apache.calcite.rel.core.Aggregate}
  * past a non-distinct {@link org.apache.calcite.rel.core.Union}.
+ *
+ * @see CoreRules#AGGREGATE_UNION_TRANSPOSE
  */
-public class AggregateUnionTransposeRule extends RelOptRule implements TransformationRule {
-  /** @deprecated Use {@link CoreRules#AGGREGATE_UNION_TRANSPOSE}. */
-  @Deprecated // to be removed before 1.25
-  public static final AggregateUnionTransposeRule INSTANCE =
-      CoreRules.AGGREGATE_UNION_TRANSPOSE;
+public class AggregateUnionTransposeRule
+    extends RelRule<AggregateUnionTransposeRule.Config>
+    implements TransformationRule {
 
   private static final Map<Class<? extends SqlAggFunction>, Boolean>
       SUPPORTED_AGGREGATES = new IdentityHashMap<>();
@@ -70,12 +72,17 @@ public class AggregateUnionTransposeRule extends RelOptRule implements Transform
   }
 
   /** Creates an AggregateUnionTransposeRule. */
+  protected AggregateUnionTransposeRule(Config config) {
+    super(config);
+  }
+
+  @Deprecated // to be removed before 2.0
   public AggregateUnionTransposeRule(Class<? extends Aggregate> aggregateClass,
       Class<? extends Union> unionClass, RelBuilderFactory relBuilderFactory) {
-    super(
-        operand(aggregateClass,
-            operand(unionClass, any())),
-        relBuilderFactory, null);
+    this(Config.DEFAULT
+        .withRelBuilderFactory(relBuilderFactory)
+        .as(Config.class)
+        .withOperandFor(aggregateClass, unionClass));
   }
 
   @Deprecated // to be removed before 2.0
@@ -87,7 +94,7 @@ public class AggregateUnionTransposeRule extends RelOptRule implements Transform
         RelBuilder.proto(aggregateFactory, setOpFactory));
   }
 
-  public void onMatch(RelOptRuleCall call) {
+  @Override public void onMatch(RelOptRuleCall call) {
     Aggregate aggRel = call.rel(0);
     Union union = call.rel(1);
 
@@ -182,4 +189,23 @@ public class AggregateUnionTransposeRule extends RelOptRule implements Transform
     }
     return newCalls;
   }
+
+  /** Rule configuration. */
+  public interface Config extends RelRule.Config {
+    Config DEFAULT = EMPTY.as(Config.class)
+        .withOperandFor(LogicalAggregate.class, LogicalUnion.class);
+
+    @Override default AggregateUnionTransposeRule toRule() {
+      return new AggregateUnionTransposeRule(this);
+    }
+
+    /** Defines an operand tree for the given classes. */
+    default Config withOperandFor(Class<? extends Aggregate> aggregateClass,
+        Class<? extends Union> unionClass) {
+      return withOperandSupplier(b0 ->
+          b0.operand(aggregateClass).oneInput(b1 ->
+              b1.operand(unionClass).anyInputs()))
+          .as(Config.class);
+    }
+  }
 }
diff --git a/core/src/main/java/org/apache/calcite/rel/rules/AggregateValuesRule.java b/core/src/main/java/org/apache/calcite/rel/rules/AggregateValuesRule.java
index a016e10..4c0bfa3 100644
--- a/core/src/main/java/org/apache/calcite/rel/rules/AggregateValuesRule.java
+++ b/core/src/main/java/org/apache/calcite/rel/rules/AggregateValuesRule.java
@@ -16,8 +16,8 @@
  */
 package org.apache.calcite.rel.rules;
 
-import org.apache.calcite.plan.RelOptRule;
 import org.apache.calcite.plan.RelOptRuleCall;
+import org.apache.calcite.plan.RelRule;
 import org.apache.calcite.rel.core.Aggregate;
 import org.apache.calcite.rel.core.AggregateCall;
 import org.apache.calcite.rel.core.Values;
@@ -53,24 +53,20 @@ import java.util.List;
  *
  * @see CoreRules#AGGREGATE_VALUES
  */
-public class AggregateValuesRule extends RelOptRule implements SubstitutionRule {
-  /** @deprecated Use {@link CoreRules#AGGREGATE_VALUES}. */
-  @Deprecated // to be removed before 1.25
-  public static final AggregateValuesRule INSTANCE =
-      CoreRules.AGGREGATE_VALUES;
+public class AggregateValuesRule
+    extends RelRule<AggregateValuesRule.Config>
+    implements SubstitutionRule {
 
-  /**
-   * Creates an AggregateValuesRule.
-   *
-   * @param relBuilderFactory Builder for relational expressions
-   */
+  /** Creates an AggregateValuesRule. */
+  protected AggregateValuesRule(Config config) {
+    super(config);
+  }
+
+  @Deprecated // to be removed before 2.0
   public AggregateValuesRule(RelBuilderFactory relBuilderFactory) {
-    super(
-        operandJ(Aggregate.class, null,
-            aggregate -> aggregate.getGroupCount() == 0,
-            operandJ(Values.class, null,
-                values -> values.getTuples().isEmpty(), none())),
-        relBuilderFactory, null);
+    this(Config.DEFAULT
+        .withRelBuilderFactory(relBuilderFactory)
+        .as(Config.class));
   }
 
   @Override public void onMatch(RelOptRuleCall call) {
@@ -109,4 +105,27 @@ public class AggregateValuesRule extends RelOptRule implements SubstitutionRule
     // New plan is absolutely better than old plan.
     call.getPlanner().prune(aggregate);
   }
+
+  /** Rule configuration. */
+  public interface Config extends RelRule.Config {
+    Config DEFAULT = EMPTY.as(Config.class)
+        .withOperandFor(Aggregate.class, Values.class);
+
+    @Override default AggregateValuesRule toRule() {
+      return new AggregateValuesRule(this);
+    }
+
+    /** Defines an operand tree for the given classes. */
+    default Config withOperandFor(Class<? extends Aggregate> aggregateClass,
+        Class<? extends Values> valuesClass) {
+      return withOperandSupplier(b0 ->
+          b0.operand(aggregateClass)
+              .predicate(aggregate -> aggregate.getGroupCount() == 0)
+              .oneInput(b1 ->
+                  b1.operand(valuesClass)
+                      .predicate(values -> values.getTuples().isEmpty())
+                      .noInputs()))
+          .as(Config.class);
+    }
+  }
 }
diff --git a/core/src/main/java/org/apache/calcite/rel/rules/CalcMergeRule.java b/core/src/main/java/org/apache/calcite/rel/rules/CalcMergeRule.java
index 1b4fbd8..bb4c56d 100644
--- a/core/src/main/java/org/apache/calcite/rel/rules/CalcMergeRule.java
+++ b/core/src/main/java/org/apache/calcite/rel/rules/CalcMergeRule.java
@@ -16,8 +16,8 @@
  */
 package org.apache.calcite.rel.rules;
 
-import org.apache.calcite.plan.RelOptRule;
 import org.apache.calcite.plan.RelOptRuleCall;
+import org.apache.calcite.plan.RelRule;
 import org.apache.calcite.rel.core.Calc;
 import org.apache.calcite.rex.RexOver;
 import org.apache.calcite.rex.RexProgram;
@@ -33,33 +33,26 @@ import org.apache.calcite.tools.RelBuilderFactory;
  * same project list as the upper
  * {@link org.apache.calcite.rel.logical.LogicalCalc}, but expressed in terms of
  * the lower {@link org.apache.calcite.rel.logical.LogicalCalc}'s inputs.
+ *
+ * @see CoreRules#CALC_MERGE
  */
-public class CalcMergeRule extends RelOptRule implements TransformationRule {
-  //~ Static fields/initializers ---------------------------------------------
-
-  /** @deprecated Use {@link CoreRules#CALC_MERGE}. */
-  @Deprecated // to be removed before 1.25
-  public static final CalcMergeRule INSTANCE =
-      CoreRules.CALC_MERGE;
+public class CalcMergeRule extends RelRule<CalcMergeRule.Config>
+    implements TransformationRule {
 
-  //~ Constructors -----------------------------------------------------------
+  /** Creates a CalcMergeRule. */
+  protected CalcMergeRule(Config config) {
+    super(config);
+  }
 
-  /**
-   * Creates a CalcMergeRule.
-   *
-   * @param relBuilderFactory Builder for relational expressions
-   */
+  @Deprecated // to be removed before 2.0
   public CalcMergeRule(RelBuilderFactory relBuilderFactory) {
-    super(
-        operand(
-            Calc.class,
-            operand(Calc.class, any())),
-        relBuilderFactory, null);
+    this(Config.DEFAULT.withRelBuilderFactory(relBuilderFactory)
+        .as(Config.class));
   }
 
   //~ Methods ----------------------------------------------------------------
 
-  public void onMatch(RelOptRuleCall call) {
+  @Override public void onMatch(RelOptRuleCall call) {
     final Calc topCalc = call.rel(0);
     final Calc bottomCalc = call.rel(1);
 
@@ -95,4 +88,17 @@ public class CalcMergeRule extends RelOptRule implements TransformationRule {
 
     call.transformTo(newCalc);
   }
+
+  /** Rule configuration. */
+  public interface Config extends RelRule.Config {
+    Config DEFAULT = Config.EMPTY
+        .withOperandSupplier(b0 ->
+            b0.operand(Calc.class).oneInput(b1 ->
+                b1.operand(Calc.class).anyInputs()))
+        .as(Config.class);
+
+    @Override default CalcMergeRule toRule() {
+      return new CalcMergeRule(this);
+    }
+  }
 }
diff --git a/core/src/main/java/org/apache/calcite/rel/rules/CalcRemoveRule.java b/core/src/main/java/org/apache/calcite/rel/rules/CalcRemoveRule.java
index 320d364..737be2b 100644
--- a/core/src/main/java/org/apache/calcite/rel/rules/CalcRemoveRule.java
+++ b/core/src/main/java/org/apache/calcite/rel/rules/CalcRemoveRule.java
@@ -16,11 +16,11 @@
  */
 package org.apache.calcite.rel.rules;
 
-import org.apache.calcite.plan.RelOptRule;
 import org.apache.calcite.plan.RelOptRuleCall;
+import org.apache.calcite.plan.RelRule;
 import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rel.core.Calc;
 import org.apache.calcite.rel.logical.LogicalCalc;
-import org.apache.calcite.rex.RexProgram;
 import org.apache.calcite.tools.RelBuilderFactory;
 
 /**
@@ -33,33 +33,25 @@ import org.apache.calcite.tools.RelBuilderFactory;
  *
  * @see ProjectRemoveRule
  */
-public class CalcRemoveRule extends RelOptRule implements SubstitutionRule {
-  //~ Static fields/initializers ---------------------------------------------
+public class CalcRemoveRule extends RelRule<CalcRemoveRule.Config>
+    implements SubstitutionRule {
 
-  /** @deprecated Use {@link CoreRules#CALC_REMOVE}. */
-  @Deprecated // to be removed before 1.25
-  public static final CalcRemoveRule INSTANCE =
-      CoreRules.CALC_REMOVE;
-
-  //~ Constructors -----------------------------------------------------------
+  /** Creates a CalcRemoveRule. */
+  protected CalcRemoveRule(Config config) {
+    super(config);
+  }
 
-  /**
-   * Creates a CalcRemoveRule.
-   *
-   * @param relBuilderFactory Builder for relational expressions
-   */
+  @Deprecated // to be removed before 2.0
   public CalcRemoveRule(RelBuilderFactory relBuilderFactory) {
-    super(operand(LogicalCalc.class, any()), relBuilderFactory, null);
+    this(Config.DEFAULT.withRelBuilderFactory(relBuilderFactory)
+        .as(Config.class));
   }
 
   //~ Methods ----------------------------------------------------------------
 
-  public void onMatch(RelOptRuleCall call) {
-    LogicalCalc calc = call.rel(0);
-    RexProgram program = calc.getProgram();
-    if (!program.isTrivial()) {
-      return;
-    }
+  @Override public void onMatch(RelOptRuleCall call) {
+    final Calc calc = call.rel(0);
+    assert calc.getProgram().isTrivial() : "rule predicate";
     RelNode input = calc.getInput();
     input = call.getPlanner().register(input, calc);
     call.transformTo(
@@ -67,4 +59,18 @@ public class CalcRemoveRule extends RelOptRule implements SubstitutionRule {
             input,
             calc.getTraitSet()));
   }
+
+  /** Rule configuration. */
+  public interface Config extends RelRule.Config {
+    Config DEFAULT = EMPTY
+        .withOperandSupplier(b ->
+            b.operand(LogicalCalc.class)
+                .predicate(calc -> calc.getProgram().isTrivial())
+                .anyInputs())
+        .as(Config.class);
+
+    @Override default CalcRemoveRule toRule() {
+      return new CalcRemoveRule(this);
+    }
+  }
 }
diff --git a/core/src/main/java/org/apache/calcite/rel/rules/CalcSplitRule.java b/core/src/main/java/org/apache/calcite/rel/rules/CalcSplitRule.java
index 72e24c7..160af8e 100644
--- a/core/src/main/java/org/apache/calcite/rel/rules/CalcSplitRule.java
+++ b/core/src/main/java/org/apache/calcite/rel/rules/CalcSplitRule.java
@@ -16,8 +16,8 @@
  */
 package org.apache.calcite.rel.rules;
 
-import org.apache.calcite.plan.RelOptRule;
 import org.apache.calcite.plan.RelOptRuleCall;
+import org.apache.calcite.plan.RelRule;
 import org.apache.calcite.rel.core.Calc;
 import org.apache.calcite.rel.core.Filter;
 import org.apache.calcite.rex.RexNode;
@@ -36,20 +36,21 @@ import com.google.common.collect.ImmutableList;
  * convert {@code Project} and {@code Filter} to {@code Calc}. But useful for
  * specific tasks, such as optimizing before calling an
  * {@link org.apache.calcite.interpreter.Interpreter}.
+ *
+ * @see CoreRules#CALC_SPLIT
  */
-public class CalcSplitRule extends RelOptRule implements TransformationRule {
-  /** @deprecated Use {@link CoreRules#CALC_SPLIT}. */
-  @Deprecated // to be removed before 1.25
-  public static final CalcSplitRule INSTANCE =
-      CoreRules.CALC_SPLIT;
+public class CalcSplitRule extends RelRule<CalcSplitRule.Config>
+    implements TransformationRule {
+
+  /** Creates a CalcSplitRule. */
+  protected CalcSplitRule(Config config) {
+    super(config);
+  }
 
-  /**
-   * Creates a CalcSplitRule.
-   *
-   * @param relBuilderFactory Builder for relational expressions
-   */
+  @Deprecated // to be removed before 2.0
   public CalcSplitRule(RelBuilderFactory relBuilderFactory) {
-    super(operand(Calc.class, any()), relBuilderFactory, null);
+    this(Config.DEFAULT.withRelBuilderFactory(relBuilderFactory)
+        .as(Config.class));
   }
 
   @Override public void onMatch(RelOptRuleCall call) {
@@ -62,4 +63,15 @@ public class CalcSplitRule extends RelOptRule implements TransformationRule {
     relBuilder.project(projectFilter.left, calc.getRowType().getFieldNames());
     call.transformTo(relBuilder.build());
   }
+
+  /** Rule configuration. */
+  public interface Config extends RelRule.Config {
+    Config DEFAULT = EMPTY
+        .withOperandSupplier(b -> b.operand(Calc.class).anyInputs())
+        .as(Config.class);
+
+    @Override default CalcSplitRule toRule() {
+      return new CalcSplitRule(this);
+    }
+  }
 }
diff --git a/core/src/main/java/org/apache/calcite/rel/rules/CoerceInputsRule.java b/core/src/main/java/org/apache/calcite/rel/rules/CoerceInputsRule.java
index 7e477a7..ea5b5ca 100644
--- a/core/src/main/java/org/apache/calcite/rel/rules/CoerceInputsRule.java
+++ b/core/src/main/java/org/apache/calcite/rel/rules/CoerceInputsRule.java
@@ -17,13 +17,13 @@
 package org.apache.calcite.rel.rules;
 
 import org.apache.calcite.plan.Convention;
-import org.apache.calcite.plan.RelOptRule;
 import org.apache.calcite.plan.RelOptRuleCall;
 import org.apache.calcite.plan.RelOptUtil;
+import org.apache.calcite.plan.RelRule;
 import org.apache.calcite.rel.RelNode;
-import org.apache.calcite.rel.core.RelFactories;
 import org.apache.calcite.rel.type.RelDataType;
 import org.apache.calcite.tools.RelBuilderFactory;
+import org.apache.calcite.util.ImmutableBeans;
 
 import java.util.ArrayList;
 import java.util.List;
@@ -32,39 +32,36 @@ import java.util.List;
  * CoerceInputsRule pre-casts inputs to a particular type. This can be used to
  * assist operator implementations which impose requirements on their input
  * types.
+ *
+ * @see CoreRules#COERCE_INPUTS
  */
-public class CoerceInputsRule extends RelOptRule implements TransformationRule {
-  //~ Instance fields --------------------------------------------------------
-
-  private final Class consumerRelClass;
-
-  private final boolean coerceNames;
-
+public class CoerceInputsRule
+    extends RelRule<CoerceInputsRule.Config>
+    implements TransformationRule {
   //~ Constructors -----------------------------------------------------------
 
+  /** Creates a CoerceInputsRule. */
+  protected CoerceInputsRule(Config config) {
+    super(config);
+  }
+
   @Deprecated // to be removed before 2.0
   public CoerceInputsRule(
       Class<? extends RelNode> consumerRelClass,
       boolean coerceNames) {
-    this(consumerRelClass, coerceNames, RelFactories.LOGICAL_BUILDER);
+    this(Config.DEFAULT
+        .withCoerceNames(coerceNames)
+        .withOperandFor(consumerRelClass));
   }
 
-  /**
-   * Creates a CoerceInputsRule.
-   *
-   * @param consumerRelClass  Class of RelNode that will consume the inputs
-   * @param coerceNames       If true, coerce names and types; if false, coerce
-   *                          type only
-   * @param relBuilderFactory Builder for relational expressions
-   */
+  @Deprecated // to be removed before 2.0
   public CoerceInputsRule(Class<? extends RelNode> consumerRelClass,
       boolean coerceNames, RelBuilderFactory relBuilderFactory) {
-    super(
-        operand(consumerRelClass, any()),
-        relBuilderFactory,
-        "CoerceInputsRule:" + consumerRelClass.getName());
-    this.consumerRelClass = consumerRelClass;
-    this.coerceNames = coerceNames;
+    this(Config.DEFAULT
+        .withRelBuilderFactory(relBuilderFactory)
+        .as(Config.class)
+        .withCoerceNames(coerceNames)
+        .withConsumerRelClass(consumerRelClass));
   }
 
   //~ Methods ----------------------------------------------------------------
@@ -73,9 +70,9 @@ public class CoerceInputsRule extends RelOptRule implements TransformationRule {
     return Convention.NONE;
   }
 
-  public void onMatch(RelOptRuleCall call) {
+  @Override public void onMatch(RelOptRuleCall call) {
     RelNode consumerRel = call.rel(0);
-    if (consumerRel.getClass() != consumerRelClass) {
+    if (consumerRel.getClass() != config.consumerRelClass()) {
       // require exact match on type
       return;
     }
@@ -89,7 +86,7 @@ public class CoerceInputsRule extends RelOptRule implements TransformationRule {
           RelOptUtil.createCastRel(
               input,
               expectedType,
-              coerceNames);
+              config.isCoerceNames());
       if (newInput != input) {
         newInputs.set(i, newInput);
         coerce = true;
@@ -97,7 +94,7 @@ public class CoerceInputsRule extends RelOptRule implements TransformationRule {
       assert RelOptUtil.areRowTypesEqual(
           newInputs.get(i).getRowType(),
           expectedType,
-          coerceNames);
+          config.isCoerceNames());
     }
     if (!coerce) {
       return;
@@ -108,4 +105,38 @@ public class CoerceInputsRule extends RelOptRule implements TransformationRule {
             newInputs);
     call.transformTo(newConsumerRel);
   }
+
+  /** Rule configuration. */
+  public interface Config extends RelRule.Config {
+    Config DEFAULT = EMPTY.as(Config.class)
+        .withCoerceNames(false)
+        .withOperandFor(RelNode.class);
+
+    @Override default CoerceInputsRule toRule() {
+      return new CoerceInputsRule(this);
+    }
+
+    /** Whether to coerce names. */
+    @ImmutableBeans.Property
+    @ImmutableBeans.BooleanDefault(false)
+    boolean isCoerceNames();
+
+    /** Sets {@link #isCoerceNames()}. */
+    Config withCoerceNames(boolean coerceNames);
+
+    /** Class of {@link RelNode} to coerce to. */
+    @ImmutableBeans.Property
+    Class<? extends RelNode> consumerRelClass();
+
+    /** Sets {@link #consumerRelClass()}. */
+    Config withConsumerRelClass(Class<? extends RelNode> relClass);
+
+    /** Defines an operand tree for the given classes. */
+    default Config withOperandFor(Class<? extends RelNode> consumerRelClass) {
+      return withConsumerRelClass(consumerRelClass)
+          .withOperandSupplier(b -> b.operand(consumerRelClass).anyInputs())
+          .withDescription("CoerceInputsRule:" + consumerRelClass.getName())
+          .as(Config.class);
+    }
+  }
 }
diff --git a/core/src/main/java/org/apache/calcite/rel/rules/CoreRules.java b/core/src/main/java/org/apache/calcite/rel/rules/CoreRules.java
index 6052dc4..2861cd3 100644
--- a/core/src/main/java/org/apache/calcite/rel/rules/CoreRules.java
+++ b/core/src/main/java/org/apache/calcite/rel/rules/CoreRules.java
@@ -16,8 +16,6 @@
  */
 package org.apache.calcite.rel.rules;
 
-import org.apache.calcite.adapter.enumerable.EnumerableInterpreter;
-import org.apache.calcite.plan.RelOptRuleCall;
 import org.apache.calcite.rel.RelNode;
 import org.apache.calcite.rel.core.Aggregate;
 import org.apache.calcite.rel.core.Calc;
@@ -27,7 +25,6 @@ import org.apache.calcite.rel.core.Intersect;
 import org.apache.calcite.rel.core.Join;
 import org.apache.calcite.rel.core.Minus;
 import org.apache.calcite.rel.core.Project;
-import org.apache.calcite.rel.core.RelFactories;
 import org.apache.calcite.rel.core.SetOp;
 import org.apache.calcite.rel.core.Sort;
 import org.apache.calcite.rel.core.TableScan;
@@ -38,30 +35,17 @@ import org.apache.calcite.rel.logical.LogicalCalc;
 import org.apache.calcite.rel.logical.LogicalCorrelate;
 import org.apache.calcite.rel.logical.LogicalExchange;
 import org.apache.calcite.rel.logical.LogicalFilter;
-import org.apache.calcite.rel.logical.LogicalIntersect;
 import org.apache.calcite.rel.logical.LogicalJoin;
 import org.apache.calcite.rel.logical.LogicalMatch;
-import org.apache.calcite.rel.logical.LogicalMinus;
 import org.apache.calcite.rel.logical.LogicalProject;
-import org.apache.calcite.rel.logical.LogicalSort;
 import org.apache.calcite.rel.logical.LogicalSortExchange;
 import org.apache.calcite.rel.logical.LogicalTableFunctionScan;
-import org.apache.calcite.rel.logical.LogicalUnion;
-import org.apache.calcite.rel.logical.LogicalValues;
 import org.apache.calcite.rel.logical.LogicalWindow;
 import org.apache.calcite.rel.rules.materialize.MaterializedViewRules;
 import org.apache.calcite.rex.RexInputRef;
-import org.apache.calcite.rex.RexOver;
 import org.apache.calcite.rex.RexUtil;
 import org.apache.calcite.schema.impl.StarTable;
 
-import static org.apache.calcite.plan.RelOptRule.any;
-import static org.apache.calcite.plan.RelOptRule.none;
-import static org.apache.calcite.plan.RelOptRule.operand;
-import static org.apache.calcite.plan.RelOptRule.operandJ;
-import static org.apache.calcite.plan.RelOptRule.some;
-import static org.apache.calcite.rel.rules.ProjectMergeRule.DEFAULT_BLOAT;
-
 /** Rules that perform logical transformations on relational expressions.
  *
  * @see MaterializedViewRules */
@@ -73,75 +57,43 @@ public class CoreRules {
    * on top of a {@link Project} and if possible
    * aggregates through the Project or removes the Project. */
   public static final AggregateProjectMergeRule AGGREGATE_PROJECT_MERGE =
-      new AggregateProjectMergeRule(Aggregate.class, Project.class, RelFactories.LOGICAL_BUILDER);
+      AggregateProjectMergeRule.Config.DEFAULT.toRule();
 
   /** Rule that removes constant keys from an {@link Aggregate}. */
   public static final AggregateProjectPullUpConstantsRule
       AGGREGATE_PROJECT_PULL_UP_CONSTANTS =
-      new AggregateProjectPullUpConstantsRule(LogicalAggregate.class,
-          LogicalProject.class, RelFactories.LOGICAL_BUILDER,
-          "AggregateProjectPullUpConstantsRule");
+      AggregateProjectPullUpConstantsRule.Config.DEFAULT.toRule();
 
   /** More general form of {@link #AGGREGATE_PROJECT_PULL_UP_CONSTANTS}
    * that matches any relational expression. */
   public static final AggregateProjectPullUpConstantsRule
       AGGREGATE_ANY_PULL_UP_CONSTANTS =
-      new AggregateProjectPullUpConstantsRule(LogicalAggregate.class,
-          RelNode.class, RelFactories.LOGICAL_BUILDER,
-          "AggregatePullUpConstantsRule");
+      AggregateProjectPullUpConstantsRule.Config.DEFAULT
+          .withOperandFor(LogicalAggregate.class, RelNode.class)
+          .toRule();
 
   /** Rule that matches an {@link Aggregate} on
    * a {@link StarTable.StarTableScan}. */
   public static final AggregateStarTableRule AGGREGATE_STAR_TABLE =
-      new AggregateStarTableRule(
-          operandJ(Aggregate.class, null, Aggregate::isSimple,
-              some(operand(StarTable.StarTableScan.class, none()))),
-          RelFactories.LOGICAL_BUILDER,
-          "AggregateStarTableRule");
+      AggregateStarTableRule.Config.DEFAULT.toRule();
 
   /** Variant of {@link #AGGREGATE_STAR_TABLE} that accepts a {@link Project}
    * between the {@link Aggregate} and its {@link StarTable.StarTableScan}
    * input. */
-  public static final AggregateStarTableRule AGGREGATE_PROJECT_STAR_TABLE =
-      new AggregateStarTableRule(
-          operandJ(Aggregate.class, null, Aggregate::isSimple,
-              operand(Project.class,
-                  operand(StarTable.StarTableScan.class, none()))),
-          RelFactories.LOGICAL_BUILDER,
-          "AggregateStarTableRule:project") {
-        @Override public void onMatch(RelOptRuleCall call) {
-          final Aggregate aggregate = call.rel(0);
-          final Project project = call.rel(1);
-          final StarTable.StarTableScan scan = call.rel(2);
-          final RelNode rel =
-              AggregateProjectMergeRule.apply(call, aggregate, project);
-          final Aggregate aggregate2;
-          final Project project2;
-          if (rel instanceof Aggregate) {
-            project2 = null;
-            aggregate2 = (Aggregate) rel;
-          } else if (rel instanceof Project) {
-            project2 = (Project) rel;
-            aggregate2 = (Aggregate) project2.getInput();
-          } else {
-            return;
-          }
-          apply(call, project2, aggregate2, scan);
-        }
-      };
+  public static final AggregateProjectStarTableRule AGGREGATE_PROJECT_STAR_TABLE =
+      AggregateProjectStarTableRule.Config.DEFAULT.toRule();
 
   /** Rule that reduces aggregate functions in
    * an {@link Aggregate} to simpler forms. */
   public static final AggregateReduceFunctionsRule AGGREGATE_REDUCE_FUNCTIONS =
-      new AggregateReduceFunctionsRule(operand(LogicalAggregate.class, any()),
-          RelFactories.LOGICAL_BUILDER);
+      AggregateReduceFunctionsRule.Config.DEFAULT.toRule();
 
   /** Rule that matches an {@link Aggregate} on an {@link Aggregate},
    * and merges into a single Aggregate if the top aggregate's group key is a
    * subset of the lower aggregate's group key, and the aggregates are
    * expansions of rollups. */
   public static final AggregateMergeRule AGGREGATE_MERGE =
-      new AggregateMergeRule();
+      AggregateMergeRule.Config.DEFAULT.toRule();
 
   /** Rule that removes an {@link Aggregate}
    * if it computes no aggregate functions
@@ -149,8 +101,7 @@ public class CoreRules {
    * or all the aggregate functions are splittable,
    * and the underlying relational expression is already distinct. */
   public static final AggregateRemoveRule AGGREGATE_REMOVE =
-      new AggregateRemoveRule(LogicalAggregate.class,
-          RelFactories.LOGICAL_BUILDER);
+      AggregateRemoveRule.Config.DEFAULT.toRule();
 
   /** Rule that expands distinct aggregates
    * (such as {@code COUNT(DISTINCT x)}) from a
@@ -158,54 +109,47 @@ public class CoreRules {
    * This instance operates only on logical expressions. */
   public static final AggregateExpandDistinctAggregatesRule
       AGGREGATE_EXPAND_DISTINCT_AGGREGATES =
-      new AggregateExpandDistinctAggregatesRule(LogicalAggregate.class, true,
-          RelFactories.LOGICAL_BUILDER);
+      AggregateExpandDistinctAggregatesRule.Config.DEFAULT.toRule();
 
   /** As {@link #AGGREGATE_EXPAND_DISTINCT_AGGREGATES} but generates a Join. */
   public static final AggregateExpandDistinctAggregatesRule
       AGGREGATE_EXPAND_DISTINCT_AGGREGATES_TO_JOIN =
-      new AggregateExpandDistinctAggregatesRule(LogicalAggregate.class, false,
-          RelFactories.LOGICAL_BUILDER);
+      AggregateExpandDistinctAggregatesRule.Config.JOIN.toRule();
 
   /** Rule that matches an {@link Aggregate}
    * on a {@link Filter} and transposes them,
    * pushing the aggregate below the filter. */
   public static final AggregateFilterTransposeRule AGGREGATE_FILTER_TRANSPOSE =
-      new AggregateFilterTransposeRule();
+      AggregateFilterTransposeRule.Config.DEFAULT.toRule();
 
   /** Rule that matches an {@link Aggregate}
    * on a {@link Join} and removes the left input
    * of the join provided that the left input is also a left join if
    * possible. */
   public static final AggregateJoinJoinRemoveRule AGGREGATE_JOIN_JOIN_REMOVE =
-      new AggregateJoinJoinRemoveRule(LogicalAggregate.class,
-          LogicalJoin.class, RelFactories.LOGICAL_BUILDER);
+      AggregateJoinJoinRemoveRule.Config.DEFAULT.toRule();
 
   /** Rule that matches an {@link Aggregate}
    * on a {@link Join} and removes the join
    * provided that the join is a left join or right join and it computes no
    * aggregate functions or all the aggregate calls have distinct. */
   public static final AggregateJoinRemoveRule AGGREGATE_JOIN_REMOVE =
-      new AggregateJoinRemoveRule(LogicalAggregate.class, LogicalJoin.class,
-          RelFactories.LOGICAL_BUILDER);
+      AggregateJoinRemoveRule.Config.DEFAULT.toRule();
 
   /** Rule that pushes an {@link Aggregate}
    * past a {@link Join}. */
   public static final AggregateJoinTransposeRule AGGREGATE_JOIN_TRANSPOSE =
-      new AggregateJoinTransposeRule(LogicalAggregate.class, LogicalJoin.class,
-          RelFactories.LOGICAL_BUILDER, false);
+      AggregateJoinTransposeRule.Config.DEFAULT.toRule();
 
   /** As {@link #AGGREGATE_JOIN_TRANSPOSE}, but extended to push down aggregate
    * functions. */
   public static final AggregateJoinTransposeRule AGGREGATE_JOIN_TRANSPOSE_EXTENDED =
-      new AggregateJoinTransposeRule(LogicalAggregate.class, LogicalJoin.class,
-          RelFactories.LOGICAL_BUILDER, true);
+      AggregateJoinTransposeRule.Config.EXTENDED.toRule();
 
   /** Rule that pushes an {@link Aggregate}
    * past a non-distinct {@link Union}. */
   public static final AggregateUnionTransposeRule AGGREGATE_UNION_TRANSPOSE =
-      new AggregateUnionTransposeRule(LogicalAggregate.class,
-          LogicalUnion.class, RelFactories.LOGICAL_BUILDER);
+      AggregateUnionTransposeRule.Config.DEFAULT.toRule();
 
   /** Rule that matches an {@link Aggregate} whose input is a {@link Union}
    * one of whose inputs is an {@code Aggregate}.
@@ -216,95 +160,85 @@ public class CoreRules {
    * use {@link #AGGREGATE_UNION_AGGREGATE_FIRST}
    * and {@link #AGGREGATE_UNION_AGGREGATE_SECOND} instead. */
   public static final AggregateUnionAggregateRule AGGREGATE_UNION_AGGREGATE =
-      new AggregateUnionAggregateRule(LogicalAggregate.class,
-          LogicalUnion.class, RelNode.class, RelNode.class,
-          RelFactories.LOGICAL_BUILDER, "AggregateUnionAggregateRule");
+      AggregateUnionAggregateRule.Config.DEFAULT.toRule();
 
   /** As {@link #AGGREGATE_UNION_AGGREGATE}, but matches an {@code Aggregate}
    * only as the left input of the {@code Union}. */
   public static final AggregateUnionAggregateRule AGGREGATE_UNION_AGGREGATE_FIRST =
-      new AggregateUnionAggregateRule(LogicalAggregate.class, LogicalUnion.class,
-          LogicalAggregate.class, RelNode.class, RelFactories.LOGICAL_BUILDER,
-          "AggregateUnionAggregateRule:first-input-agg");
+      AggregateUnionAggregateRule.Config.AGG_FIRST.toRule();
 
   /** As {@link #AGGREGATE_UNION_AGGREGATE}, but matches an {@code Aggregate}
    * only as the right input of the {@code Union}. */
   public static final AggregateUnionAggregateRule AGGREGATE_UNION_AGGREGATE_SECOND =
-      new AggregateUnionAggregateRule(LogicalAggregate.class, LogicalUnion.class,
-          RelNode.class, LogicalAggregate.class, RelFactories.LOGICAL_BUILDER,
-          "AggregateUnionAggregateRule:second-input-agg");
+      AggregateUnionAggregateRule.Config.AGG_SECOND.toRule();
 
   /** Rule that converts CASE-style filtered aggregates into true filtered
    * aggregates. */
   public static final AggregateCaseToFilterRule AGGREGATE_CASE_TO_FILTER =
-      new AggregateCaseToFilterRule(RelFactories.LOGICAL_BUILDER, null);
+      AggregateCaseToFilterRule.Config.DEFAULT.toRule();
 
   /** Rule that merges a {@link Calc} onto a {@code Calc}. */
   public static final CalcMergeRule CALC_MERGE =
-      new CalcMergeRule(RelFactories.LOGICAL_BUILDER);
+      CalcMergeRule.Config.DEFAULT.toRule();
 
   /** Rule that removes a trivial {@link LogicalCalc}. */
   public static final CalcRemoveRule CALC_REMOVE =
-      new CalcRemoveRule(RelFactories.LOGICAL_BUILDER);
+      CalcRemoveRule.Config.DEFAULT.toRule();
 
   /** Rule that reduces operations on the DECIMAL type, such as casts or
    * arithmetic, into operations involving more primitive types such as BIGINT
    * and DOUBLE. */
   public static final ReduceDecimalsRule CALC_REDUCE_DECIMALS =
-      new ReduceDecimalsRule(RelFactories.LOGICAL_BUILDER);
+      ReduceDecimalsRule.Config.DEFAULT.toRule();
 
   /** Rule that reduces constants inside a {@link LogicalCalc}.
    *
    * @see #FILTER_REDUCE_EXPRESSIONS */
   public static final ReduceExpressionsRule.CalcReduceExpressionsRule
       CALC_REDUCE_EXPRESSIONS =
-      new ReduceExpressionsRule.CalcReduceExpressionsRule(LogicalCalc.class,
-          true, RelFactories.LOGICAL_BUILDER);
+      ReduceExpressionsRule.CalcReduceExpressionsRule.Config.DEFAULT.toRule();
 
   /** Rule that converts a {@link Calc} to a {@link Project} and
    * {@link Filter}. */
   public static final CalcSplitRule CALC_SPLIT =
-      new CalcSplitRule(RelFactories.LOGICAL_BUILDER);
+      CalcSplitRule.Config.DEFAULT.toRule();
 
   /** Rule that transforms a {@link Calc}
    * that contains windowed aggregates to a mixture of
    * {@link LogicalWindow} and {@code Calc}. */
   public static final ProjectToWindowRule.CalcToWindowRule CALC_TO_WINDOW =
-      new ProjectToWindowRule.CalcToWindowRule(RelFactories.LOGICAL_BUILDER);
+      ProjectToWindowRule.CalcToWindowRule.Config.DEFAULT.toRule();
 
   /** Rule that pre-casts inputs to a particular type. This can assist operator
    * implementations that impose requirements on their input types. */
   public static final CoerceInputsRule COERCE_INPUTS =
-      new CoerceInputsRule(RelNode.class, false, RelFactories.LOGICAL_BUILDER);
+      CoerceInputsRule.Config.DEFAULT.toRule();
 
   /** Rule that removes constants inside a {@link LogicalExchange}. */
-  @SuppressWarnings("deprecation")
   public static final ExchangeRemoveConstantKeysRule EXCHANGE_REMOVE_CONSTANT_KEYS =
-      new ExchangeRemoveConstantKeysRule(LogicalExchange.class,
-          "ExchangeRemoveConstantKeysRule");
+      ExchangeRemoveConstantKeysRule.Config.DEFAULT.toRule();
 
   /** Rule that removes constants inside a {@link LogicalSortExchange}. */
-  @SuppressWarnings("deprecation")
   public static final ExchangeRemoveConstantKeysRule SORT_EXCHANGE_REMOVE_CONSTANT_KEYS =
-      new ExchangeRemoveConstantKeysRule.SortExchangeRemoveConstantKeysRule(
-          LogicalSortExchange.class, "SortExchangeRemoveConstantKeysRule");
+      ExchangeRemoveConstantKeysRule.Config.SORT.toRule();
 
   /** Rule that tries to push filter expressions into a join
    * condition and into the inputs of the join. */
   public static final FilterJoinRule.FilterIntoJoinRule FILTER_INTO_JOIN =
-      new FilterJoinRule.FilterIntoJoinRule(true, RelFactories.LOGICAL_BUILDER,
-          (join, joinType, exp) -> true);
+      FilterJoinRule.FilterIntoJoinRule.Config.DEFAULT.toRule();
 
   /** Dumber version of {@link #FILTER_INTO_JOIN}. Not intended for production
    * use, but keeps some tests working for which {@code FILTER_INTO_JOIN} is too
    * smart. */
   public static final FilterJoinRule.FilterIntoJoinRule FILTER_INTO_JOIN_DUMB =
-      new FilterJoinRule.FilterIntoJoinRule(false, RelFactories.LOGICAL_BUILDER,
-          (join, joinType, exp) -> true);
+      FILTER_INTO_JOIN.config
+          .withSmart(false)
+          .as(FilterJoinRule.FilterIntoJoinRule.Config.class)
+          .toRule();
 
   /** Rule that combines two {@link LogicalFilter}s. */
   public static final FilterMergeRule FILTER_MERGE =
-      new FilterMergeRule(RelFactories.LOGICAL_BUILDER);
+      FilterMergeRule.Config.DEFAULT.toRule();
 
   /** Rule that merges a {@link Filter} and a {@link LogicalCalc}. The
    * result is a {@link LogicalCalc} whose filter condition is the logical AND
@@ -312,20 +246,19 @@ public class CoreRules {
    *
    * @see #PROJECT_CALC_MERGE */
   public static final FilterCalcMergeRule FILTER_CALC_MERGE =
-      new FilterCalcMergeRule(RelFactories.LOGICAL_BUILDER);
+      FilterCalcMergeRule.Config.DEFAULT.toRule();
 
   /** Rule that converts a {@link LogicalFilter} to a {@link LogicalCalc}.
    *
    * @see #PROJECT_TO_CALC */
   public static final FilterToCalcRule FILTER_TO_CALC =
-      new FilterToCalcRule(RelFactories.LOGICAL_BUILDER);
+      FilterToCalcRule.Config.DEFAULT.toRule();
 
   /** Rule that pushes a {@link Filter} past an {@link Aggregate}.
    *
    * @see #AGGREGATE_FILTER_TRANSPOSE */
   public static final FilterAggregateTransposeRule FILTER_AGGREGATE_TRANSPOSE =
-      new FilterAggregateTransposeRule(Filter.class,
-          RelFactories.LOGICAL_BUILDER, Aggregate.class);
+      FilterAggregateTransposeRule.Config.DEFAULT.toRule();
 
   /** The default instance of
    * {@link org.apache.calcite.rel.rules.FilterProjectTransposeRule}.
@@ -336,70 +269,45 @@ public class CoreRules {
    * {@link Correlate} from being de-correlated.
    */
   public static final FilterProjectTransposeRule FILTER_PROJECT_TRANSPOSE =
-      new FilterProjectTransposeRule(Filter.class, Project.class, true, true,
-          RelFactories.LOGICAL_BUILDER);
+      FilterProjectTransposeRule.Config.DEFAULT.toRule();
 
   /** Rule that pushes a {@link LogicalFilter}
    * past a {@link LogicalTableFunctionScan}. */
   public static final FilterTableFunctionTransposeRule
       FILTER_TABLE_FUNCTION_TRANSPOSE =
-      new FilterTableFunctionTransposeRule(RelFactories.LOGICAL_BUILDER);
+      FilterTableFunctionTransposeRule.Config.DEFAULT.toRule();
 
   /** Rule that matches a {@link Filter} on a {@link TableScan}. */
   public static final FilterTableScanRule FILTER_SCAN =
-      new FilterTableScanRule(
-          operand(Filter.class,
-              operandJ(TableScan.class, null, FilterTableScanRule::test,
-                  none())),
-          RelFactories.LOGICAL_BUILDER,
-          "FilterTableScanRule") {
-        public void onMatch(RelOptRuleCall call) {
-          final Filter filter = call.rel(0);
-          final TableScan scan = call.rel(1);
-          apply(call, filter, scan);
-        }
-      };
+      FilterTableScanRule.Config.DEFAULT.toRule();
 
   /** Rule that matches a {@link Filter} on an
    * {@link org.apache.calcite.adapter.enumerable.EnumerableInterpreter} on a
    * {@link TableScan}. */
   public static final FilterTableScanRule FILTER_INTERPRETER_SCAN =
-      new FilterTableScanRule(
-          operand(Filter.class,
-              operand(
-                  EnumerableInterpreter.class,
-                  operandJ(TableScan.class, null, FilterTableScanRule::test,
-                      none()))),
-          RelFactories.LOGICAL_BUILDER,
-          "FilterTableScanRule:interpreter") {
-        public void onMatch(RelOptRuleCall call) {
-          final Filter filter = call.rel(0);
-          final TableScan scan = call.rel(2);
-          apply(call, filter, scan);
-        }
-      };
+      FilterTableScanRule.Config.INTERPRETER.toRule();
 
   /** Rule that pushes a {@link Filter} above a {@link Correlate} into the
    * inputs of the {@code Correlate}. */
   public static final FilterCorrelateRule FILTER_CORRELATE =
-      new FilterCorrelateRule(RelFactories.LOGICAL_BUILDER);
+      FilterCorrelateRule.Config.DEFAULT.toRule();
 
   /** Rule that merges a {@link Filter} into a {@link MultiJoin},
    * creating a richer {@code MultiJoin}.
    *
    * @see #PROJECT_MULTI_JOIN_MERGE */
   public static final FilterMultiJoinMergeRule FILTER_MULTI_JOIN_MERGE =
-      new FilterMultiJoinMergeRule(RelFactories.LOGICAL_BUILDER);
+      FilterMultiJoinMergeRule.Config.DEFAULT.toRule();
 
   /** Rule that replaces {@code IS NOT DISTINCT FROM}
    * in a {@link Filter} with logically equivalent operations. */
   public static final FilterRemoveIsNotDistinctFromRule
       FILTER_EXPAND_IS_NOT_DISTINCT_FROM =
-      new FilterRemoveIsNotDistinctFromRule(RelFactories.LOGICAL_BUILDER);
+      FilterRemoveIsNotDistinctFromRule.Config.DEFAULT.toRule();
 
   /** Rule that pushes a {@link Filter} past a {@link SetOp}. */
   public static final FilterSetOpTransposeRule FILTER_SET_OP_TRANSPOSE =
-      new FilterSetOpTransposeRule(RelFactories.LOGICAL_BUILDER);
+      FilterSetOpTransposeRule.Config.DEFAULT.toRule();
 
   /** Rule that reduces constants inside a {@link LogicalFilter}.
    *
@@ -410,73 +318,65 @@ public class CoreRules {
    */
   public static final ReduceExpressionsRule.FilterReduceExpressionsRule
       FILTER_REDUCE_EXPRESSIONS =
-      new ReduceExpressionsRule.FilterReduceExpressionsRule(LogicalFilter.class,
-          false, RelFactories.LOGICAL_BUILDER);
+      ReduceExpressionsRule.FilterReduceExpressionsRule.Config.DEFAULT.toRule();
 
   /** Rule that flattens an {@link Intersect} on an {@code Intersect}
    * into a single {@code Intersect}. */
   public static final UnionMergeRule INTERSECT_MERGE =
-      new UnionMergeRule(LogicalIntersect.class, "IntersectMergeRule",
-          RelFactories.LOGICAL_BUILDER);
+      UnionMergeRule.Config.INTERSECT.toRule();
 
   /** Rule that translates a distinct
    * {@link Intersect} into a group of operators
    * composed of {@link Union}, {@link Aggregate}, etc. */
   public static final IntersectToDistinctRule INTERSECT_TO_DISTINCT =
-      new IntersectToDistinctRule(LogicalIntersect.class, RelFactories.LOGICAL_BUILDER);
+      IntersectToDistinctRule.Config.DEFAULT.toRule();
 
   /** Rule that converts a {@link LogicalMatch} to the result of calling
    * {@link LogicalMatch#copy}. */
-  public static final MatchRule MATCH = new MatchRule();
+  public static final MatchRule MATCH = MatchRule.Config.DEFAULT.toRule();
 
   /** Rule that flattens a {@link Minus} on a {@code Minus}
    * into a single {@code Minus}. */
   public static final UnionMergeRule MINUS_MERGE =
-      new UnionMergeRule(LogicalMinus.class, "MinusMergeRule",
-          RelFactories.LOGICAL_BUILDER);
+      UnionMergeRule.Config.MINUS.toRule();
 
   /** Rule that merges a {@link LogicalProject} and a {@link LogicalCalc}.
    *
    * @see #FILTER_CALC_MERGE */
   public static final ProjectCalcMergeRule PROJECT_CALC_MERGE =
-      new ProjectCalcMergeRule(RelFactories.LOGICAL_BUILDER);
+      ProjectCalcMergeRule.Config.DEFAULT.toRule();
 
   /** Rule that matches a {@link Project} on a {@link Correlate} and
    * pushes the projections to the Correlate's left and right inputs. */
   public static final ProjectCorrelateTransposeRule PROJECT_CORRELATE_TRANSPOSE =
-      new ProjectCorrelateTransposeRule(expr -> !(expr instanceof RexOver),
-          RelFactories.LOGICAL_BUILDER);
+      ProjectCorrelateTransposeRule.Config.DEFAULT.toRule();
 
   /** Rule that pushes a {@link Project} past a {@link Filter}.
    *
    * @see #PROJECT_FILTER_TRANSPOSE_WHOLE_PROJECT_EXPRESSIONS
    * @see #PROJECT_FILTER_TRANSPOSE_WHOLE_EXPRESSIONS */
   public static final ProjectFilterTransposeRule PROJECT_FILTER_TRANSPOSE =
-      new ProjectFilterTransposeRule(LogicalProject.class, LogicalFilter.class,
-          RelFactories.LOGICAL_BUILDER, expr -> false, false, false);
+      ProjectFilterTransposeRule.Config.DEFAULT.toRule();
 
   /** As {@link #PROJECT_FILTER_TRANSPOSE}, but pushes down project and filter
    * expressions whole, not field references. */
   public static final ProjectFilterTransposeRule
       PROJECT_FILTER_TRANSPOSE_WHOLE_EXPRESSIONS =
-      new ProjectFilterTransposeRule(LogicalProject.class, LogicalFilter.class,
-          RelFactories.LOGICAL_BUILDER, expr -> false, true, true);
+      ProjectFilterTransposeRule.Config.PROJECT_FILTER.toRule();
 
   /** As {@link #PROJECT_FILTER_TRANSPOSE},
    * pushes down field references for filters,
    * but pushes down project expressions whole. */
   public static final ProjectFilterTransposeRule
       PROJECT_FILTER_TRANSPOSE_WHOLE_PROJECT_EXPRESSIONS =
-      new ProjectFilterTransposeRule(LogicalProject.class, LogicalFilter.class,
-          RelFactories.LOGICAL_BUILDER, expr -> false, true, false);
+      ProjectFilterTransposeRule.Config.PROJECT.toRule();
 
   /** Rule that reduces constants inside a {@link LogicalProject}.
    *
    * @see #FILTER_REDUCE_EXPRESSIONS */
   public static final ReduceExpressionsRule.ProjectReduceExpressionsRule
       PROJECT_REDUCE_EXPRESSIONS =
-      new ReduceExpressionsRule.ProjectReduceExpressionsRule(
-          LogicalProject.class, true, RelFactories.LOGICAL_BUILDER);
+      ReduceExpressionsRule.ProjectReduceExpressionsRule.Config.DEFAULT.toRule();
 
   /** Rule that converts sub-queries from project expressions into
    * {@link Correlate} instances.
@@ -484,7 +384,7 @@ public class CoreRules {
    * @see #FILTER_SUB_QUERY_TO_CORRELATE
    * @see #JOIN_SUB_QUERY_TO_CORRELATE */
   public static final SubQueryRemoveRule PROJECT_SUB_QUERY_TO_CORRELATE =
-      new SubQueryRemoveRule.SubQueryProjectRemoveRule(RelFactories.LOGICAL_BUILDER);
+      SubQueryRemoveRule.Config.PROJECT.toRule();
 
   /** Rule that converts a sub-queries from filter expressions into
    * {@link Correlate} instances.
@@ -492,7 +392,7 @@ public class CoreRules {
    * @see #PROJECT_SUB_QUERY_TO_CORRELATE
    * @see #JOIN_SUB_QUERY_TO_CORRELATE */
   public static final SubQueryRemoveRule FILTER_SUB_QUERY_TO_CORRELATE =
-      new SubQueryRemoveRule.SubQueryFilterRemoveRule(RelFactories.LOGICAL_BUILDER);
+      SubQueryRemoveRule.Config.FILTER.toRule();
 
   /** Rule that converts sub-queries from join expressions into
    * {@link Correlate} instances.
@@ -500,15 +400,15 @@ public class CoreRules {
    * @see #PROJECT_SUB_QUERY_TO_CORRELATE
    * @see #FILTER_SUB_QUERY_TO_CORRELATE */
   public static final SubQueryRemoveRule JOIN_SUB_QUERY_TO_CORRELATE =
-      new SubQueryRemoveRule.SubQueryJoinRemoveRule(RelFactories.LOGICAL_BUILDER);
+      SubQueryRemoveRule.Config.JOIN.toRule();
 
   /** Rule that transforms a {@link Project}
    *  into a mixture of {@code LogicalProject}
    * and {@link LogicalWindow}. */
   public static final ProjectToWindowRule.ProjectToLogicalProjectAndWindowRule
       PROJECT_TO_LOGICAL_PROJECT_AND_WINDOW =
-      new ProjectToWindowRule.ProjectToLogicalProjectAndWindowRule(
-          RelFactories.LOGICAL_BUILDER);
+      ProjectToWindowRule.ProjectToLogicalProjectAndWindowRule.Config.DEFAULT
+          .toRule();
 
   /** Rule that creates a {@link Join#isSemiJoin semi-join} from a
    * {@link Project} on top of a {@link Join} with an {@link Aggregate} as its
@@ -516,36 +416,30 @@ public class CoreRules {
    *
    * @see #JOIN_TO_SEMI_JOIN */
   public static final SemiJoinRule.ProjectToSemiJoinRule PROJECT_TO_SEMI_JOIN =
-      new SemiJoinRule.ProjectToSemiJoinRule(Project.class, Join.class, Aggregate.class,
-          RelFactories.LOGICAL_BUILDER, "SemiJoinRule:project");
+      SemiJoinRule.ProjectToSemiJoinRule.Config.DEFAULT.toRule();
 
   /** Rule that matches an {@link Project} on a {@link Join} and removes the
    * left input of the join provided that the left input is also a left join. */
   public static final ProjectJoinJoinRemoveRule PROJECT_JOIN_JOIN_REMOVE =
-      new ProjectJoinJoinRemoveRule(LogicalProject.class,
-          LogicalJoin.class, RelFactories.LOGICAL_BUILDER);
+      ProjectJoinJoinRemoveRule.Config.DEFAULT.toRule();
 
   /** Rule that matches an {@link Project} on a {@link Join} and removes the
    * join provided that the join is a left join or right join and the join keys
    * are unique. */
   public static final ProjectJoinRemoveRule PROJECT_JOIN_REMOVE =
-      new ProjectJoinRemoveRule(LogicalProject.class,
-          LogicalJoin.class, RelFactories.LOGICAL_BUILDER);
+      ProjectJoinRemoveRule.Config.DEFAULT.toRule();
 
   /** Rule that pushes a {@link LogicalProject} past a {@link LogicalJoin}
    * by splitting the projection into a projection on top of each child of
    * the join. */
   public static final ProjectJoinTransposeRule PROJECT_JOIN_TRANSPOSE =
-      new ProjectJoinTransposeRule(
-          LogicalProject.class, LogicalJoin.class,
-          expr -> !(expr instanceof RexOver),
-          RelFactories.LOGICAL_BUILDER);
+      ProjectJoinTransposeRule.Config.DEFAULT.toRule();
 
   /** Rule that merges a {@link Project} into another {@link Project},
    * provided the projects are not projecting identical sets of input
    * references. */
   public static final ProjectMergeRule PROJECT_MERGE =
-      new ProjectMergeRule(true, DEFAULT_BLOAT, RelFactories.LOGICAL_BUILDER);
+      ProjectMergeRule.Config.DEFAULT.toRule();
 
   /** Rule that pushes a {@link LogicalProject} past a {@link SetOp}.
    *
@@ -553,20 +447,19 @@ public class CoreRules {
    * only the {@link RexInputRef}s referenced in the original
    * {@code LogicalProject}. */
   public static final ProjectSetOpTransposeRule PROJECT_SET_OP_TRANSPOSE =
-      new ProjectSetOpTransposeRule(expr -> !(expr instanceof RexOver),
-          RelFactories.LOGICAL_BUILDER);
+      ProjectSetOpTransposeRule.Config.DEFAULT.toRule();
 
   /** Rule that pushes a {@link Project} into a {@link MultiJoin},
    * creating a richer {@code MultiJoin}.
    *
    * @see #FILTER_MULTI_JOIN_MERGE */
   public static final ProjectMultiJoinMergeRule PROJECT_MULTI_JOIN_MERGE =
-      new ProjectMultiJoinMergeRule(RelFactories.LOGICAL_BUILDER);
+      ProjectMultiJoinMergeRule.Config.DEFAULT.toRule();
 
   /** Rule that, given a {@link Project} node that merely returns its input,
    *  converts the node into its input. */
   public static final ProjectRemoveRule PROJECT_REMOVE =
-      new ProjectRemoveRule(RelFactories.LOGICAL_BUILDER);
+      ProjectRemoveRule.Config.DEFAULT.toRule();
 
   /** Rule that converts a {@link Project} on a {@link TableScan}
    * of a {@link org.apache.calcite.schema.ProjectableFilterableTable}
@@ -574,148 +467,94 @@ public class CoreRules {
    *
    * @see #PROJECT_INTERPRETER_TABLE_SCAN */
   public static final ProjectTableScanRule PROJECT_TABLE_SCAN =
-      new ProjectTableScanRule(
-          operand(Project.class,
-              operandJ(TableScan.class, null, ProjectTableScanRule::test,
-                  none())),
-          RelFactories.LOGICAL_BUILDER,
-          "ProjectScanRule") {
-        @Override public void onMatch(RelOptRuleCall call) {
-          final Project project = call.rel(0);
-          final TableScan scan = call.rel(1);
-          apply(call, project, scan);
-        }
-      };
+      ProjectTableScanRule.Config.DEFAULT.toRule();
 
   /** As {@link #PROJECT_TABLE_SCAN}, but with an intervening
    * {@link org.apache.calcite.adapter.enumerable.EnumerableInterpreter}. */
   public static final ProjectTableScanRule PROJECT_INTERPRETER_TABLE_SCAN =
-      new ProjectTableScanRule(
-          operand(Project.class,
-              operand(EnumerableInterpreter.class,
-                  operandJ(TableScan.class, null, ProjectTableScanRule::test,
-                      none()))),
-          RelFactories.LOGICAL_BUILDER,
-          "ProjectScanRule:interpreter") {
-        @Override public void onMatch(RelOptRuleCall call) {
-          final Project project = call.rel(0);
-          final TableScan scan = call.rel(2);
-          apply(call, project, scan);
-        }
-      };
+      ProjectTableScanRule.Config.INTERPRETER.toRule();
 
   /** Rule that converts a {@link LogicalProject} to a {@link LogicalCalc}.
    *
    * @see #FILTER_TO_CALC */
   public static final ProjectToCalcRule PROJECT_TO_CALC =
-      new ProjectToCalcRule(RelFactories.LOGICAL_BUILDER);
+      ProjectToCalcRule.Config.DEFAULT.toRule();
 
   /** Rule that pushes a {@link LogicalProject} past a {@link LogicalWindow}. */
   public static final ProjectWindowTransposeRule PROJECT_WINDOW_TRANSPOSE =
-      new ProjectWindowTransposeRule(RelFactories.LOGICAL_BUILDER);
+      ProjectWindowTransposeRule.Config.DEFAULT.toRule();
 
   /** Rule that pushes predicates in a Join into the inputs to the join. */
   public static final FilterJoinRule.JoinConditionPushRule JOIN_CONDITION_PUSH =
-      new FilterJoinRule.JoinConditionPushRule(RelFactories.LOGICAL_BUILDER,
-          (join, joinType, exp) -> true);
+      FilterJoinRule.JoinConditionPushRule.Config.DEFAULT.toRule();
 
   /** Rule to add a semi-join into a {@link Join}. */
   public static final JoinAddRedundantSemiJoinRule JOIN_ADD_REDUNDANT_SEMI_JOIN =
-      new JoinAddRedundantSemiJoinRule(LogicalJoin.class,
-          RelFactories.LOGICAL_BUILDER);
+      JoinAddRedundantSemiJoinRule.Config.DEFAULT.toRule();
 
   /** Rule that changes a join based on the associativity rule,
    * ((a JOIN b) JOIN c) &rarr; (a JOIN (b JOIN c)). */
   public static final JoinAssociateRule JOIN_ASSOCIATE =
-      new JoinAssociateRule(RelFactories.LOGICAL_BUILDER);
+      JoinAssociateRule.Config.DEFAULT.toRule();
 
   /** Rule that permutes the inputs to an inner {@link Join}. */
   public static final JoinCommuteRule JOIN_COMMUTE =
-      new JoinCommuteRule(false);
+      JoinCommuteRule.Config.DEFAULT.toRule();
 
   /** As {@link #JOIN_COMMUTE} but swaps outer joins as well as inner joins. */
   public static final JoinCommuteRule JOIN_COMMUTE_OUTER =
-      new JoinCommuteRule(true);
+      JoinCommuteRule.Config.DEFAULT.withSwapOuter(true).toRule();
 
   /** Rule to convert an
    * {@link LogicalJoin inner join} to a
    * {@link LogicalFilter filter} on top of a
    * {@link LogicalJoin cartesian inner join}. */
   public static final JoinExtractFilterRule JOIN_EXTRACT_FILTER =
-      new JoinExtractFilterRule(LogicalJoin.class,
-          RelFactories.LOGICAL_BUILDER);
+      JoinExtractFilterRule.Config.DEFAULT.toRule();
 
   /** Rule that matches a {@link LogicalJoin} whose inputs are
    * {@link LogicalProject}s, and pulls the project expressions up. */
   public static final JoinProjectTransposeRule JOIN_PROJECT_BOTH_TRANSPOSE =
-      new JoinProjectTransposeRule(
-          operand(LogicalJoin.class,
-              operand(LogicalProject.class, any()),
-              operand(LogicalProject.class, any())),
-          "JoinProjectTransposeRule(Project-Project)");
+      JoinProjectTransposeRule.Config.DEFAULT.toRule();
 
   /** As {@link #JOIN_PROJECT_BOTH_TRANSPOSE} but only the left input is
    * a {@link LogicalProject}. */
   public static final JoinProjectTransposeRule JOIN_PROJECT_LEFT_TRANSPOSE =
-      new JoinProjectTransposeRule(
-          operand(LogicalJoin.class,
-              some(operand(LogicalProject.class, any()))),
-          "JoinProjectTransposeRule(Project-Other)");
+      JoinProjectTransposeRule.Config.LEFT.toRule();
 
   /** As {@link #JOIN_PROJECT_BOTH_TRANSPOSE} but only the right input is
    * a {@link LogicalProject}. */
   public static final JoinProjectTransposeRule JOIN_PROJECT_RIGHT_TRANSPOSE =
-      new JoinProjectTransposeRule(
-          operand(
-              LogicalJoin.class,
-              operand(RelNode.class, any()),
-              operand(LogicalProject.class, any())),
-          "JoinProjectTransposeRule(Other-Project)");
+      JoinProjectTransposeRule.Config.RIGHT.toRule();
 
   /** As {@link #JOIN_PROJECT_BOTH_TRANSPOSE} but match outer as well as
    * inner join. */
   public static final JoinProjectTransposeRule
       JOIN_PROJECT_BOTH_TRANSPOSE_INCLUDE_OUTER =
-      new JoinProjectTransposeRule(
-          operand(LogicalJoin.class,
-              operand(LogicalProject.class, any()),
-              operand(LogicalProject.class, any())),
-          "Join(IncludingOuter)ProjectTransposeRule(Project-Project)",
-          true, RelFactories.LOGICAL_BUILDER);
+      JoinProjectTransposeRule.Config.OUTER.toRule();
 
   /** As {@link #JOIN_PROJECT_LEFT_TRANSPOSE} but match outer as well as
    * inner join. */
   public static final JoinProjectTransposeRule
       JOIN_PROJECT_LEFT_TRANSPOSE_INCLUDE_OUTER =
-      new JoinProjectTransposeRule(
-          operand(LogicalJoin.class,
-              some(operand(LogicalProject.class, any()))),
-          "Join(IncludingOuter)ProjectTransposeRule(Project-Other)",
-          true, RelFactories.LOGICAL_BUILDER);
+      JoinProjectTransposeRule.Config.LEFT_OUTER.toRule();
 
   /** As {@link #JOIN_PROJECT_RIGHT_TRANSPOSE} but match outer as well as
    * inner join. */
   public static final JoinProjectTransposeRule
       JOIN_PROJECT_RIGHT_TRANSPOSE_INCLUDE_OUTER =
-      new JoinProjectTransposeRule(
-          operand(
-              LogicalJoin.class,
-              operand(RelNode.class, any()),
-              operand(LogicalProject.class, any())),
-          "Join(IncludingOuter)ProjectTransposeRule(Other-Project)",
-          true, RelFactories.LOGICAL_BUILDER);
+      JoinProjectTransposeRule.Config.RIGHT_OUTER.toRule();
 
   /** Rule that matches a {@link Join} and pushes down expressions on either
    * side of "equal" conditions. */
   public static final JoinPushExpressionsRule JOIN_PUSH_EXPRESSIONS =
-      new JoinPushExpressionsRule(Join.class, RelFactories.LOGICAL_BUILDER);
+      JoinPushExpressionsRule.Config.DEFAULT.toRule();
 
   /** Rule that infers predicates from on a {@link Join} and creates
    * {@link Filter}s if those predicates can be pushed to its inputs. */
   public static final JoinPushTransitivePredicatesRule
       JOIN_PUSH_TRANSITIVE_PREDICATES =
-      new JoinPushTransitivePredicatesRule(Join.class,
-          RelFactories.LOGICAL_BUILDER);
+      JoinPushTransitivePredicatesRule.Config.DEFAULT.toRule();
 
   /** Rule that reduces constants inside a {@link Join}.
    *
@@ -723,47 +562,34 @@ public class CoreRules {
    * @see #PROJECT_REDUCE_EXPRESSIONS */
   public static final ReduceExpressionsRule.JoinReduceExpressionsRule
       JOIN_REDUCE_EXPRESSIONS =
-      new ReduceExpressionsRule.JoinReduceExpressionsRule(Join.class, false,
-          RelFactories.LOGICAL_BUILDER);
+      ReduceExpressionsRule.JoinReduceExpressionsRule.Config.DEFAULT.toRule();
 
   /** Rule that converts a {@link LogicalJoin}
    * into a {@link LogicalCorrelate}. */
   public static final JoinToCorrelateRule JOIN_TO_CORRELATE =
-      new JoinToCorrelateRule(LogicalJoin.class, RelFactories.LOGICAL_BUILDER,
-          "JoinToCorrelateRule");
+      JoinToCorrelateRule.Config.DEFAULT.toRule();
 
   /** Rule that flattens a tree of {@link LogicalJoin}s
    * into a single {@link MultiJoin} with N inputs. */
   public static final JoinToMultiJoinRule JOIN_TO_MULTI_JOIN =
-      new JoinToMultiJoinRule(LogicalJoin.class, RelFactories.LOGICAL_BUILDER);
+      JoinToMultiJoinRule.Config.DEFAULT.toRule();
 
   /** Rule that creates a {@link Join#isSemiJoin semi-join} from a
    * {@link Join} with an empty {@link Aggregate} as its right input.
    *
    * @see #PROJECT_TO_SEMI_JOIN */
   public static final SemiJoinRule.JoinToSemiJoinRule JOIN_TO_SEMI_JOIN =
-      new SemiJoinRule.JoinToSemiJoinRule(Join.class, Aggregate.class,
-          RelFactories.LOGICAL_BUILDER, "SemiJoinRule:join");
+      SemiJoinRule.JoinToSemiJoinRule.Config.DEFAULT.toRule();
 
   /** Rule that pushes a {@link Join}
    * past a non-distinct {@link Union} as its left input. */
   public static final JoinUnionTransposeRule JOIN_LEFT_UNION_TRANSPOSE =
-      new JoinUnionTransposeRule(
-          operand(Join.class,
-              operand(Union.class, any()),
-              operand(RelNode.class, any())),
-          RelFactories.LOGICAL_BUILDER,
-          "JoinUnionTransposeRule(Union-Other)");
+      JoinUnionTransposeRule.Config.LEFT.toRule();
 
   /** Rule that pushes a {@link Join}
    * past a non-distinct {@link Union} as its right input. */
   public static final JoinUnionTransposeRule JOIN_RIGHT_UNION_TRANSPOSE =
-      new JoinUnionTransposeRule(
-          operand(Join.class,
-              operand(RelNode.class, any()),
-              operand(Union.class, any())),
-          RelFactories.LOGICAL_BUILDER,
-          "JoinUnionTransposeRule(Other-Union)");
+      JoinUnionTransposeRule.Config.RIGHT.toRule();
 
   /** Rule that re-orders a {@link Join} using a heuristic planner.
    *
@@ -773,7 +599,7 @@ public class CoreRules {
    * @see #JOIN_TO_MULTI_JOIN
    * @see #MULTI_JOIN_OPTIMIZE_BUSHY */
   public static final LoptOptimizeJoinRule MULTI_JOIN_OPTIMIZE =
-      new LoptOptimizeJoinRule(RelFactories.LOGICAL_BUILDER);
+      LoptOptimizeJoinRule.Config.DEFAULT.toRule();
 
   /** Rule that finds an approximately optimal ordering for join operators
    * using a heuristic algorithm and can handle bushy joins.
@@ -784,7 +610,23 @@ public class CoreRules {
    * @see #MULTI_JOIN_OPTIMIZE
    */
   public static final MultiJoinOptimizeBushyRule MULTI_JOIN_OPTIMIZE_BUSHY =
-      new MultiJoinOptimizeBushyRule(RelFactories.LOGICAL_BUILDER);
+      MultiJoinOptimizeBushyRule.Config.DEFAULT.toRule();
+
+  /** Rule that matches a {@link LogicalJoin} whose inputs are both a
+   * {@link MultiJoin} with intervening {@link LogicalProject}s,
+   * and pulls the Projects up above the Join. */
+  public static final MultiJoinProjectTransposeRule MULTI_JOIN_BOTH_PROJECT =
+      MultiJoinProjectTransposeRule.Config.BOTH_PROJECT.toRule();
+
+  /** As {@link #MULTI_JOIN_BOTH_PROJECT} but only the left input has
+   * an intervening Project. */
+  public static final MultiJoinProjectTransposeRule MULTI_JOIN_LEFT_PROJECT =
+      MultiJoinProjectTransposeRule.Config.LEFT_PROJECT.toRule();
+
+  /** As {@link #MULTI_JOIN_BOTH_PROJECT} but only the right input has
+   * an intervening Project. */
+  public static final MultiJoinProjectTransposeRule MULTI_JOIN_RIGHT_PROJECT =
+      MultiJoinProjectTransposeRule.Config.RIGHT_PROJECT.toRule();
 
   /** Rule that pushes a {@link Join#isSemiJoin semi-join} down in a tree past
    * a {@link Filter}.
@@ -792,16 +634,15 @@ public class CoreRules {
    * @see #SEMI_JOIN_PROJECT_TRANSPOSE
    * @see #SEMI_JOIN_JOIN_TRANSPOSE */
   public static final SemiJoinFilterTransposeRule SEMI_JOIN_FILTER_TRANSPOSE =
-      new SemiJoinFilterTransposeRule(RelFactories.LOGICAL_BUILDER);
+      SemiJoinFilterTransposeRule.Config.DEFAULT.toRule();
 
   /** Rule that pushes a {@link Join#isSemiJoin semi-join} down in a tree past
    * a {@link Project}.
    *
    * @see #SEMI_JOIN_FILTER_TRANSPOSE
    * @see #SEMI_JOIN_JOIN_TRANSPOSE */
-  @SuppressWarnings("deprecation")
   public static final SemiJoinProjectTransposeRule SEMI_JOIN_PROJECT_TRANSPOSE =
-      new SemiJoinProjectTransposeRule(RelFactories.LOGICAL_BUILDER);
+      SemiJoinProjectTransposeRule.Config.DEFAULT.toRule();
 
   /** Rule that pushes a {@link Join#isSemiJoin semi-join} down in a tree past a
    * {@link Join}.
@@ -809,11 +650,11 @@ public class CoreRules {
    * @see #SEMI_JOIN_FILTER_TRANSPOSE
    * @see #SEMI_JOIN_PROJECT_TRANSPOSE */
   public static final SemiJoinJoinTransposeRule SEMI_JOIN_JOIN_TRANSPOSE =
-      new SemiJoinJoinTransposeRule(RelFactories.LOGICAL_BUILDER);
+      SemiJoinJoinTransposeRule.Config.DEFAULT.toRule();
 
   /** Rule that removes a {@link Join#isSemiJoin semi-join} from a join tree. */
   public static final SemiJoinRemoveRule SEMI_JOIN_REMOVE =
-      new SemiJoinRemoveRule(RelFactories.LOGICAL_BUILDER);
+      SemiJoinRemoveRule.Config.DEFAULT.toRule();
 
   /** Rule that pushes a {@link Sort} past a {@link Union}.
    *
@@ -822,105 +663,78 @@ public class CoreRules {
    * if the Sort does not have a limit, i.e., {@link Sort#fetch} is null.
    *
    * @see #SORT_UNION_TRANSPOSE_MATCH_NULL_FETCH */
-  @SuppressWarnings("deprecation")
   public static final SortUnionTransposeRule SORT_UNION_TRANSPOSE =
-      new SortUnionTransposeRule(false);
+      SortUnionTransposeRule.Config.DEFAULT.toRule();
 
   /** As {@link #SORT_UNION_TRANSPOSE}, but for a Union implementation that
    * preserves the ordering of its inputs. It is still worth applying this rule
    * even if the Sort does not have a limit, for the merge of already sorted
    * inputs that the Union can do is usually cheap. */
-  @SuppressWarnings("deprecation")
   public static final SortUnionTransposeRule SORT_UNION_TRANSPOSE_MATCH_NULL_FETCH =
-      new SortUnionTransposeRule(true);
+      SortUnionTransposeRule.Config.DEFAULT.withMatchNullFetch(true).toRule();
 
   /** Rule that copies a {@link Sort} past a {@link Join} without its limit and
    * offset. The original {@link Sort} is preserved but can potentially be
    * removed by {@link #SORT_REMOVE} if redundant. */
   public static final SortJoinCopyRule SORT_JOIN_COPY =
-      new SortJoinCopyRule(LogicalSort.class,
-          LogicalJoin.class, RelFactories.LOGICAL_BUILDER);
+      SortJoinCopyRule.Config.DEFAULT.toRule();
 
   /** Rule that removes a {@link Sort} if its input is already sorted. */
   public static final SortRemoveRule SORT_REMOVE =
-      new SortRemoveRule(RelFactories.LOGICAL_BUILDER);
+      SortRemoveRule.Config.DEFAULT.toRule();
 
   /** Rule that removes keys from a {@link Sort}
    * if those keys are known to be constant, or removes the entire Sort if all
    * keys are constant. */
-  @SuppressWarnings("deprecation")
   public static final SortRemoveConstantKeysRule SORT_REMOVE_CONSTANT_KEYS =
-      new SortRemoveConstantKeysRule();
+      SortRemoveConstantKeysRule.Config.DEFAULT.toRule();
 
   /** Rule that pushes a {@link Sort} past a {@link Join}. */
   public static final SortJoinTransposeRule SORT_JOIN_TRANSPOSE =
-      new SortJoinTransposeRule(LogicalSort.class,
-          LogicalJoin.class, RelFactories.LOGICAL_BUILDER);
+      SortJoinTransposeRule.Config.DEFAULT.toRule();
 
   /** Rule that pushes a {@link Sort} past a {@link Project}. */
   public static final SortProjectTransposeRule SORT_PROJECT_TRANSPOSE =
-      new SortProjectTransposeRule(Sort.class, LogicalProject.class,
-          RelFactories.LOGICAL_BUILDER, null);
+      SortProjectTransposeRule.Config.DEFAULT.toRule();
 
   /** Rule that flattens a {@link Union} on a {@code Union}
    * into a single {@code Union}. */
   public static final UnionMergeRule UNION_MERGE =
-      new UnionMergeRule(LogicalUnion.class, "UnionMergeRule",
-          RelFactories.LOGICAL_BUILDER);
+      UnionMergeRule.Config.DEFAULT.toRule();
 
   /** Rule that removes a {@link Union} if it has only one input.
    *
    * @see PruneEmptyRules#UNION_INSTANCE */
   public static final UnionEliminatorRule UNION_REMOVE =
-      new UnionEliminatorRule(LogicalUnion.class, RelFactories.LOGICAL_BUILDER);
+      UnionEliminatorRule.Config.DEFAULT.toRule();
 
   /** Rule that pulls up constants through a Union operator. */
   public static final UnionPullUpConstantsRule UNION_PULL_UP_CONSTANTS =
-      new UnionPullUpConstantsRule(Union.class, RelFactories.LOGICAL_BUILDER);
+       UnionPullUpConstantsRule.Config.DEFAULT.toRule();
 
   /** Rule that translates a distinct {@link Union}
    * (<code>all</code> = <code>false</code>)
    * into an {@link Aggregate} on top of a non-distinct {@link Union}
    * (<code>all</code> = <code>true</code>). */
   public static final UnionToDistinctRule UNION_TO_DISTINCT =
-      new UnionToDistinctRule(LogicalUnion.class, RelFactories.LOGICAL_BUILDER);
+      UnionToDistinctRule.Config.DEFAULT.toRule();
 
   /** Rule that applies an {@link Aggregate} to a {@link Values} (currently just
    * an empty {@code Values}). */
   public static final AggregateValuesRule AGGREGATE_VALUES =
-      new AggregateValuesRule(RelFactories.LOGICAL_BUILDER);
+      AggregateValuesRule.Config.DEFAULT.toRule();
 
   /** Rule that merges a {@link Filter} onto an underlying
    * {@link org.apache.calcite.rel.logical.LogicalValues},
    * resulting in a {@code Values} with potentially fewer rows. */
   public static final ValuesReduceRule FILTER_VALUES_MERGE =
-      new ValuesReduceRule(
-          operand(LogicalFilter.class,
-              operandJ(LogicalValues.class, null, Values::isNotEmpty, none())),
-          RelFactories.LOGICAL_BUILDER,
-          "ValuesReduceRule(Filter)") {
-        public void onMatch(RelOptRuleCall call) {
-          LogicalFilter filter = call.rel(0);
-          LogicalValues values = call.rel(1);
-          apply(call, null, filter, values);
-        }
-      };
+      ValuesReduceRule.Config.FILTER.toRule();
 
   /** Rule that merges a {@link Project} onto an underlying
    * {@link org.apache.calcite.rel.logical.LogicalValues},
    * resulting in a {@code Values} with different columns. */
   public static final ValuesReduceRule PROJECT_VALUES_MERGE =
-      new ValuesReduceRule(
-          operand(LogicalProject.class,
-              operandJ(LogicalValues.class, null, Values::isNotEmpty, none())),
-          RelFactories.LOGICAL_BUILDER,
-          "ValuesReduceRule(Project)") {
-        public void onMatch(RelOptRuleCall call) {
-          LogicalProject project = call.rel(0);
-          LogicalValues values = call.rel(1);
-          apply(call, project, null, values);
-        }
-      };
+      ValuesReduceRule.Config.PROJECT.toRule();
 
   /** Rule that merges a {@link Project}
    * on top of a {@link Filter} onto an underlying
@@ -928,26 +742,12 @@ public class CoreRules {
    * resulting in a {@code Values} with different columns
    * and potentially fewer rows. */
   public static final ValuesReduceRule PROJECT_FILTER_VALUES_MERGE =
-      new ValuesReduceRule(
-          operand(LogicalProject.class,
-              operand(LogicalFilter.class,
-                  operandJ(LogicalValues.class, null, Values::isNotEmpty,
-                      none()))),
-          RelFactories.LOGICAL_BUILDER,
-          "ValuesReduceRule(Project-Filter)") {
-        public void onMatch(RelOptRuleCall call) {
-          LogicalProject project = call.rel(0);
-          LogicalFilter filter = call.rel(1);
-          LogicalValues values = call.rel(2);
-          apply(call, project, filter, values);
-        }
-      };
+      ValuesReduceRule.Config.PROJECT_FILTER.toRule();
 
   /** Rule that reduces constants inside a {@link LogicalWindow}.
    *
    * @see #FILTER_REDUCE_EXPRESSIONS */
   public static final ReduceExpressionsRule.WindowReduceExpressionsRule
       WINDOW_REDUCE_EXPRESSIONS =
-      new ReduceExpressionsRule.WindowReduceExpressionsRule(LogicalWindow.class,
-          true, RelFactories.LOGICAL_BUILDER);
+      ReduceExpressionsRule.WindowReduceExpressionsRule.Config.DEFAULT.toRule();
 }
diff --git a/core/src/main/java/org/apache/calcite/rel/rules/DateRangeRules.java b/core/src/main/java/org/apache/calcite/rel/rules/DateRangeRules.java
index 9e1dac4..ba2a777 100644
--- a/core/src/main/java/org/apache/calcite/rel/rules/DateRangeRules.java
+++ b/core/src/main/java/org/apache/calcite/rel/rules/DateRangeRules.java
@@ -21,8 +21,8 @@ import org.apache.calcite.avatica.util.TimeUnitRange;
 import org.apache.calcite.config.CalciteConnectionConfig;
 import org.apache.calcite.plan.RelOptRule;
 import org.apache.calcite.plan.RelOptRuleCall;
+import org.apache.calcite.plan.RelRule;
 import org.apache.calcite.rel.core.Filter;
-import org.apache.calcite.rel.core.RelFactories;
 import org.apache.calcite.rex.RexBuilder;
 import org.apache.calcite.rex.RexCall;
 import org.apache.calcite.rex.RexLiteral;
@@ -63,7 +63,6 @@ import java.util.Map;
 import java.util.Objects;
 import java.util.Set;
 import java.util.TimeZone;
-import java.util.function.Predicate;
 import javax.annotation.Nonnull;
 
 /**
@@ -90,21 +89,13 @@ public abstract class DateRangeRules {
 
   private DateRangeRules() {}
 
-  private static final Predicate<Filter> FILTER_PREDICATE =
-      filter -> {
-        try (ExtractFinder finder = ExtractFinder.THREAD_INSTANCES.get()) {
-          assert finder.timeUnits.isEmpty() && finder.opKinds.isEmpty()
-              : "previous user did not clean up";
-          filter.getCondition().accept(finder);
-          // bail out if there is no EXTRACT of YEAR, or call to FLOOR or CEIL
-          return finder.timeUnits.contains(TimeUnitRange.YEAR)
-              || finder.opKinds.contains(SqlKind.FLOOR)
-              || finder.opKinds.contains(SqlKind.CEIL);
-        }
-      };
-
+  /** Rule that matches a {@link Filter} and converts calls to {@code EXTRACT},
+   * {@code FLOOR} and {@code CEIL} functions to date ranges (typically using
+   * the {@code BETWEEN} operator). */
   public static final RelOptRule FILTER_INSTANCE =
-      new FilterDateRangeRule(RelFactories.LOGICAL_BUILDER);
+      FilterDateRangeRule.Config.DEFAULT
+          .as(FilterDateRangeRule.Config.class)
+          .toRule();
 
   private static final Map<TimeUnitRange, Integer> TIME_UNIT_CODES =
       ImmutableMap.<TimeUnitRange, Integer>builder()
@@ -167,12 +158,35 @@ public abstract class DateRangeRules {
   }
 
   /** Rule that converts EXTRACT, FLOOR and CEIL in a {@link Filter} into a date
-   * range. */
+   * range.
+   *
+   * @see DateRangeRules#FILTER_INSTANCE */
   @SuppressWarnings("WeakerAccess")
-  public static class FilterDateRangeRule extends RelOptRule implements TransformationRule {
+  public static class FilterDateRangeRule
+      extends RelRule<FilterDateRangeRule.Config>
+      implements TransformationRule {
+    /** Creates a FilterDateRangeRule. */
+    protected FilterDateRangeRule(Config config) {
+      super(config);
+    }
+
+    @Deprecated // to be removed before 2.0
     public FilterDateRangeRule(RelBuilderFactory relBuilderFactory) {
-      super(operandJ(Filter.class, null, FILTER_PREDICATE, any()),
-          relBuilderFactory, "FilterDateRangeRule");
+      this(Config.DEFAULT.withRelBuilderFactory(relBuilderFactory)
+          .as(Config.class));
+    }
+
+    /** Whether this an EXTRACT of YEAR, or a call to FLOOR or CEIL.
+     * If none of these, we cannot apply the rule. */
+    private static boolean containsRoundingExpression(Filter filter) {
+      try (ExtractFinder finder = ExtractFinder.THREAD_INSTANCES.get()) {
+        assert finder.timeUnits.isEmpty() && finder.opKinds.isEmpty()
+            : "previous user did not clean up";
+        filter.getCondition().accept(finder);
+        return finder.timeUnits.contains(TimeUnitRange.YEAR)
+            || finder.opKinds.contains(SqlKind.FLOOR)
+            || finder.opKinds.contains(SqlKind.CEIL);
+      }
     }
 
     @Override public void onMatch(RelOptRuleCall call) {
@@ -191,6 +205,20 @@ public abstract class DateRangeRules {
           .filter(condition);
       call.transformTo(relBuilder.build());
     }
+
+    /** Rule configuration. */
+    public interface Config extends RelRule.Config {
+      Config DEFAULT = EMPTY
+          .withOperandSupplier(b ->
+              b.operand(Filter.class)
+                  .predicate(FilterDateRangeRule::containsRoundingExpression)
+                  .anyInputs())
+          .as(Config.class);
+
+      @Override default FilterDateRangeRule toRule() {
+        return new FilterDateRangeRule(this);
+      }
+    }
   }
 
   /** Visitor that searches for calls to {@code EXTRACT}, {@code FLOOR} or
diff --git a/core/src/main/java/org/apache/calcite/rel/rules/ExchangeRemoveConstantKeysRule.java b/core/src/main/java/org/apache/calcite/rel/rules/ExchangeRemoveConstantKeysRule.java
index efafd85..096162f 100644
--- a/core/src/main/java/org/apache/calcite/rel/rules/ExchangeRemoveConstantKeysRule.java
+++ b/core/src/main/java/org/apache/calcite/rel/rules/ExchangeRemoveConstantKeysRule.java
@@ -19,6 +19,7 @@ package org.apache.calcite.rel.rules;
 import org.apache.calcite.plan.RelOptPredicateList;
 import org.apache.calcite.plan.RelOptRule;
 import org.apache.calcite.plan.RelOptRuleCall;
+import org.apache.calcite.plan.RelRule;
 import org.apache.calcite.rel.RelCollation;
 import org.apache.calcite.rel.RelCollations;
 import org.apache.calcite.rel.RelDistribution;
@@ -26,15 +27,18 @@ import org.apache.calcite.rel.RelDistributions;
 import org.apache.calcite.rel.RelFieldCollation;
 import org.apache.calcite.rel.RelNode;
 import org.apache.calcite.rel.core.Exchange;
-import org.apache.calcite.rel.core.RelFactories;
 import org.apache.calcite.rel.core.SortExchange;
+import org.apache.calcite.rel.logical.LogicalExchange;
+import org.apache.calcite.rel.logical.LogicalSortExchange;
 import org.apache.calcite.rel.metadata.RelMetadataQuery;
 import org.apache.calcite.rex.RexInputRef;
+import org.apache.calcite.util.ImmutableBeans;
 
 import java.util.ArrayList;
 import java.util.HashSet;
 import java.util.List;
 import java.util.Set;
+import java.util.function.Predicate;
 import java.util.stream.Collectors;
 
 /**
@@ -44,26 +48,18 @@ import java.util.stream.Collectors;
  * <p>For example,
  * <code>SELECT key,value FROM (SELECT 1 AS key, value FROM src) r DISTRIBUTE
  * BY key</code> can be reduced to
- * <code>SELECT 1 AS key, value FROM src</code>.</p>
+ * <code>SELECT 1 AS key, value FROM src</code>.
  *
+ * @see CoreRules#EXCHANGE_REMOVE_CONSTANT_KEYS
+ * @see CoreRules#SORT_EXCHANGE_REMOVE_CONSTANT_KEYS
  */
-public class ExchangeRemoveConstantKeysRule extends RelOptRule
+public class ExchangeRemoveConstantKeysRule
+    extends RelRule<ExchangeRemoveConstantKeysRule.Config>
     implements SubstitutionRule {
-  /** @deprecated Use {@link CoreRules#EXCHANGE_REMOVE_CONSTANT_KEYS}. */
-  @Deprecated // to be removed before 1.25
-  public static final ExchangeRemoveConstantKeysRule EXCHANGE_INSTANCE =
-      CoreRules.EXCHANGE_REMOVE_CONSTANT_KEYS;
-
-  /** @deprecated Use {@link CoreRules#SORT_EXCHANGE_REMOVE_CONSTANT_KEYS}. */
-  @Deprecated // to be removed before 1.25
-  public static final ExchangeRemoveConstantKeysRule SORT_EXCHANGE_INSTANCE =
-      CoreRules.SORT_EXCHANGE_REMOVE_CONSTANT_KEYS;
-
-  @SuppressWarnings("DeprecatedIsStillUsed")
-  @Deprecated // to be removed before 1.25
-  ExchangeRemoveConstantKeysRule(Class<? extends RelNode> clazz,
-      String description) {
-    super(operand(clazz, any()), RelFactories.LOGICAL_BUILDER, description);
+
+  /** Creates an ExchangeRemoveConstantKeysRule. */
+  protected ExchangeRemoveConstantKeysRule(Config config) {
+    super(config);
   }
 
   /** Removes constant in distribution keys. */
@@ -74,13 +70,12 @@ public class ExchangeRemoveConstantKeysRule extends RelOptRule
         .collect(Collectors.toList());
   }
 
-  @Override public boolean matches(RelOptRuleCall call) {
-    final Exchange exchange = call.rel(0);
-    return exchange.getDistribution().getType()
-        == RelDistribution.Type.HASH_DISTRIBUTED;
+  @Override public void onMatch(RelOptRuleCall call) {
+    config.matchHandler().accept(this, call);
   }
 
-  @Override public void onMatch(RelOptRuleCall call) {
+  private static void matchExchange(ExchangeRemoveConstantKeysRule rule,
+      RelOptRuleCall call) {
     final Exchange exchange = call.rel(0);
     final RelMetadataQuery mq = call.getMetadataQuery();
     final RelNode input = exchange.getInput();
@@ -114,83 +109,103 @@ public class ExchangeRemoveConstantKeysRule extends RelOptRule
     }
   }
 
-  /**
-   * Rule that reduces constants inside a {@link SortExchange}.
-   */
-  public static class SortExchangeRemoveConstantKeysRule
-      extends ExchangeRemoveConstantKeysRule {
-
-    @SuppressWarnings("DeprecatedIsStillUsed")
-    @Deprecated // to be removed before 1.25
-    SortExchangeRemoveConstantKeysRule(Class<? extends RelNode> clazz,
-        String description) {
-      super(clazz, description);
+  private static void matchSortExchange(ExchangeRemoveConstantKeysRule rule,
+      RelOptRuleCall call) {
+    final SortExchange sortExchange = call.rel(0);
+    final RelMetadataQuery mq = call.getMetadataQuery();
+    final RelNode input = sortExchange.getInput();
+    final RelOptPredicateList predicates = mq.getPulledUpPredicates(input);
+    if (predicates == null) {
+      return;
     }
 
-    @Override public boolean matches(RelOptRuleCall call) {
-      final SortExchange sortExchange = call.rel(0);
-      return  sortExchange.getDistribution().getType()
-          == RelDistribution.Type.HASH_DISTRIBUTED
-          || !sortExchange.getCollation().getFieldCollations().isEmpty();
+    final Set<Integer> constants = new HashSet<>();
+    predicates.constantMap.keySet().forEach(key -> {
+      if (key instanceof RexInputRef) {
+        constants.add(((RexInputRef) key).getIndex());
+      }
+    });
+
+    if (constants.isEmpty()) {
+      return;
     }
 
-    @Override public void onMatch(RelOptRuleCall call) {
-      final SortExchange sortExchange = call.rel(0);
-      final RelMetadataQuery mq = call.getMetadataQuery();
-      final RelNode input = sortExchange.getInput();
-      final RelOptPredicateList predicates = mq.getPulledUpPredicates(input);
-      if (predicates == null) {
-        return;
-      }
+    List<Integer> distributionKeys = new ArrayList<>();
+    boolean distributionSimplified = false;
+    boolean hashDistribution = sortExchange.getDistribution().getType()
+        == RelDistribution.Type.HASH_DISTRIBUTED;
+    if (hashDistribution) {
+      distributionKeys = simplifyDistributionKeys(
+          sortExchange.getDistribution(), constants);
+      distributionSimplified =
+          distributionKeys.size() != sortExchange.getDistribution().getKeys()
+              .size();
+    }
 
-      final Set<Integer> constants = new HashSet<>();
-      predicates.constantMap.keySet().forEach(key -> {
-        if (key instanceof RexInputRef) {
-          constants.add(((RexInputRef) key).getIndex());
-        }
-      });
+    final List<RelFieldCollation> fieldCollations = sortExchange
+        .getCollation().getFieldCollations().stream().filter(
+            fc -> !constants.contains(fc.getFieldIndex()))
+        .collect(Collectors.toList());
 
-      if (constants.isEmpty()) {
-        return;
-      }
+    boolean collationSimplified =
+        fieldCollations.size() != sortExchange.getCollation()
+            .getFieldCollations().size();
+    if (distributionSimplified
+        || collationSimplified) {
+      RelDistribution distribution = distributionSimplified
+          ? (distributionKeys.isEmpty()
+          ? RelDistributions.SINGLETON
+          : RelDistributions.hash(distributionKeys))
+          : sortExchange.getDistribution();
+      RelCollation collation = collationSimplified
+          ? RelCollations.of(fieldCollations)
+          : sortExchange.getCollation();
 
-      List<Integer> distributionKeys = new ArrayList<>();
-      boolean distributionSimplified = false;
-      boolean hashDistribution = sortExchange.getDistribution().getType()
-          == RelDistribution.Type.HASH_DISTRIBUTED;
-      if (hashDistribution) {
-        distributionKeys = simplifyDistributionKeys(
-            sortExchange.getDistribution(), constants);
-        distributionSimplified =
-            distributionKeys.size() != sortExchange.getDistribution().getKeys()
-                .size();
-      }
+      call.transformTo(call.builder()
+          .push(sortExchange.getInput())
+          .sortExchange(distribution, collation)
+          .build());
+      call.getPlanner().prune(sortExchange);
+    }
+  }
 
-      final List<RelFieldCollation> fieldCollations = sortExchange
-          .getCollation().getFieldCollations().stream().filter(
-              fc -> !constants.contains(fc.getFieldIndex()))
-           .collect(Collectors.toList());
-
-      boolean collationSimplified =
-           fieldCollations.size() != sortExchange.getCollation()
-               .getFieldCollations().size();
-      if (distributionSimplified
-           || collationSimplified) {
-        RelDistribution distribution = distributionSimplified
-            ? (distributionKeys.isEmpty()
-                ? RelDistributions.SINGLETON
-                : RelDistributions.hash(distributionKeys))
-            : sortExchange.getDistribution();
-        RelCollation collation = collationSimplified
-            ? RelCollations.of(fieldCollations)
-            : sortExchange.getCollation();
-
-        call.transformTo(call.builder()
-            .push(sortExchange.getInput())
-            .sortExchange(distribution, collation)
-            .build());
-        call.getPlanner().prune(sortExchange);
-      }
+  /** Rule configuration. */
+  public interface Config extends RelRule.Config {
+    Config DEFAULT = EMPTY
+        .as(Config.class)
+        .withOperandFor(LogicalExchange.class,
+            exchange -> exchange.getDistribution().getType()
+                == RelDistribution.Type.HASH_DISTRIBUTED)
+        .withMatchHandler(ExchangeRemoveConstantKeysRule::matchExchange);
+
+    Config SORT = EMPTY
+        .withDescription("SortExchangeRemoveConstantKeysRule")
+        .as(Config.class)
+        .withOperandFor(LogicalSortExchange.class,
+            sortExchange -> sortExchange.getDistribution().getType()
+                == RelDistribution.Type.HASH_DISTRIBUTED
+                || !sortExchange.getCollation().getFieldCollations()
+                .isEmpty())
+        .withMatchHandler(ExchangeRemoveConstantKeysRule::matchSortExchange);
+
+    @Override default ExchangeRemoveConstantKeysRule toRule() {
+      return new ExchangeRemoveConstantKeysRule(this);
+    }
+
+    /** Forwards a call to {@link #onMatch(RelOptRuleCall)}. */
+    @ImmutableBeans.Property
+    <R extends RelOptRule> MatchHandler<R> matchHandler();
+
+    /** Sets {@link #matchHandler()}. */
+    <R extends RelOptRule> Config withMatchHandler(MatchHandler<R> matchHandler);
+
+    /** Defines an operand tree for the given classes. */
+    default <R extends Exchange> Config withOperandFor(Class<R> exchangeClass,
+        Predicate<R> predicate) {
+      return withOperandSupplier(b ->
+          b.operand(exchangeClass).predicate(predicate)
+              .anyInputs())
+          .as(Config.class);
     }
   }
 }
diff --git a/core/src/main/java/org/apache/calcite/rel/rules/FilterAggregateTransposeRule.java b/core/src/main/java/org/apache/calcite/rel/rules/FilterAggregateTransposeRule.java
index 752a305..ca6576d 100644
--- a/core/src/main/java/org/apache/calcite/rel/rules/FilterAggregateTransposeRule.java
+++ b/core/src/main/java/org/apache/calcite/rel/rules/FilterAggregateTransposeRule.java
@@ -17,10 +17,10 @@
 package org.apache.calcite.rel.rules;
 
 import org.apache.calcite.plan.Contexts;
-import org.apache.calcite.plan.RelOptRule;
 import org.apache.calcite.plan.RelOptRuleCall;
 import org.apache.calcite.plan.RelOptRuleOperand;
 import org.apache.calcite.plan.RelOptUtil;
+import org.apache.calcite.plan.RelRule;
 import org.apache.calcite.rel.RelNode;
 import org.apache.calcite.rel.core.Aggregate;
 import org.apache.calcite.rel.core.Aggregate.Group;
@@ -43,35 +43,33 @@ import java.util.List;
  * past a {@link org.apache.calcite.rel.core.Aggregate}.
  *
  * @see org.apache.calcite.rel.rules.AggregateFilterTransposeRule
+ * @see CoreRules#FILTER_AGGREGATE_TRANSPOSE
  */
-public class FilterAggregateTransposeRule extends RelOptRule implements TransformationRule {
+public class FilterAggregateTransposeRule
+    extends RelRule<FilterAggregateTransposeRule.Config>
+    implements TransformationRule {
 
-  /** @deprecated Use {@link CoreRules#FILTER_AGGREGATE_TRANSPOSE}. */
-  @Deprecated // to be removed before 1.25
-  public static final FilterAggregateTransposeRule INSTANCE =
-      CoreRules.FILTER_AGGREGATE_TRANSPOSE;
-
-  //~ Constructors -----------------------------------------------------------
+  /** Creates a FilterAggregateTransposeRule. */
+  protected FilterAggregateTransposeRule(Config config) {
+    super(config);
+  }
 
-  /**
-   * Creates a FilterAggregateTransposeRule.
-   *
-   * <p>If {@code filterFactory} is null, creates the same kind of filter as
-   * matched in the rule. Similarly {@code aggregateFactory}.</p>
-   */
+  @Deprecated // to be removed before 2.0
   public FilterAggregateTransposeRule(
       Class<? extends Filter> filterClass,
-      RelBuilderFactory builderFactory,
+      RelBuilderFactory relBuilderFactory,
       Class<? extends Aggregate> aggregateClass) {
-    this(
-        operand(filterClass,
-            operand(aggregateClass, any())),
-        builderFactory);
+    this(Config.DEFAULT.withRelBuilderFactory(relBuilderFactory)
+        .as(Config.class)
+        .withOperandFor(filterClass, aggregateClass));
   }
 
+  @Deprecated // to be removed before 2.0
   protected FilterAggregateTransposeRule(RelOptRuleOperand operand,
-      RelBuilderFactory builderFactory) {
-    super(operand, builderFactory, null);
+      RelBuilderFactory relBuilderFactory) {
+    this(Config.DEFAULT.withRelBuilderFactory(relBuilderFactory)
+        .withOperandSupplier(b -> b.exactly(operand))
+        .as(Config.class));
   }
 
   @Deprecated // to be removed before 2.0
@@ -85,7 +83,7 @@ public class FilterAggregateTransposeRule extends RelOptRule implements Transfor
 
   //~ Methods ----------------------------------------------------------------
 
-  public void onMatch(RelOptRuleCall call) {
+  @Override public void onMatch(RelOptRuleCall call) {
     final Filter filterRel = call.rel(0);
     final Aggregate aggRel = call.rel(1);
 
@@ -147,4 +145,34 @@ public class FilterAggregateTransposeRule extends RelOptRule implements Transfor
     }
     return true;
   }
+
+  /** Rule configuration. */
+  public interface Config extends RelRule.Config {
+    Config DEFAULT = EMPTY.as(Config.class)
+        .withOperandFor(Filter.class, Aggregate.class);
+
+    @Override default FilterAggregateTransposeRule toRule() {
+      return new FilterAggregateTransposeRule(this);
+    }
+
+    /** Defines an operand tree for the given 2 classes. */
+    default Config withOperandFor(Class<? extends Filter> filterClass,
+        Class<? extends Aggregate> aggregateClass) {
+      return withOperandSupplier(b0 ->
+          b0.operand(filterClass).oneInput(b1 ->
+              b1.operand(aggregateClass).anyInputs()))
+          .as(Config.class);
+    }
+
+    /** Defines an operand tree for the given 3 classes. */
+    default Config withOperandFor(Class<? extends Filter> filterClass,
+        Class<? extends Aggregate> aggregateClass,
+        Class<? extends RelNode> relClass) {
+      return withOperandSupplier(b0 ->
+          b0.operand(filterClass).oneInput(b1 ->
+              b1.operand(aggregateClass).oneInput(b2 ->
+                  b2.operand(relClass).anyInputs())))
+          .as(Config.class);
+    }
+  }
 }
diff --git a/core/src/main/java/org/apache/calcite/rel/rules/FilterCalcMergeRule.java b/core/src/main/java/org/apache/calcite/rel/rules/FilterCalcMergeRule.java
index e958cc9..c98e327 100644
--- a/core/src/main/java/org/apache/calcite/rel/rules/FilterCalcMergeRule.java
+++ b/core/src/main/java/org/apache/calcite/rel/rules/FilterCalcMergeRule.java
@@ -16,8 +16,9 @@
  */
 package org.apache.calcite.rel.rules;
 
-import org.apache.calcite.plan.RelOptRule;
 import org.apache.calcite.plan.RelOptRuleCall;
+import org.apache.calcite.plan.RelRule;
+import org.apache.calcite.rel.core.Calc;
 import org.apache.calcite.rel.core.Filter;
 import org.apache.calcite.rel.logical.LogicalCalc;
 import org.apache.calcite.rel.logical.LogicalFilter;
@@ -34,40 +35,35 @@ import org.apache.calcite.tools.RelBuilderFactory;
  * whose filter condition is the logical AND of the two.
  *
  * @see FilterMergeRule
+ * @see ProjectCalcMergeRule
+ * @see CoreRules#FILTER_CALC_MERGE
  */
-public class FilterCalcMergeRule extends RelOptRule implements TransformationRule {
-  //~ Static fields/initializers ---------------------------------------------
+public class FilterCalcMergeRule
+    extends RelRule<FilterCalcMergeRule.Config>
+    implements TransformationRule {
 
-  /** @deprecated Use {@link CoreRules#FILTER_CALC_MERGE}. */
-  @Deprecated // to be removed before 1.25
-  public static final FilterCalcMergeRule INSTANCE =
-      CoreRules.FILTER_CALC_MERGE;
-
-  //~ Constructors -----------------------------------------------------------
+  /** Creates a FilterCalcMergeRule. */
+  protected FilterCalcMergeRule(Config config) {
+    super(config);
+  }
 
-  /**
-   * Creates a FilterCalcMergeRule.
-   *
-   * @param relBuilderFactory Builder for relational expressions
-   */
+  @Deprecated // to be removed before 2.0
   public FilterCalcMergeRule(RelBuilderFactory relBuilderFactory) {
-    super(
-        operand(
-            Filter.class,
-            operand(LogicalCalc.class, any())),
-        relBuilderFactory, null);
+    this(Config.DEFAULT
+        .withRelBuilderFactory(relBuilderFactory)
+        .as(Config.class));
   }
 
   //~ Methods ----------------------------------------------------------------
 
-  public void onMatch(RelOptRuleCall call) {
+  @Override public void onMatch(RelOptRuleCall call) {
     final LogicalFilter filter = call.rel(0);
     final LogicalCalc calc = call.rel(1);
 
     // Don't merge a filter onto a calc which contains windowed aggregates.
     // That would effectively be pushing a multiset down through a filter.
     // We'll have chance to merge later, when the over is expanded.
-    if (calc.getProgram().containsAggs()) {
+    if (calc.containsOver()) {
       return;
     }
 
@@ -92,4 +88,23 @@ public class FilterCalcMergeRule extends RelOptRule implements TransformationRul
         LogicalCalc.create(calc.getInput(), mergedProgram);
     call.transformTo(newCalc);
   }
+
+  /** Rule configuration. */
+  public interface Config extends RelRule.Config {
+    Config DEFAULT = EMPTY.as(Config.class)
+        .withOperandFor(Filter.class, LogicalCalc.class);
+
+    @Override default FilterCalcMergeRule toRule() {
+      return new FilterCalcMergeRule(this);
+    }
+
+    /** Defines an operand tree for the given classes. */
+    default Config withOperandFor(Class<? extends Filter> filterClass,
+        Class<? extends Calc> calcClass) {
+      return withOperandSupplier(b0 ->
+          b0.operand(filterClass).oneInput(b1 ->
+              b1.operand(calcClass).anyInputs()))
+          .as(Config.class);
+    }
+  }
 }
diff --git a/core/src/main/java/org/apache/calcite/rel/rules/FilterCorrelateRule.java b/core/src/main/java/org/apache/calcite/rel/rules/FilterCorrelateRule.java
index abca2be..71ed11a 100644
--- a/core/src/main/java/org/apache/calcite/rel/rules/FilterCorrelateRule.java
+++ b/core/src/main/java/org/apache/calcite/rel/rules/FilterCorrelateRule.java
@@ -16,9 +16,9 @@
  */
 package org.apache.calcite.rel.rules;
 
-import org.apache.calcite.plan.RelOptRule;
 import org.apache.calcite.plan.RelOptRuleCall;
 import org.apache.calcite.plan.RelOptUtil;
+import org.apache.calcite.plan.RelRule;
 import org.apache.calcite.rel.RelNode;
 import org.apache.calcite.rel.core.Correlate;
 import org.apache.calcite.rel.core.Filter;
@@ -37,30 +37,24 @@ import java.util.List;
 /**
  * Planner rule that pushes a {@link Filter} above a {@link Correlate} into the
  * inputs of the Correlate.
+ *
+ * @see CoreRules#FILTER_CORRELATE
  */
-public class FilterCorrelateRule extends RelOptRule implements TransformationRule {
-
-  /** @deprecated Use {@link CoreRules#FILTER_CORRELATE}. */
-  @Deprecated // to be removed before 1.25
-  public static final FilterCorrelateRule INSTANCE =
-      CoreRules.FILTER_CORRELATE;
-
-  //~ Constructors -----------------------------------------------------------
-
-  /**
-   * Creates a FilterCorrelateRule.
-   */
-  public FilterCorrelateRule(RelBuilderFactory builderFactory) {
-    super(
-        operand(Filter.class,
-            operand(Correlate.class, RelOptRule.any())),
-        builderFactory, "FilterCorrelateRule");
+public class FilterCorrelateRule
+    extends RelRule<FilterCorrelateRule.Config>
+    implements TransformationRule {
+
+  /** Creates a FilterCorrelateRule. */
+  protected FilterCorrelateRule(Config config) {
+    super(config);
+  }
+
+  @Deprecated // to be removed before 2.0
+  public FilterCorrelateRule(RelBuilderFactory relBuilderFactory) {
+    this(Config.DEFAULT.withRelBuilderFactory(relBuilderFactory)
+        .as(Config.class));
   }
 
-  /**
-   * Creates a FilterCorrelateRule with an explicit root operand and
-   * factories.
-   */
   @Deprecated // to be removed before 2.0
   public FilterCorrelateRule(RelFactories.FilterFactory filterFactory,
       RelFactories.ProjectFactory projectFactory) {
@@ -69,7 +63,7 @@ public class FilterCorrelateRule extends RelOptRule implements TransformationRul
 
   //~ Methods ----------------------------------------------------------------
 
-  public void onMatch(RelOptRuleCall call) {
+  @Override public void onMatch(RelOptRuleCall call) {
     final Filter filter = call.rel(0);
     final Correlate corr = call.rel(1);
 
@@ -129,4 +123,23 @@ public class FilterCorrelateRule extends RelOptRule implements TransformationRul
 
     call.transformTo(relBuilder.build());
   }
+
+  /** Rule configuration. */
+  public interface Config extends RelRule.Config {
+    Config DEFAULT = EMPTY.as(Config.class)
+        .withOperandFor(Filter.class, Correlate.class);
+
+    @Override default FilterCorrelateRule toRule() {
+      return new FilterCorrelateRule(this);
+    }
+
+    /** Defines an operand tree for the given classes. */
+    default Config withOperandFor(Class<? extends Filter> filterClass,
+        Class<? extends Correlate> correlateClass) {
+      return withOperandSupplier(b0 ->
+          b0.operand(filterClass).oneInput(b1 ->
+              b1.operand(correlateClass).anyInputs()))
+          .as(Config.class);
+    }
+  }
 }
diff --git a/core/src/main/java/org/apache/calcite/rel/rules/FilterJoinRule.java b/core/src/main/java/org/apache/calcite/rel/rules/FilterJoinRule.java
index 72a6dd3..77804cd 100644
--- a/core/src/main/java/org/apache/calcite/rel/rules/FilterJoinRule.java
+++ b/core/src/main/java/org/apache/calcite/rel/rules/FilterJoinRule.java
@@ -16,10 +16,9 @@
  */
 package org.apache.calcite.rel.rules;
 
-import org.apache.calcite.plan.RelOptRule;
 import org.apache.calcite.plan.RelOptRuleCall;
-import org.apache.calcite.plan.RelOptRuleOperand;
 import org.apache.calcite.plan.RelOptUtil;
+import org.apache.calcite.plan.RelRule;
 import org.apache.calcite.rel.RelNode;
 import org.apache.calcite.rel.core.Filter;
 import org.apache.calcite.rel.core.Join;
@@ -32,6 +31,7 @@ import org.apache.calcite.rex.RexNode;
 import org.apache.calcite.rex.RexUtil;
 import org.apache.calcite.tools.RelBuilder;
 import org.apache.calcite.tools.RelBuilderFactory;
+import org.apache.calcite.util.ImmutableBeans;
 
 import com.google.common.collect.ImmutableList;
 import com.google.common.collect.Sets;
@@ -39,81 +39,26 @@ import com.google.common.collect.Sets;
 import java.util.ArrayList;
 import java.util.Iterator;
 import java.util.List;
-import java.util.Objects;
 
 import static org.apache.calcite.plan.RelOptUtil.conjunctions;
 
 /**
  * Planner rule that pushes filters above and
  * within a join node into the join node and/or its children nodes.
+ *
+ * @param <C> Configuration type
  */
-public abstract class FilterJoinRule extends RelOptRule implements TransformationRule {
+public abstract class FilterJoinRule<C extends FilterJoinRule.Config>
+    extends RelRule<C>
+    implements TransformationRule {
   /** Predicate that always returns true. With this predicate, every filter
    * will be pushed into the ON clause. */
+  @Deprecated // to be removed before 2.0
   public static final Predicate TRUE_PREDICATE = (join, joinType, exp) -> true;
 
-  /** @deprecated Use {@link CoreRules#FILTER_INTO_JOIN}. */
-  @SuppressWarnings("StaticInitializerReferencesSubClass")
-  @Deprecated // to be removed before 1.25
-  public static final FilterJoinRule FILTER_ON_JOIN =
-      CoreRules.FILTER_INTO_JOIN;
-
-  /** @deprecated Use {@link CoreRules#FILTER_INTO_JOIN_DUMB}. */
-  @SuppressWarnings("StaticInitializerReferencesSubClass")
-  @Deprecated // to be removed before 1.25
-  public static final FilterJoinRule DUMB_FILTER_ON_JOIN =
-      CoreRules.FILTER_INTO_JOIN_DUMB;
-
-  /** @deprecated Use {@link CoreRules#JOIN_CONDITION_PUSH}. */
-  @SuppressWarnings("StaticInitializerReferencesSubClass")
-  @Deprecated // to be removed before 1.25
-  public static final FilterJoinRule JOIN =
-      CoreRules.JOIN_CONDITION_PUSH;
-
-  /** Whether to try to strengthen join-type. */
-  private final boolean smart;
-
-  /** Predicate that returns whether a filter is valid in the ON clause of a
-   * join for this particular kind of join. If not, Calcite will push it back to
-   * above the join. */
-  private final Predicate predicate;
-
-  //~ Constructors -----------------------------------------------------------
-
-  /**
-   * Creates a FilterJoinRule with an explicit root operand and
-   * factories.
-   */
-  protected FilterJoinRule(RelOptRuleOperand operand, String id,
-      boolean smart, RelBuilderFactory relBuilderFactory, Predicate predicate) {
-    super(operand, relBuilderFactory, "FilterJoinRule:" + id);
-    this.smart = smart;
-    this.predicate = Objects.requireNonNull(predicate);
-  }
-
-  /**
-   * Creates a FilterJoinRule with an explicit root operand and
-   * factories.
-   */
-  @Deprecated // to be removed before 2.0
-  protected FilterJoinRule(RelOptRuleOperand operand, String id,
-      boolean smart, RelFactories.FilterFactory filterFactory,
-      RelFactories.ProjectFactory projectFactory) {
-    this(operand, id, smart, RelBuilder.proto(filterFactory, projectFactory),
-        TRUE_PREDICATE);
-  }
-
-  /**
-   * Creates a FilterJoinRule with an explicit root operand and
-   * factories.
-   */
-  @Deprecated // to be removed before 2.0
-  protected FilterJoinRule(RelOptRuleOperand operand, String id,
-      boolean smart, RelFactories.FilterFactory filterFactory,
-      RelFactories.ProjectFactory projectFactory,
-      Predicate predicate) {
-    this(operand, id, smart, RelBuilder.proto(filterFactory, projectFactory),
-        predicate);
+  /** Creates a FilterJoinRule. */
+  protected FilterJoinRule(C config) {
+    super(config);
   }
 
   //~ Methods ----------------------------------------------------------------
@@ -141,7 +86,7 @@ public abstract class FilterJoinRule extends RelOptRule implements Transformatio
 
     // Simplify Outer Joins
     JoinRelType joinType = join.getJoinType();
-    if (smart
+    if (config.isSmart()
         && !origAboveFilters.isEmpty()
         && join.getJoinType() != JoinRelType.INNER) {
       joinType = RelOptUtil.simplifyJoin(join, origAboveFilters, joinType);
@@ -325,7 +270,8 @@ public abstract class FilterJoinRule extends RelOptRule implements Transformatio
     while (filterIter.hasNext()) {
       RexNode exp = filterIter.next();
       // Do not pull up filter conditions for semi/anti join.
-      if (!predicate.apply(join, joinType, exp) && joinType.projectsRight()) {
+      if (!config.getPredicate().apply(join, joinType, exp)
+          && joinType.projectsRight()) {
         aboveFilters.add(exp);
         filterIter.remove();
       }
@@ -333,12 +279,25 @@ public abstract class FilterJoinRule extends RelOptRule implements Transformatio
   }
 
   /** Rule that pushes parts of the join condition to its inputs. */
-  public static class JoinConditionPushRule extends FilterJoinRule {
+  public static class JoinConditionPushRule
+      extends FilterJoinRule<JoinConditionPushRule.Config> {
+    /** Creates a JoinConditionPushRule. */
+    protected JoinConditionPushRule(Config config) {
+      super(config);
+    }
+
+    @Deprecated // to be removed before 2.0
     public JoinConditionPushRule(RelBuilderFactory relBuilderFactory,
         Predicate predicate) {
-      super(RelOptRule.operand(Join.class, RelOptRule.any()),
-          "FilterJoinRule:no-filter", true, relBuilderFactory,
-          predicate);
+      this(Config.EMPTY
+          .withRelBuilderFactory(relBuilderFactory)
+          .withOperandSupplier(b ->
+              b.operand(Join.class).anyInputs())
+          .withDescription("FilterJoinRule:no-filter")
+          .as(Config.class)
+          .withSmart(true)
+          .withPredicate(predicate)
+          .as(Config.class));
     }
 
     @Deprecated // to be removed before 2.0
@@ -351,18 +310,47 @@ public abstract class FilterJoinRule extends RelOptRule implements Transformatio
       Join join = call.rel(0);
       perform(call, null, join);
     }
+
+    /** Rule configuration. */
+    public interface Config extends FilterJoinRule.Config {
+      Config DEFAULT = EMPTY
+          .withOperandSupplier(b ->
+              b.operand(Join.class).anyInputs())
+          .as(JoinConditionPushRule.Config.class)
+          .withSmart(true)
+          .withPredicate((join, joinType, exp) -> true)
+          .as(JoinConditionPushRule.Config.class);
+
+      @Override default JoinConditionPushRule toRule() {
+        return new JoinConditionPushRule(this);
+      }
+    }
   }
 
   /** Rule that tries to push filter expressions into a join
-   * condition and into the inputs of the join. */
-  public static class FilterIntoJoinRule extends FilterJoinRule {
+   * condition and into the inputs of the join.
+   *
+   * @see CoreRules#FILTER_INTO_JOIN */
+  public static class FilterIntoJoinRule
+      extends FilterJoinRule<FilterIntoJoinRule.Config> {
+    /** Creates a FilterIntoJoinRule. */
+    protected FilterIntoJoinRule(Config config) {
+      super(config);
+    }
+
+    @Deprecated // to be removed before 2.0
     public FilterIntoJoinRule(boolean smart,
         RelBuilderFactory relBuilderFactory, Predicate predicate) {
-      super(
-          operand(Filter.class,
-              operand(Join.class, RelOptRule.any())),
-          "FilterJoinRule:filter", smart, relBuilderFactory,
-          predicate);
+      this(Config.EMPTY
+          .withRelBuilderFactory(relBuilderFactory)
+          .withOperandSupplier(b0 ->
+              b0.operand(Filter.class).oneInput(b1 ->
+                  b1.operand(Join.class).anyInputs()))
+          .withDescription("FilterJoinRule:filter")
+          .as(Config.class)
+          .withSmart(smart)
+          .withPredicate(predicate)
+          .as(Config.class));
     }
 
     @Deprecated // to be removed before 2.0
@@ -370,7 +358,17 @@ public abstract class FilterJoinRule extends RelOptRule implements Transformatio
         RelFactories.FilterFactory filterFactory,
         RelFactories.ProjectFactory projectFactory,
         Predicate predicate) {
-      this(smart, RelBuilder.proto(filterFactory, projectFactory), predicate);
+      this(Config.EMPTY
+          .withRelBuilderFactory(
+              RelBuilder.proto(filterFactory, projectFactory))
+          .withOperandSupplier(b0 ->
+              b0.operand(Filter.class).oneInput(b1 ->
+                  b1.operand(Join.class).anyInputs()))
+          .withDescription("FilterJoinRule:filter")
+          .as(Config.class)
+          .withSmart(smart)
+          .withPredicate(predicate)
+          .as(Config.class));
     }
 
     @Override public void onMatch(RelOptRuleCall call) {
@@ -378,12 +376,49 @@ public abstract class FilterJoinRule extends RelOptRule implements Transformatio
       Join join = call.rel(1);
       perform(call, filter, join);
     }
+
+    /** Rule configuration. */
+    public interface Config extends FilterJoinRule.Config {
+      Config DEFAULT = EMPTY
+          .withOperandSupplier(b0 ->
+              b0.operand(Filter.class).oneInput(b1 ->
+                  b1.operand(Join.class).anyInputs()))
+          .as(FilterIntoJoinRule.Config.class)
+          .withSmart(true)
+          .withPredicate((join, joinType, exp) -> true)
+          .as(FilterIntoJoinRule.Config.class);
+
+      @Override default FilterIntoJoinRule toRule() {
+        return new FilterIntoJoinRule(this);
+      }
+    }
   }
 
   /** Predicate that returns whether a filter is valid in the ON clause of a
    * join for this particular kind of join. If not, Calcite will push it back to
    * above the join. */
+  @FunctionalInterface
   public interface Predicate {
     boolean apply(Join join, JoinRelType joinType, RexNode exp);
   }
+
+  /** Rule configuration. */
+  public interface Config extends RelRule.Config {
+    /** Whether to try to strengthen join-type, default false. */
+    @ImmutableBeans.Property
+    @ImmutableBeans.BooleanDefault(false)
+    boolean isSmart();
+
+    /** Sets {@link #isSmart()}. */
+    Config withSmart(boolean smart);
+
+    /** Predicate that returns whether a filter is valid in the ON clause of a
+     * join for this particular kind of join. If not, Calcite will push it back to
+     * above the join. */
+    @ImmutableBeans.Property
+    Predicate getPredicate();
+
+    /** Sets {@link #getPredicate()} ()}. */
+    Config withPredicate(Predicate predicate);
+  }
 }
diff --git a/core/src/main/java/org/apache/calcite/rel/rules/FilterMergeRule.java b/core/src/main/java/org/apache/calcite/rel/rules/FilterMergeRule.java
index d7d4c99..f25a56c 100644
--- a/core/src/main/java/org/apache/calcite/rel/rules/FilterMergeRule.java
+++ b/core/src/main/java/org/apache/calcite/rel/rules/FilterMergeRule.java
@@ -17,8 +17,8 @@
 package org.apache.calcite.rel.rules;
 
 import org.apache.calcite.plan.Contexts;
-import org.apache.calcite.plan.RelOptRule;
 import org.apache.calcite.plan.RelOptRuleCall;
+import org.apache.calcite.plan.RelRule;
 import org.apache.calcite.rel.core.Filter;
 import org.apache.calcite.rel.core.RelFactories;
 import org.apache.calcite.tools.RelBuilder;
@@ -28,22 +28,19 @@ import org.apache.calcite.tools.RelBuilderFactory;
  * Planner rule that combines two
  * {@link org.apache.calcite.rel.logical.LogicalFilter}s.
  */
-public class FilterMergeRule extends RelOptRule implements SubstitutionRule {
-  /** @deprecated Use {@link CoreRules#FILTER_MERGE}. */
-  @Deprecated // to be removed before 1.25
-  public static final FilterMergeRule INSTANCE =
-      CoreRules.FILTER_MERGE;
+public class FilterMergeRule extends RelRule<FilterMergeRule.Config>
+    implements SubstitutionRule {
 
-  //~ Constructors -----------------------------------------------------------
+  /** Creates a FilterMergeRule. */
+  protected FilterMergeRule(Config config) {
+    super(config);
+  }
 
-  /**
-   * Creates a FilterMergeRule.
-   */
+  @Deprecated // to be removed before 2.0
   public FilterMergeRule(RelBuilderFactory relBuilderFactory) {
-    super(
-        operand(Filter.class,
-            operand(Filter.class, any())),
-        relBuilderFactory, null);
+    this(Config.DEFAULT
+        .withRelBuilderFactory(relBuilderFactory)
+        .as(Config.class));
   }
 
   @Deprecated // to be removed before 2.0
@@ -53,7 +50,7 @@ public class FilterMergeRule extends RelOptRule implements SubstitutionRule {
 
   //~ Methods ----------------------------------------------------------------
 
-  public void onMatch(RelOptRuleCall call) {
+  @Override public void onMatch(RelOptRuleCall call) {
     final Filter topFilter = call.rel(0);
     final Filter bottomFilter = call.rel(1);
 
@@ -64,4 +61,21 @@ public class FilterMergeRule extends RelOptRule implements SubstitutionRule {
     call.transformTo(relBuilder.build());
   }
 
+  /** Rule configuration. */
+  public interface Config extends RelRule.Config {
+    Config DEFAULT = EMPTY.as(Config.class)
+        .withOperandFor(Filter.class);
+
+    @Override default FilterMergeRule toRule() {
+      return new FilterMergeRule(this);
+    }
+
+    /** Defines an operand tree for the given classes. */
+    default Config withOperandFor(Class<? extends Filter> filterClass) {
+      return withOperandSupplier(b0 ->
+          b0.operand(filterClass).oneInput(b1 ->
+              b1.operand(filterClass).anyInputs()))
+          .as(Config.class);
+    }
+  }
 }
diff --git a/core/src/main/java/org/apache/calcite/rel/rules/FilterMultiJoinMergeRule.java b/core/src/main/java/org/apache/calcite/rel/rules/FilterMultiJoinMergeRule.java
index f585d17..b92881e 100644
--- a/core/src/main/java/org/apache/calcite/rel/rules/FilterMultiJoinMergeRule.java
+++ b/core/src/main/java/org/apache/calcite/rel/rules/FilterMultiJoinMergeRule.java
@@ -16,10 +16,9 @@
  */
 package org.apache.calcite.rel.rules;
 
-import org.apache.calcite.plan.RelOptRule;
 import org.apache.calcite.plan.RelOptRuleCall;
+import org.apache.calcite.plan.RelRule;
 import org.apache.calcite.rel.core.Filter;
-import org.apache.calcite.rel.logical.LogicalFilter;
 import org.apache.calcite.rex.RexBuilder;
 import org.apache.calcite.rex.RexNode;
 import org.apache.calcite.rex.RexUtil;
@@ -34,41 +33,34 @@ import java.util.List;
  * creating a richer {@code MultiJoin}.
  *
  * @see org.apache.calcite.rel.rules.ProjectMultiJoinMergeRule
+ * @see CoreRules#FILTER_MULTI_JOIN_MERGE
  */
-public class FilterMultiJoinMergeRule extends RelOptRule implements TransformationRule {
-  /** @deprecated Use {@link CoreRules#FILTER_MULTI_JOIN_MERGE}. */
-  @Deprecated // to be removed before 1.25
-  public static final FilterMultiJoinMergeRule INSTANCE =
-      CoreRules.FILTER_MULTI_JOIN_MERGE;
+public class FilterMultiJoinMergeRule
+    extends RelRule<FilterMultiJoinMergeRule.Config>
+    implements TransformationRule {
 
-  //~ Constructors -----------------------------------------------------------
+  /** Creates a FilterMultiJoinMergeRule. */
+  protected FilterMultiJoinMergeRule(Config config) {
+    super(config);
+  }
 
-  /**
-   * Creates a FilterMultiJoinMergeRule that uses {@link Filter}
-   * of type {@link LogicalFilter}
-   * @param relBuilderFactory builder factory for relational expressions
-   */
+  @Deprecated // to be removed before 2.0
   public FilterMultiJoinMergeRule(RelBuilderFactory relBuilderFactory) {
-    this(LogicalFilter.class, relBuilderFactory);
+    this(Config.DEFAULT.withRelBuilderFactory(relBuilderFactory)
+        .as(Config.class));
   }
 
-  /**
-   * Creates a FilterMultiJoinMergeRule that uses a generic
-   * {@link Filter}
-   * @param filterClass filter class
-   * @param relBuilderFactory builder factory for relational expressions
-   */
+  @Deprecated // to be removed before 2.0
   public FilterMultiJoinMergeRule(Class<? extends Filter> filterClass,
       RelBuilderFactory relBuilderFactory) {
-    super(
-        operand(filterClass,
-          operand(MultiJoin.class, any())),
-        relBuilderFactory, null);
+    this(Config.DEFAULT.withRelBuilderFactory(relBuilderFactory)
+        .as(Config.class)
+        .withOperandFor(filterClass, MultiJoin.class));
   }
 
   //~ Methods ----------------------------------------------------------------
 
-  public void onMatch(RelOptRuleCall call) {
+  @Override public void onMatch(RelOptRuleCall call) {
     Filter filter = call.rel(0);
     MultiJoin multiJoin = call.rel(1);
 
@@ -94,4 +86,23 @@ public class FilterMultiJoinMergeRule extends RelOptRule implements Transformati
 
     call.transformTo(newMultiJoin);
   }
+
+  /** Rule configuration. */
+  public interface Config extends RelRule.Config {
+    Config DEFAULT = EMPTY.as(Config.class)
+        .withOperandFor(Filter.class, MultiJoin.class);
+
+    @Override default FilterMultiJoinMergeRule toRule() {
+      return new FilterMultiJoinMergeRule(this);
+    }
+
+    /** Defines an operand tree for the given classes. */
+    default Config withOperandFor(Class<? extends Filter> filterClass,
+        Class<? extends MultiJoin> multiJoinClass) {
+      return withOperandSupplier(b0 ->
+          b0.operand(filterClass).oneInput(b1 ->
+              b1.operand(multiJoinClass).anyInputs()))
+          .as(Config.class);
+    }
+  }
 }
diff --git a/core/src/main/java/org/apache/calcite/rel/rules/FilterProjectTransposeRule.java b/core/src/main/java/org/apache/calcite/rel/rules/FilterProjectTransposeRule.java
index 192ba81..5d94844 100644
--- a/core/src/main/java/org/apache/calcite/rel/rules/FilterProjectTransposeRule.java
+++ b/core/src/main/java/org/apache/calcite/rel/rules/FilterProjectTransposeRule.java
@@ -16,10 +16,10 @@
  */
 package org.apache.calcite.rel.rules;
 
-import org.apache.calcite.plan.RelOptRule;
 import org.apache.calcite.plan.RelOptRuleCall;
 import org.apache.calcite.plan.RelOptRuleOperand;
 import org.apache.calcite.plan.RelOptUtil;
+import org.apache.calcite.plan.RelRule;
 import org.apache.calcite.plan.RelTraitSet;
 import org.apache.calcite.rel.RelCollationTraitDef;
 import org.apache.calcite.rel.RelDistributionTraitDef;
@@ -28,10 +28,10 @@ import org.apache.calcite.rel.core.Filter;
 import org.apache.calcite.rel.core.Project;
 import org.apache.calcite.rel.core.RelFactories;
 import org.apache.calcite.rex.RexNode;
-import org.apache.calcite.rex.RexOver;
 import org.apache.calcite.rex.RexUtil;
 import org.apache.calcite.tools.RelBuilder;
 import org.apache.calcite.tools.RelBuilderFactory;
+import org.apache.calcite.util.ImmutableBeans;
 
 import java.util.Collections;
 import java.util.function.Predicate;
@@ -40,17 +40,17 @@ import java.util.function.Predicate;
  * Planner rule that pushes
  * a {@link org.apache.calcite.rel.core.Filter}
  * past a {@link org.apache.calcite.rel.core.Project}.
+ *
+ * @see CoreRules#FILTER_PROJECT_TRANSPOSE
  */
-public class FilterProjectTransposeRule extends RelOptRule implements TransformationRule {
-  /** @deprecated Use {@link CoreRules#FILTER_PROJECT_TRANSPOSE}. */
-  @Deprecated // to be removed before 1.25
-  public static final FilterProjectTransposeRule INSTANCE =
-      CoreRules.FILTER_PROJECT_TRANSPOSE;
-
-  private final boolean copyFilter;
-  private final boolean copyProject;
+public class FilterProjectTransposeRule
+    extends RelRule<FilterProjectTransposeRule.Config>
+    implements TransformationRule {
 
-  //~ Constructors -----------------------------------------------------------
+  /** Creates a FilterProjectTransposeRule. */
+  protected FilterProjectTransposeRule(Config config) {
+    super(config);
+  }
 
   /**
    * Creates a FilterProjectTransposeRule.
@@ -62,15 +62,20 @@ public class FilterProjectTransposeRule extends RelOptRule implements Transforma
    * filter (since in some cases it can prevent a
    * {@link org.apache.calcite.rel.core.Correlate} from being de-correlated).
    */
+  @Deprecated // to be removed before 2.0
   public FilterProjectTransposeRule(
       Class<? extends Filter> filterClass,
       Class<? extends Project> projectClass,
       boolean copyFilter, boolean copyProject,
       RelBuilderFactory relBuilderFactory) {
-    this(filterClass,
-        filter -> !RexUtil.containsCorrelation(filter.getCondition()),
-        projectClass, project -> true,
-        copyFilter, copyProject, relBuilderFactory);
+    this(Config.DEFAULT
+        .withRelBuilderFactory(relBuilderFactory)
+        .as(Config.class)
+        .withOperandFor(filterClass,
+            f -> !RexUtil.containsCorrelation(f.getCondition()),
+            projectClass, project -> true)
+        .withCopyFilter(copyFilter)
+        .withCopyProject(copyProject));
   }
 
   /**
@@ -85,6 +90,7 @@ public class FilterProjectTransposeRule extends RelOptRule implements Transforma
    * and/or the Project (using {@code projectPredicate} allows making the rule
    * more restrictive.
    */
+  @Deprecated // to be removed before 2.0
   public <F extends Filter, P extends Project> FilterProjectTransposeRule(
       Class<F> filterClass,
       Predicate<? super F> filterPredicate,
@@ -92,10 +98,16 @@ public class FilterProjectTransposeRule extends RelOptRule implements Transforma
       Predicate<? super P> projectPredicate,
       boolean copyFilter, boolean copyProject,
       RelBuilderFactory relBuilderFactory) {
-    this(
-        operandJ(filterClass, null, filterPredicate,
-            operandJ(projectClass, null, projectPredicate, any())),
-        copyFilter, copyProject, relBuilderFactory);
+    this(Config.DEFAULT
+        .withRelBuilderFactory(relBuilderFactory)
+        .withOperandSupplier(b0 ->
+            b0.operand(filterClass).predicate(filterPredicate)
+                .oneInput(b1 ->
+                    b1.operand(projectClass).predicate(projectPredicate)
+                        .anyInputs()))
+            .as(Config.class)
+            .withCopyFilter(copyFilter)
+            .withCopyProject(copyProject));
   }
 
   @Deprecated // to be removed before 2.0
@@ -104,30 +116,42 @@ public class FilterProjectTransposeRule extends RelOptRule implements Transforma
       RelFactories.FilterFactory filterFactory,
       Class<? extends Project> projectClass,
       RelFactories.ProjectFactory projectFactory) {
-    this(filterClass, filter -> !RexUtil.containsCorrelation(filter.getCondition()),
-        projectClass, project -> true,
-        filterFactory == null,
-        projectFactory == null,
-        RelBuilder.proto(filterFactory, projectFactory));
+    this(Config.DEFAULT
+        .withRelBuilderFactory(RelBuilder.proto(filterFactory, projectFactory))
+        .withOperandSupplier(b0 ->
+            b0.operand(filterClass)
+                .predicate(filter ->
+                    !RexUtil.containsCorrelation(filter.getCondition()))
+                .oneInput(b2 ->
+                    b2.operand(projectClass)
+                        .predicate(project -> true)
+                        .anyInputs()))
+        .as(Config.class)
+        .withCopyFilter(filterFactory == null)
+        .withCopyProject(projectFactory == null));
   }
 
+  @Deprecated // to be removed before 2.0
   protected FilterProjectTransposeRule(
       RelOptRuleOperand operand,
       boolean copyFilter,
       boolean copyProject,
       RelBuilderFactory relBuilderFactory) {
-    super(operand, relBuilderFactory, null);
-    this.copyFilter = copyFilter;
-    this.copyProject = copyProject;
+    this(Config.DEFAULT
+        .withRelBuilderFactory(relBuilderFactory)
+        .withOperandSupplier(b -> b.exactly(operand))
+        .as(Config.class)
+        .withCopyFilter(copyFilter)
+        .withCopyProject(copyProject));
   }
 
   //~ Methods ----------------------------------------------------------------
 
-  public void onMatch(RelOptRuleCall call) {
+  @Override public void onMatch(RelOptRuleCall call) {
     final Filter filter = call.rel(0);
     final Project project = call.rel(1);
 
-    if (RexOver.containsOver(project.getProjects(), null)) {
+    if (project.containsOver()) {
       // In general a filter cannot be pushed below a windowing calculation.
       // Applying the filter before the aggregation function changes
       // the results of the windowing invocation.
@@ -142,7 +166,7 @@ public class FilterProjectTransposeRule extends RelOptRule implements Transforma
 
     final RelBuilder relBuilder = call.builder();
     RelNode newFilterRel;
-    if (copyFilter) {
+    if (config.isCopyFilter()) {
       final RelNode input = project.getInput();
       final RelTraitSet traitSet = filter.getTraitSet()
           .replaceIfs(RelCollationTraitDef.INSTANCE,
@@ -158,14 +182,77 @@ public class FilterProjectTransposeRule extends RelOptRule implements Transforma
           relBuilder.push(project.getInput()).filter(newCondition).build();
     }
 
-    RelNode newProjRel =
-        copyProject
+    RelNode newProject =
+        config.isCopyProject()
             ? project.copy(project.getTraitSet(), newFilterRel,
                 project.getProjects(), project.getRowType())
             : relBuilder.push(newFilterRel)
                 .project(project.getProjects(), project.getRowType().getFieldNames())
                 .build();
 
-    call.transformTo(newProjRel);
+    call.transformTo(newProject);
+  }
+
+  /** Rule configuration.
+   *
+   * <p>If {@code copyFilter} is true, creates the same kind of Filter as
+   * matched in the rule, otherwise it creates a Filter using the RelBuilder
+   * obtained by the {@code relBuilderFactory}.
+   * Similarly for {@code copyProject}.
+   *
+   * <p>Defining predicates for the Filter (using {@code filterPredicate})
+   * and/or the Project (using {@code projectPredicate} allows making the rule
+   * more restrictive. */
+  public interface Config extends RelRule.Config {
+    Config DEFAULT = EMPTY.as(Config.class)
+        .withOperandFor(Filter.class,
+            f -> !RexUtil.containsCorrelation(f.getCondition()),
+            Project.class, p -> true)
+        .withCopyFilter(true)
+        .withCopyProject(true);
+
+    @Override default FilterProjectTransposeRule toRule() {
+      return new FilterProjectTransposeRule(this);
+    }
+
+    /** Whether to create a {@link Filter} of the same convention as the
+     * matched Filter. */
+    @ImmutableBeans.Property
+    @ImmutableBeans.BooleanDefault(true)
+    boolean isCopyFilter();
+
+    /** Sets {@link #isCopyFilter()}. */
+    Config withCopyFilter(boolean copyFilter);
+
+    /** Whether to create a {@link Project} of the same convention as the
+     * matched Project. */
+    @ImmutableBeans.Property
+    @ImmutableBeans.BooleanDefault(true)
+    boolean isCopyProject();
+
+    /** Sets {@link #isCopyProject()}. */
+    Config withCopyProject(boolean copyProject);
+
+    /** Defines an operand tree for the given 2 classes. */
+    default Config withOperandFor(Class<? extends Filter> filterClass,
+        Predicate<Filter> filterPredicate,
+        Class<? extends Project> projectClass,
+        Predicate<Project> projectPredicate) {
+      return withOperandSupplier(b0 ->
+          b0.operand(filterClass).predicate(filterPredicate).oneInput(b1 ->
+              b1.operand(projectClass).predicate(projectPredicate).anyInputs()))
+          .as(Config.class);
+    }
+
+    /** Defines an operand tree for the given 3 classes. */
+    default Config withOperandFor(Class<? extends Filter> filterClass,
+        Class<? extends Project> projectClass,
+        Class<? extends RelNode> relClass) {
+      return withOperandSupplier(b0 ->
+          b0.operand(filterClass).oneInput(b1 ->
+              b1.operand(projectClass).oneInput(b2 ->
+                  b2.operand(relClass).anyInputs())))
+          .as(Config.class);
+    }
   }
 }
diff --git a/core/src/main/java/org/apache/calcite/rel/rules/FilterRemoveIsNotDistinctFromRule.java b/core/src/main/java/org/apache/calcite/rel/rules/FilterRemoveIsNotDistinctFromRule.java
index 070b142..8a908fa 100644
--- a/core/src/main/java/org/apache/calcite/rel/rules/FilterRemoveIsNotDistinctFromRule.java
+++ b/core/src/main/java/org/apache/calcite/rel/rules/FilterRemoveIsNotDistinctFromRule.java
@@ -16,9 +16,9 @@
  */
 package org.apache.calcite.rel.rules;
 
-import org.apache.calcite.plan.RelOptRule;
 import org.apache.calcite.plan.RelOptRuleCall;
 import org.apache.calcite.plan.RelOptUtil;
+import org.apache.calcite.plan.RelRule;
 import org.apache.calcite.rel.RelNode;
 import org.apache.calcite.rel.core.Filter;
 import org.apache.calcite.rex.RexBuilder;
@@ -35,31 +35,27 @@ import org.apache.calcite.tools.RelBuilderFactory;
  * in a {@link Filter} with logically equivalent operations.
  *
  * @see org.apache.calcite.sql.fun.SqlStdOperatorTable#IS_NOT_DISTINCT_FROM
+ * @see CoreRules#FILTER_EXPAND_IS_NOT_DISTINCT_FROM
  */
-public final class FilterRemoveIsNotDistinctFromRule extends RelOptRule
+public final class FilterRemoveIsNotDistinctFromRule
+    extends RelRule<FilterRemoveIsNotDistinctFromRule.Config>
     implements TransformationRule {
-  //~ Static fields/initializers ---------------------------------------------
 
-  /** @deprecated Use {@link CoreRules#FILTER_EXPAND_IS_NOT_DISTINCT_FROM}. */
-  @Deprecated // to be removed before 1.25
-  public static final FilterRemoveIsNotDistinctFromRule INSTANCE =
-      CoreRules.FILTER_EXPAND_IS_NOT_DISTINCT_FROM;
-
-  //~ Constructors -----------------------------------------------------------
+  /** Creates a FilterRemoveIsNotDistinctFromRule. */
+  protected FilterRemoveIsNotDistinctFromRule(Config config) {
+    super(config);
+  }
 
-  /**
-   * Creates a FilterRemoveIsNotDistinctFromRule.
-   *
-   * @param relBuilderFactory Builder for relational expressions
-   */
+  @Deprecated // to be removed before 2.0
   public FilterRemoveIsNotDistinctFromRule(
       RelBuilderFactory relBuilderFactory) {
-    super(operand(Filter.class, any()), relBuilderFactory, null);
+    this(Config.DEFAULT.withRelBuilderFactory(relBuilderFactory)
+        .as(Config.class));
   }
 
   //~ Methods ----------------------------------------------------------------
 
-  public void onMatch(RelOptRuleCall call) {
+  @Override public void onMatch(RelOptRuleCall call) {
     Filter oldFilter = call.rel(0);
     RexNode oldFilterCond = oldFilter.getCondition();
 
@@ -92,16 +88,15 @@ public final class FilterRemoveIsNotDistinctFromRule extends RelOptRule
   /** Shuttle that removes 'x IS NOT DISTINCT FROM y' and converts it
    * to 'CASE WHEN x IS NULL THEN y IS NULL WHEN y IS NULL THEN x IS
    * NULL ELSE x = y END'. */
-  private class RemoveIsNotDistinctFromRexShuttle extends RexShuttle {
-    RexBuilder rexBuilder;
+  private static class RemoveIsNotDistinctFromRexShuttle extends RexShuttle {
+    final RexBuilder rexBuilder;
 
     RemoveIsNotDistinctFromRexShuttle(
         RexBuilder rexBuilder) {
       this.rexBuilder = rexBuilder;
     }
 
-    // override RexShuttle
-    public RexNode visitCall(RexCall call) {
+    @Override public RexNode visitCall(RexCall call) {
       RexNode newCall = super.visitCall(call);
 
       if (call.getOperator()
@@ -117,4 +112,15 @@ public final class FilterRemoveIsNotDistinctFromRule extends RelOptRule
       return newCall;
     }
   }
+
+  /** Rule configuration. */
+  public interface Config extends RelRule.Config {
+    Config DEFAULT = EMPTY
+        .withOperandSupplier(b -> b.operand(Filter.class).anyInputs())
+        .as(Config.class);
+
+    @Override default FilterRemoveIsNotDistinctFromRule toRule() {
+      return new FilterRemoveIsNotDistinctFromRule(this);
+    }
+  }
 }
diff --git a/core/src/main/java/org/apache/calcite/rel/rules/FilterSetOpTransposeRule.java b/core/src/main/java/org/apache/calcite/rel/rules/FilterSetOpTransposeRule.java
index 6e3b7d9..d4daea9 100644
--- a/core/src/main/java/org/apache/calcite/rel/rules/FilterSetOpTransposeRule.java
+++ b/core/src/main/java/org/apache/calcite/rel/rules/FilterSetOpTransposeRule.java
@@ -17,9 +17,9 @@
 package org.apache.calcite.rel.rules;
 
 import org.apache.calcite.plan.Contexts;
-import org.apache.calcite.plan.RelOptRule;
 import org.apache.calcite.plan.RelOptRuleCall;
 import org.apache.calcite.plan.RelOptUtil;
+import org.apache.calcite.plan.RelRule;
 import org.apache.calcite.rel.RelNode;
 import org.apache.calcite.rel.core.Filter;
 import org.apache.calcite.rel.core.RelFactories;
@@ -36,34 +36,34 @@ import java.util.List;
 /**
  * Planner rule that pushes a {@link org.apache.calcite.rel.core.Filter}
  * past a {@link org.apache.calcite.rel.core.SetOp}.
+ *
+ * @see CoreRules#FILTER_SET_OP_TRANSPOSE
  */
-public class FilterSetOpTransposeRule extends RelOptRule implements TransformationRule {
-  /** @deprecated Use {@link CoreRules#FILTER_SET_OP_TRANSPOSE}. */
-  @Deprecated // to be removed before 1.25
-  public static final FilterSetOpTransposeRule INSTANCE =
-      CoreRules.FILTER_SET_OP_TRANSPOSE;
+public class FilterSetOpTransposeRule
+    extends RelRule<FilterSetOpTransposeRule.Config>
+    implements TransformationRule {
 
-  //~ Constructors -----------------------------------------------------------
+  /** Creates a FilterSetOpTransposeRule. */
+  protected FilterSetOpTransposeRule(Config config) {
+    super(config);
+  }
 
-  /**
-   * Creates a FilterSetOpTransposeRule.
-   */
+  @Deprecated // to be removed before 2.0
   public FilterSetOpTransposeRule(RelBuilderFactory relBuilderFactory) {
-    super(
-        operand(Filter.class,
-            operand(SetOp.class, any())),
-        relBuilderFactory, null);
+    this(Config.DEFAULT.withRelBuilderFactory(relBuilderFactory)
+        .as(Config.class));
   }
 
   @Deprecated // to  be removed before 2.0
   public FilterSetOpTransposeRule(RelFactories.FilterFactory filterFactory) {
-    this(RelBuilder.proto(Contexts.of(filterFactory)));
+    this(Config.DEFAULT
+        .withRelBuilderFactory(RelBuilder.proto(Contexts.of(filterFactory)))
+        .as(Config.class));
   }
 
   //~ Methods ----------------------------------------------------------------
 
-  // implement RelOptRule
-  public void onMatch(RelOptRuleCall call) {
+  @Override public void onMatch(RelOptRuleCall call) {
     Filter filterRel = call.rel(0);
     SetOp setOp = call.rel(1);
 
@@ -94,4 +94,17 @@ public class FilterSetOpTransposeRule extends RelOptRule implements Transformati
 
     call.transformTo(newSetOp);
   }
+
+  /** Rule configuration. */
+  public interface Config extends RelRule.Config {
+    Config DEFAULT = EMPTY
+        .withOperandSupplier(b0 ->
+            b0.operand(Filter.class).oneInput(b1 ->
+                b1.operand(SetOp.class).anyInputs()))
+        .as(Config.class);
+
+    @Override default FilterSetOpTransposeRule toRule() {
+      return new FilterSetOpTransposeRule(this);
+    }
+  }
 }
diff --git a/core/src/main/java/org/apache/calcite/rel/rules/FilterTableFunctionTransposeRule.java b/core/src/main/java/org/apache/calcite/rel/rules/FilterTableFunctionTransposeRule.java
index 00e3833..7042055 100644
--- a/core/src/main/java/org/apache/calcite/rel/rules/FilterTableFunctionTransposeRule.java
+++ b/core/src/main/java/org/apache/calcite/rel/rules/FilterTableFunctionTransposeRule.java
@@ -17,9 +17,9 @@
 package org.apache.calcite.rel.rules;
 
 import org.apache.calcite.plan.RelOptCluster;
-import org.apache.calcite.plan.RelOptRule;
 import org.apache.calcite.plan.RelOptRuleCall;
 import org.apache.calcite.plan.RelOptUtil;
+import org.apache.calcite.plan.RelRule;
 import org.apache.calcite.rel.RelNode;
 import org.apache.calcite.rel.logical.LogicalFilter;
 import org.apache.calcite.rel.logical.LogicalTableFunctionScan;
@@ -37,30 +37,27 @@ import java.util.Set;
  * Planner rule that pushes
  * a {@link org.apache.calcite.rel.logical.LogicalFilter}
  * past a {@link org.apache.calcite.rel.logical.LogicalTableFunctionScan}.
+ *
+ * @see CoreRules#FILTER_TABLE_FUNCTION_TRANSPOSE
  */
-public class FilterTableFunctionTransposeRule extends RelOptRule
+public class FilterTableFunctionTransposeRule
+    extends RelRule<FilterTableFunctionTransposeRule.Config>
     implements TransformationRule {
-  /** @deprecated Use {@link CoreRules#FILTER_TABLE_FUNCTION_TRANSPOSE}. */
-  @Deprecated // to be removed before 1.25
-  public static final FilterTableFunctionTransposeRule INSTANCE =
-      CoreRules.FILTER_TABLE_FUNCTION_TRANSPOSE;
 
-  //~ Constructors -----------------------------------------------------------
+  /** Creates a FilterTableFunctionTransposeRule. */
+  protected FilterTableFunctionTransposeRule(Config config) {
+    super(config);
+  }
 
-  /**
-   * Creates a FilterTableFunctionTransposeRule.
-   */
+  @Deprecated // to be removed before 2.0
   public FilterTableFunctionTransposeRule(RelBuilderFactory relBuilderFactory) {
-    super(
-        operand(LogicalFilter.class,
-            operand(LogicalTableFunctionScan.class, any())),
-        relBuilderFactory, null);
+    this(Config.DEFAULT.withRelBuilderFactory(relBuilderFactory)
+        .as(Config.class));
   }
 
   //~ Methods ----------------------------------------------------------------
 
-  // implement RelOptRule
-  public void onMatch(RelOptRuleCall call) {
+  @Override public void onMatch(RelOptRuleCall call) {
     LogicalFilter filter = call.rel(0);
     LogicalTableFunctionScan funcRel = call.rel(1);
     Set<RelColumnMapping> columnMappings = funcRel.getColumnMappings();
@@ -119,4 +116,17 @@ public class FilterTableFunctionTransposeRule extends RelOptRule
             columnMappings);
     call.transformTo(newFuncRel);
   }
+
+  /** Rule configuration. */
+  public interface Config extends RelRule.Config {
+    Config DEFAULT = EMPTY
+        .withOperandSupplier(b0 ->
+            b0.operand(LogicalFilter.class).oneInput(b1 ->
+                b1.operand(LogicalTableFunctionScan.class).anyInputs()))
+        .as(Config.class);
+
+    @Override default FilterTableFunctionTransposeRule toRule() {
+      return new FilterTableFunctionTransposeRule(this);
+    }
+  }
 }
diff --git a/core/src/main/java/org/apache/calcite/rel/rules/FilterTableScanRule.java b/core/src/main/java/org/apache/calcite/rel/rules/FilterTableScanRule.java
index 2f11cee..9065386 100644
--- a/core/src/main/java/org/apache/calcite/rel/rules/FilterTableScanRule.java
+++ b/core/src/main/java/org/apache/calcite/rel/rules/FilterTableScanRule.java
@@ -16,13 +16,13 @@
  */
 package org.apache.calcite.rel.rules;
 
+import org.apache.calcite.adapter.enumerable.EnumerableInterpreter;
 import org.apache.calcite.interpreter.Bindables;
-import org.apache.calcite.plan.RelOptRule;
 import org.apache.calcite.plan.RelOptRuleCall;
 import org.apache.calcite.plan.RelOptRuleOperand;
 import org.apache.calcite.plan.RelOptTable;
+import org.apache.calcite.plan.RelRule;
 import org.apache.calcite.rel.core.Filter;
-import org.apache.calcite.rel.core.RelFactories;
 import org.apache.calcite.rel.core.TableScan;
 import org.apache.calcite.rex.RexNode;
 import org.apache.calcite.rex.RexUtil;
@@ -43,38 +43,37 @@ import com.google.common.collect.ImmutableList;
  * or a {@link org.apache.calcite.schema.ProjectableFilterableTable}
  * to a {@link org.apache.calcite.interpreter.Bindables.BindableTableScan}.
  *
- * <p>The {@link #INTERPRETER} variant allows an intervening
+ * <p>The {@link CoreRules#FILTER_INTERPRETER_SCAN} variant allows an
+ * intervening
  * {@link org.apache.calcite.adapter.enumerable.EnumerableInterpreter}.
  *
  * @see org.apache.calcite.rel.rules.ProjectTableScanRule
+ * @see CoreRules#FILTER_SCAN
+ * @see CoreRules#FILTER_INTERPRETER_SCAN
  */
-public abstract class FilterTableScanRule extends RelOptRule {
+public class FilterTableScanRule
+    extends RelRule<FilterTableScanRule.Config> {
   @SuppressWarnings("Guava")
   @Deprecated // to be removed before 2.0
   public static final com.google.common.base.Predicate<TableScan> PREDICATE =
       FilterTableScanRule::test;
 
-  /** @deprecated Use {@link CoreRules#FILTER_SCAN}. */
-  @Deprecated // to be removed before 1.25
-  public static final FilterTableScanRule INSTANCE =
-      CoreRules.FILTER_SCAN;
-
-  /** @deprecated Use {@link CoreRules#FILTER_INTERPRETER_SCAN}. */
-  @Deprecated // to be removed before 1.25
-  public static final FilterTableScanRule INTERPRETER =
-      CoreRules.FILTER_INTERPRETER_SCAN;
-
-  //~ Constructors -----------------------------------------------------------
+  /** Creates a FilterTableScanRule. */
+  protected FilterTableScanRule(Config config) {
+    super(config);
+  }
 
   @Deprecated // to be removed before 2.0
   protected FilterTableScanRule(RelOptRuleOperand operand, String description) {
-    this(operand, RelFactories.LOGICAL_BUILDER, description);
+    this(Config.EMPTY.as(Config.class));
+    throw new UnsupportedOperationException();
   }
 
-  /** Creates a FilterTableScanRule. */
+  @Deprecated // to be removed before 2.0
   protected FilterTableScanRule(RelOptRuleOperand operand,
       RelBuilderFactory relBuilderFactory, String description) {
-    super(operand, relBuilderFactory, description);
+    this(Config.EMPTY.as(Config.class));
+    throw new UnsupportedOperationException();
   }
 
   //~ Methods ----------------------------------------------------------------
@@ -87,6 +86,22 @@ public abstract class FilterTableScanRule extends RelOptRule {
         || table.unwrap(ProjectableFilterableTable.class) != null;
   }
 
+  @Override public void onMatch(RelOptRuleCall call) {
+    if (call.rels.length == 2) {
+      // the ordinary variant
+      final Filter filter = call.rel(0);
+      final TableScan scan = call.rel(1);
+      apply(call, filter, scan);
+    } else if (call.rels.length == 3) {
+      // the variant with intervening EnumerableInterpreter
+      final Filter filter = call.rel(0);
+      final TableScan scan = call.rel(2);
+      apply(call, filter, scan);
+    } else {
+      throw new AssertionError();
+    }
+  }
+
   protected void apply(RelOptRuleCall call, Filter filter, TableScan scan) {
     final ImmutableIntList projects;
     final ImmutableList.Builder<RexNode> filters = ImmutableList.builder();
@@ -108,4 +123,27 @@ public abstract class FilterTableScanRule extends RelOptRule {
         Bindables.BindableTableScan.create(scan.getCluster(), scan.getTable(),
             filters.build(), projects));
   }
+
+  /** Rule configuration. */
+  public interface Config extends RelRule.Config {
+    Config DEFAULT = EMPTY
+        .withOperandSupplier(b0 ->
+            b0.operand(Filter.class).oneInput(b1 ->
+                b1.operand(TableScan.class)
+                    .predicate(FilterTableScanRule::test).noInputs()))
+        .as(Config.class);
+
+    Config INTERPRETER = EMPTY
+        .withOperandSupplier(b0 ->
+            b0.operand(Filter.class).oneInput(b1 ->
+                b1.operand(EnumerableInterpreter.class).oneInput(b2 ->
+                    b2.operand(TableScan.class)
+                        .predicate(FilterTableScanRule::test).noInputs())))
+        .withDescription("FilterTableScanRule:interpreter")
+        .as(Config.class);
+
+    @Override default FilterTableScanRule toRule() {
+      return new FilterTableScanRule(this);
+    }
+  }
 }
diff --git a/core/src/main/java/org/apache/calcite/rel/rules/FilterToCalcRule.java b/core/src/main/java/org/apache/calcite/rel/rules/FilterToCalcRule.java
index a6c1d3c..98afcd3 100644
--- a/core/src/main/java/org/apache/calcite/rel/rules/FilterToCalcRule.java
+++ b/core/src/main/java/org/apache/calcite/rel/rules/FilterToCalcRule.java
@@ -16,8 +16,8 @@
  */
 package org.apache.calcite.rel.rules;
 
-import org.apache.calcite.plan.RelOptRule;
 import org.apache.calcite.plan.RelOptRuleCall;
+import org.apache.calcite.plan.RelRule;
 import org.apache.calcite.rel.RelNode;
 import org.apache.calcite.rel.logical.LogicalCalc;
 import org.apache.calcite.rel.logical.LogicalFilter;
@@ -40,29 +40,27 @@ import org.apache.calcite.tools.RelBuilderFactory;
  * {@link org.apache.calcite.rel.logical.LogicalCalc}. This
  * {@link org.apache.calcite.rel.logical.LogicalFilter} will eventually be
  * converted by {@link FilterCalcMergeRule}.
+ *
+ * @see CoreRules#FILTER_TO_CALC
  */
-public class FilterToCalcRule extends RelOptRule implements TransformationRule {
-  //~ Static fields/initializers ---------------------------------------------
-
-  /** @deprecated Use {@link CoreRules#FILTER_TO_CALC}. */
-  @Deprecated // to be removed before 1.25
-  public static final FilterToCalcRule INSTANCE =
-      CoreRules.FILTER_TO_CALC;
+public class FilterToCalcRule
+    extends RelRule<FilterToCalcRule.Config>
+    implements TransformationRule {
 
-  //~ Constructors -----------------------------------------------------------
+  /** Creates a FilterToCalcRule. */
+  protected FilterToCalcRule(Config config) {
+    super(config);
+  }
 
-  /**
-   * Creates a FilterToCalcRule.
-   *
-   * @param relBuilderFactory Builder for relational expressions
-   */
+  @Deprecated // to be removed before 2.0
   public FilterToCalcRule(RelBuilderFactory relBuilderFactory) {
-    super(operand(LogicalFilter.class, any()), relBuilderFactory, null);
+    this(Config.DEFAULT.withRelBuilderFactory(relBuilderFactory)
+        .as(Config.class));
   }
 
   //~ Methods ----------------------------------------------------------------
 
-  public void onMatch(RelOptRuleCall call) {
+  @Override public void onMatch(RelOptRuleCall call) {
     final LogicalFilter filter = call.rel(0);
     final RelNode rel = filter.getInput();
 
@@ -78,4 +76,16 @@ public class FilterToCalcRule extends RelOptRule implements TransformationRule {
     final LogicalCalc calc = LogicalCalc.create(rel, program);
     call.transformTo(calc);
   }
+
+  /** Rule configuration. */
+  public interface Config extends RelRule.Config {
+    Config DEFAULT = EMPTY
+        .withOperandSupplier(b ->
+            b.operand(LogicalFilter.class).anyInputs())
+        .as(Config.class);
+
+    @Override default FilterToCalcRule toRule() {
+      return new FilterToCalcRule(this);
+    }
+  }
 }
diff --git a/core/src/main/java/org/apache/calcite/rel/rules/IntersectToDistinctRule.java b/core/src/main/java/org/apache/calcite/rel/rules/IntersectToDistinctRule.java
index 7772285..bc74cf4 100644
--- a/core/src/main/java/org/apache/calcite/rel/rules/IntersectToDistinctRule.java
+++ b/core/src/main/java/org/apache/calcite/rel/rules/IntersectToDistinctRule.java
@@ -17,10 +17,11 @@
 package org.apache.calcite.rel.rules;
 
 import org.apache.calcite.plan.RelOptCluster;
-import org.apache.calcite.plan.RelOptRule;
 import org.apache.calcite.plan.RelOptRuleCall;
+import org.apache.calcite.plan.RelRule;
 import org.apache.calcite.rel.RelNode;
 import org.apache.calcite.rel.core.Intersect;
+import org.apache.calcite.rel.logical.LogicalIntersect;
 import org.apache.calcite.rex.RexBuilder;
 import org.apache.calcite.tools.RelBuilder;
 import org.apache.calcite.tools.RelBuilderFactory;
@@ -63,24 +64,28 @@ import java.math.BigDecimal;
  * <p><code>R6 = Proj(R5 on all attributes)</code>
  *
  * @see org.apache.calcite.rel.rules.UnionToDistinctRule
+ * @see CoreRules#INTERSECT_TO_DISTINCT
  */
-public class IntersectToDistinctRule extends RelOptRule implements TransformationRule {
-  /** @deprecated Use {@link CoreRules#INTERSECT_TO_DISTINCT}. */
-  @Deprecated // to be removed before 1.25
-  public static final IntersectToDistinctRule INSTANCE =
-      CoreRules.INTERSECT_TO_DISTINCT;
-
-  //~ Constructors -----------------------------------------------------------
+public class IntersectToDistinctRule
+    extends RelRule<IntersectToDistinctRule.Config>
+    implements TransformationRule {
 
   /** Creates an IntersectToDistinctRule. */
-  public IntersectToDistinctRule(Class<? extends Intersect> intersectClazz,
+  protected IntersectToDistinctRule(Config config) {
+    super(config);
+  }
+
+  @Deprecated // to be removed before 2.0
+  public IntersectToDistinctRule(Class<? extends Intersect> intersectClass,
       RelBuilderFactory relBuilderFactory) {
-    super(operand(intersectClazz, any()), relBuilderFactory, null);
+    this(Config.DEFAULT.withRelBuilderFactory(relBuilderFactory)
+        .as(Config.class)
+        .withOperandFor(intersectClass));
   }
 
   //~ Methods ----------------------------------------------------------------
 
-  public void onMatch(RelOptRuleCall call) {
+  @Override public void onMatch(RelOptRuleCall call) {
     final Intersect intersect = call.rel(0);
     if (intersect.all) {
       return; // nothing we can do
@@ -123,4 +128,20 @@ public class IntersectToDistinctRule extends RelOptRule implements Transformatio
     // finally add a project to project out the last column
     call.transformTo(relBuilder.build());
   }
+
+  /** Rule configuration. */
+  public interface Config extends RelRule.Config {
+    Config DEFAULT = EMPTY.as(Config.class)
+        .withOperandFor(LogicalIntersect.class);
+
+    @Override default IntersectToDistinctRule toRule() {
+      return new IntersectToDistinctRule(this);
+    }
+
+    /** Defines an operand tree for the given classes. */
+    default Config withOperandFor(Class<? extends Intersect> intersectClass) {
+      return withOperandSupplier(b -> b.operand(intersectClass).anyInputs())
+          .as(Config.class);
+    }
+  }
 }
diff --git a/core/src/main/java/org/apache/calcite/rel/rules/JoinAddRedundantSemiJoinRule.java b/core/src/main/java/org/apache/calcite/rel/rules/JoinAddRedundantSemiJoinRule.java
index 4a9d30f..9c427a5 100644
--- a/core/src/main/java/org/apache/calcite/rel/rules/JoinAddRedundantSemiJoinRule.java
+++ b/core/src/main/java/org/apache/calcite/rel/rules/JoinAddRedundantSemiJoinRule.java
@@ -16,8 +16,8 @@
  */
 package org.apache.calcite.rel.rules;
 
-import org.apache.calcite.plan.RelOptRule;
 import org.apache.calcite.plan.RelOptRuleCall;
+import org.apache.calcite.plan.RelRule;
 import org.apache.calcite.rel.RelNode;
 import org.apache.calcite.rel.core.Join;
 import org.apache.calcite.rel.core.JoinInfo;
@@ -33,36 +33,38 @@ import com.google.common.collect.ImmutableSet;
  *
  * <p>LogicalJoin(X, Y) &rarr; LogicalJoin(SemiJoin(X, Y), Y)
  *
- * <p>The constructor is parameterized to allow any sub-class of
+ * <p>Can be configured to match any sub-class of
  * {@link org.apache.calcite.rel.core.Join}, not just
  * {@link org.apache.calcite.rel.logical.LogicalJoin}.
+ *
+ * @see CoreRules#JOIN_ADD_REDUNDANT_SEMI_JOIN
  */
-public class JoinAddRedundantSemiJoinRule extends RelOptRule
+public class JoinAddRedundantSemiJoinRule
+    extends RelRule<JoinAddRedundantSemiJoinRule.Config>
     implements TransformationRule {
-  /** @deprecated Use {@link CoreRules#JOIN_ADD_REDUNDANT_SEMI_JOIN}. */
-  @Deprecated // to be removed before 1.25
-  public static final JoinAddRedundantSemiJoinRule INSTANCE =
-      CoreRules.JOIN_ADD_REDUNDANT_SEMI_JOIN;
 
-  //~ Constructors -----------------------------------------------------------
+  /** Creates a JoinAddRedundantSemiJoinRule. */
+  protected JoinAddRedundantSemiJoinRule(Config config) {
+    super(config);
+  }
 
-  /**
-   * Creates an JoinAddRedundantSemiJoinRule.
-   */
+  @Deprecated // to be removed before 2.0
   public JoinAddRedundantSemiJoinRule(Class<? extends Join> clazz,
       RelBuilderFactory relBuilderFactory) {
-    super(operand(clazz, any()), relBuilderFactory, null);
+    this(Config.DEFAULT.withRelBuilderFactory(relBuilderFactory)
+        .as(Config.class)
+        .withOperandFor(clazz));
   }
 
   //~ Methods ----------------------------------------------------------------
 
-  public void onMatch(RelOptRuleCall call) {
+  @Override public void onMatch(RelOptRuleCall call) {
     Join origJoinRel = call.rel(0);
     if (origJoinRel.isSemiJoinDone()) {
       return;
     }
 
-    // can't process outer joins using semijoins
+    // can't process outer joins using semi-joins
     if (origJoinRel.getJoinType() != JoinRelType.INNER) {
       return;
     }
@@ -92,4 +94,20 @@ public class JoinAddRedundantSemiJoinRule extends RelOptRule
 
     call.transformTo(newJoinRel);
   }
+
+  /** Rule configuration. */
+  public interface Config extends RelRule.Config {
+    Config DEFAULT = EMPTY.as(Config.class)
+        .withOperandFor(LogicalJoin.class);
+
+    @Override default JoinAddRedundantSemiJoinRule toRule() {
+      return new JoinAddRedundantSemiJoinRule(this);
+    }
+
+    /** Defines an operand tree for the given classes. */
+    default Config withOperandFor(Class<? extends Join> joinClass) {
+      return withOperandSupplier(b -> b.operand(joinClass).anyInputs())
+          .as(Config.class);
+    }
+  }
 }
diff --git a/core/src/main/java/org/apache/calcite/rel/rules/JoinAssociateRule.java b/core/src/main/java/org/apache/calcite/rel/rules/JoinAssociateRule.java
index 2eb7c38..e1964b0 100644
--- a/core/src/main/java/org/apache/calcite/rel/rules/JoinAssociateRule.java
+++ b/core/src/main/java/org/apache/calcite/rel/rules/JoinAssociateRule.java
@@ -17,8 +17,8 @@
 package org.apache.calcite.rel.rules;
 
 import org.apache.calcite.plan.RelOptCluster;
-import org.apache.calcite.plan.RelOptRule;
 import org.apache.calcite.plan.RelOptRuleCall;
+import org.apache.calcite.plan.RelRule;
 import org.apache.calcite.plan.volcano.RelSubset;
 import org.apache.calcite.rel.RelNode;
 import org.apache.calcite.rel.core.Join;
@@ -44,31 +44,26 @@ import java.util.List;
  * {@link JoinCommuteRule}.
  *
  * @see JoinCommuteRule
+ * @see CoreRules#JOIN_ASSOCIATE
  */
-public class JoinAssociateRule extends RelOptRule implements TransformationRule {
-  //~ Static fields/initializers ---------------------------------------------
+public class JoinAssociateRule
+    extends RelRule<JoinAssociateRule.Config>
+    implements TransformationRule {
 
-  /** @deprecated Use {@link CoreRules#JOIN_ASSOCIATE}. */
-  @Deprecated // to be removed before 1.25
-  public static final JoinAssociateRule INSTANCE =
-      CoreRules.JOIN_ASSOCIATE;
-
-  //~ Constructors -----------------------------------------------------------
+  /** Creates a JoinAssociateRule. */
+  protected JoinAssociateRule(Config config) {
+    super(config);
+  }
 
-  /**
-   * Creates a JoinAssociateRule.
-   */
+  @Deprecated // to be removed before 2.0
   public JoinAssociateRule(RelBuilderFactory relBuilderFactory) {
-    super(
-        operand(Join.class,
-            operand(Join.class, any()),
-            operand(RelSubset.class, any())),
-        relBuilderFactory, null);
+    this(Config.DEFAULT.withRelBuilderFactory(relBuilderFactory)
+        .as(Config.class));
   }
 
   //~ Methods ----------------------------------------------------------------
 
-  public void onMatch(final RelOptRuleCall call) {
+  @Override public void onMatch(final RelOptRuleCall call) {
     final Join topJoin = call.rel(0);
     final Join bottomJoin = call.rel(1);
     final RelNode relA = bottomJoin.getLeft();
@@ -153,4 +148,24 @@ public class JoinAssociateRule extends RelOptRule implements TransformationRule
 
     call.transformTo(newTopJoin);
   }
+
+  /** Rule configuration. */
+  public interface Config extends RelRule.Config {
+    Config DEFAULT = EMPTY.as(Config.class)
+        .withOperandFor(Join.class, RelSubset.class);
+
+    @Override default JoinAssociateRule toRule() {
+      return new JoinAssociateRule(this);
+    }
+
+    /** Defines an operand tree for the given classes. */
+    default Config withOperandFor(Class<? extends Join> joinClass,
+        Class<? extends RelSubset> relSubsetClass) {
+      return withOperandSupplier(b0 ->
+          b0.operand(joinClass).inputs(
+              b1 -> b1.operand(joinClass).anyInputs(),
+              b2 -> b2.operand(relSubsetClass).anyInputs()))
+          .as(Config.class);
+    }
+  }
 }
diff --git a/core/src/main/java/org/apache/calcite/rel/rules/JoinCommuteRule.java b/core/src/main/java/org/apache/calcite/rel/rules/JoinCommuteRule.java
index 8b66d05..bec08e0 100644
--- a/core/src/main/java/org/apache/calcite/rel/rules/JoinCommuteRule.java
+++ b/core/src/main/java/org/apache/calcite/rel/rules/JoinCommuteRule.java
@@ -17,9 +17,9 @@
 package org.apache.calcite.rel.rules;
 
 import org.apache.calcite.plan.Contexts;
-import org.apache.calcite.plan.RelOptRule;
 import org.apache.calcite.plan.RelOptRuleCall;
 import org.apache.calcite.plan.RelOptUtil;
+import org.apache.calcite.plan.RelRule;
 import org.apache.calcite.rel.RelNode;
 import org.apache.calcite.rel.core.Join;
 import org.apache.calcite.rel.core.JoinRelType;
@@ -34,9 +34,9 @@ import org.apache.calcite.rex.RexNode;
 import org.apache.calcite.rex.RexShuttle;
 import org.apache.calcite.tools.RelBuilder;
 import org.apache.calcite.tools.RelBuilderFactory;
+import org.apache.calcite.util.ImmutableBeans;
 
 import java.util.List;
-import java.util.function.Predicate;
 
 /**
  * Planner rule that permutes the inputs to a
@@ -47,39 +47,26 @@ import java.util.function.Predicate;
  *
  * <p>To preserve the order of columns in the output row, the rule adds a
  * {@link org.apache.calcite.rel.core.Project}.
+ *
+ * @see CoreRules#JOIN_COMMUTE
+ * @see CoreRules#JOIN_COMMUTE_OUTER
  */
-public class JoinCommuteRule extends RelOptRule implements TransformationRule {
-  //~ Static fields/initializers ---------------------------------------------
-
-  /** @deprecated Use {@link CoreRules#JOIN_COMMUTE}. */
-  @Deprecated // to be removed before 1.25
-  public static final JoinCommuteRule INSTANCE = CoreRules.JOIN_COMMUTE;
-
-  /** @deprecated Use {@link CoreRules#JOIN_COMMUTE_OUTER}. */
-  @Deprecated // to be removed before 1.25
-  public static final JoinCommuteRule SWAP_OUTER = CoreRules.JOIN_COMMUTE_OUTER;
+public class JoinCommuteRule
+    extends RelRule<JoinCommuteRule.Config>
+    implements TransformationRule {
 
-  private final boolean swapOuter;
-
-  //~ Constructors -----------------------------------------------------------
+  /** Creates a JoinCommuteRule. */
+  protected JoinCommuteRule(Config config) {
+    super(config);
+  }
 
-  /**
-   * Creates a JoinCommuteRule.
-   */
+  @Deprecated // to be removed before 2.0
   public JoinCommuteRule(Class<? extends Join> clazz,
       RelBuilderFactory relBuilderFactory, boolean swapOuter) {
-    // FIXME Enable this rule for joins with system fields
-    super(
-        operandJ(clazz, null,
-            (Predicate<Join>) j -> j.getLeft().getId() != j.getRight().getId()
-                && j.getSystemFieldList().isEmpty(),
-            any()),
-        relBuilderFactory, null);
-    this.swapOuter = swapOuter;
-  }
-
-  JoinCommuteRule(boolean swapOuter) {
-    this(LogicalJoin.class, RelFactories.LOGICAL_BUILDER, swapOuter);
+    this(Config.DEFAULT.withRelBuilderFactory(relBuilderFactory)
+        .as(Config.class)
+        .withOperandFor(clazz)
+        .withSwapOuter(swapOuter));
   }
 
   @Deprecated // to be removed before 2.0
@@ -136,7 +123,7 @@ public class JoinCommuteRule extends RelOptRule implements TransformationRule {
     // swap.  This way, we will generate one semijoin for the original
     // join, and one for the swapped join, and no more.  This
     // doesn't prevent us from seeing any new combinations assuming
-    // that the planner tries the desired order (semijoins after swaps).
+    // that the planner tries the desired order (semi-joins after swaps).
     Join newJoin =
         join.copy(join.getTraitSet(), condition, join.getRight(),
             join.getLeft(), joinType.swap(), join.isSemiJoinDone());
@@ -147,16 +134,16 @@ public class JoinCommuteRule extends RelOptRule implements TransformationRule {
         .build();
   }
 
-  public boolean matches(RelOptRuleCall call) {
+  @Override public boolean matches(RelOptRuleCall call) {
     Join join = call.rel(0);
     // SEMI and ANTI join cannot be swapped.
     return join.getJoinType().projectsRight();
   }
 
-  public void onMatch(final RelOptRuleCall call) {
+  @Override public void onMatch(final RelOptRuleCall call) {
     Join join = call.rel(0);
 
-    final RelNode swapped = swap(join, this.swapOuter, call.builder());
+    final RelNode swapped = swap(join, config.isSwapOuter(), call.builder());
     if (swapped == null) {
       return;
     }
@@ -229,4 +216,35 @@ public class JoinCommuteRule extends RelOptRule implements TransformationRule {
           + ", rightFieldCount=" + rightFields.size());
     }
   }
+
+  /** Rule configuration. */
+  public interface Config extends RelRule.Config {
+    Config DEFAULT = EMPTY.as(Config.class)
+        .withOperandFor(LogicalJoin.class)
+        .withSwapOuter(false);
+
+    @Override default JoinCommuteRule toRule() {
+      return new JoinCommuteRule(this);
+    }
+
+    /** Defines an operand tree for the given classes. */
+    default Config withOperandFor(Class<? extends Join> joinClass) {
+      return withOperandSupplier(b ->
+          b.operand(joinClass)
+              // FIXME Enable this rule for joins with system fields
+              .predicate(j ->
+                  j.getLeft().getId() != j.getRight().getId()
+                      && j.getSystemFieldList().isEmpty())
+              .anyInputs())
+          .as(Config.class);
+    }
+
+    /** Whether to swap outer joins. */
+    @ImmutableBeans.Property
+    @ImmutableBeans.BooleanDefault(false)
+    boolean isSwapOuter();
+
+    /** Sets {@link #isSwapOuter()}. */
+    Config withSwapOuter(boolean swapOuter);
+  }
 }
diff --git a/core/src/main/java/org/apache/calcite/rel/rules/JoinExtractFilterRule.java b/core/src/main/java/org/apache/calcite/rel/rules/JoinExtractFilterRule.java
index bf6b916..8cdb6fe 100644
--- a/core/src/main/java/org/apache/calcite/rel/rules/JoinExtractFilterRule.java
+++ b/core/src/main/java/org/apache/calcite/rel/rules/JoinExtractFilterRule.java
@@ -17,6 +17,7 @@
 package org.apache.calcite.rel.rules;
 
 import org.apache.calcite.rel.core.Join;
+import org.apache.calcite.rel.logical.LogicalJoin;
 import org.apache.calcite.tools.RelBuilderFactory;
 
 /**
@@ -29,28 +30,37 @@ import org.apache.calcite.tools.RelBuilderFactory;
  * can be combined with conditions and expressions above the join. It also makes
  * the <code>FennelCartesianJoinRule</code> applicable.
  *
- * <p>The constructor is parameterized to allow any sub-class of
+ * <p>Can be configured to match any sub-class of
  * {@link org.apache.calcite.rel.core.Join}, not just
- * {@link org.apache.calcite.rel.logical.LogicalJoin}.</p>
+ * {@link org.apache.calcite.rel.logical.LogicalJoin}.
+ *
+ * @see CoreRules#JOIN_EXTRACT_FILTER
  */
 public final class JoinExtractFilterRule extends AbstractJoinExtractFilterRule {
-  //~ Static fields/initializers ---------------------------------------------
-
-  /** @deprecated Use {@link CoreRules#JOIN_EXTRACT_FILTER}. */
-  @Deprecated // to be removed before 1.25
-  public static final JoinExtractFilterRule INSTANCE =
-      CoreRules.JOIN_EXTRACT_FILTER;
 
-  //~ Constructors -----------------------------------------------------------
+  /** Creates a JoinExtractFilterRule. */
+  protected JoinExtractFilterRule(Config config) {
+    super(config);
+  }
 
-  /**
-   * Creates a JoinExtractFilterRule.
-   */
+  @Deprecated // to be removed before 2.0
   public JoinExtractFilterRule(Class<? extends Join> clazz,
       RelBuilderFactory relBuilderFactory) {
-    super(operand(clazz, any()), relBuilderFactory, null);
+    this(Config.DEFAULT
+        .withRelBuilderFactory(relBuilderFactory)
+        .withOperandSupplier(b ->
+            b.operand(clazz).anyInputs())
+        .as(Config.class));
   }
 
-  //~ Methods ----------------------------------------------------------------
+  /** Rule configuration. */
+  public interface Config extends AbstractJoinExtractFilterRule.Config {
+    Config DEFAULT = EMPTY
+        .withOperandSupplier(b -> b.operand(LogicalJoin.class).anyInputs())
+        .as(Config.class);
 
+    @Override default JoinExtractFilterRule toRule() {
+      return new JoinExtractFilterRule(this);
+    }
+  }
 }
diff --git a/core/src/main/java/org/apache/calcite/rel/rules/JoinProjectTransposeRule.java b/core/src/main/java/org/apache/calcite/rel/rules/JoinProjectTransposeRule.java
index 47bd17b..c790a37 100644
--- a/core/src/main/java/org/apache/calcite/rel/rules/JoinProjectTransposeRule.java
+++ b/core/src/main/java/org/apache/calcite/rel/rules/JoinProjectTransposeRule.java
@@ -17,28 +17,29 @@
 package org.apache.calcite.rel.rules;
 
 import org.apache.calcite.plan.Contexts;
-import org.apache.calcite.plan.RelOptRule;
 import org.apache.calcite.plan.RelOptRuleCall;
 import org.apache.calcite.plan.RelOptRuleOperand;
 import org.apache.calcite.plan.RelOptUtil;
+import org.apache.calcite.plan.RelRule;
 import org.apache.calcite.plan.Strong;
 import org.apache.calcite.rel.RelNode;
 import org.apache.calcite.rel.core.Join;
 import org.apache.calcite.rel.core.JoinRelType;
 import org.apache.calcite.rel.core.Project;
-import org.apache.calcite.rel.core.RelFactories;
 import org.apache.calcite.rel.core.RelFactories.ProjectFactory;
+import org.apache.calcite.rel.logical.LogicalJoin;
+import org.apache.calcite.rel.logical.LogicalProject;
 import org.apache.calcite.rel.type.RelDataType;
 import org.apache.calcite.rel.type.RelDataTypeField;
 import org.apache.calcite.rex.RexBuilder;
 import org.apache.calcite.rex.RexLocalRef;
 import org.apache.calcite.rex.RexNode;
-import org.apache.calcite.rex.RexOver;
 import org.apache.calcite.rex.RexProgram;
 import org.apache.calcite.rex.RexProgramBuilder;
 import org.apache.calcite.sql.validate.SqlValidatorUtil;
 import org.apache.calcite.tools.RelBuilder;
 import org.apache.calcite.tools.RelBuilderFactory;
+import org.apache.calcite.util.ImmutableBeans;
 import org.apache.calcite.util.Pair;
 
 import java.util.ArrayList;
@@ -55,129 +56,108 @@ import java.util.List;
  * {@link org.apache.calcite.rel.logical.LogicalProject} doesn't originate from
  * a null generating input in an outer join.
  */
-public class JoinProjectTransposeRule extends RelOptRule implements TransformationRule {
-  //~ Static fields/initializers ---------------------------------------------
-
-  /** @deprecated Use {@link CoreRules#JOIN_PROJECT_BOTH_TRANSPOSE}. */
-  @Deprecated // to be removed before 1.25
-  public static final JoinProjectTransposeRule BOTH_PROJECT =
-      CoreRules.JOIN_PROJECT_BOTH_TRANSPOSE;
-
-  /** @deprecated Use {@link CoreRules#JOIN_PROJECT_LEFT_TRANSPOSE}. */
-  @Deprecated // to be removed before 1.25
-  public static final JoinProjectTransposeRule LEFT_PROJECT =
-      CoreRules.JOIN_PROJECT_LEFT_TRANSPOSE;
-
-  /** @deprecated Use {@link CoreRules#JOIN_PROJECT_RIGHT_TRANSPOSE}. */
-  @Deprecated // to be removed before 1.25
-  public static final JoinProjectTransposeRule RIGHT_PROJECT =
-      CoreRules.JOIN_PROJECT_RIGHT_TRANSPOSE;
-
-  /** @deprecated Use
-   * {@link CoreRules#JOIN_PROJECT_BOTH_TRANSPOSE_INCLUDE_OUTER}. */
-  @Deprecated // to be removed before 1.25
-  public static final JoinProjectTransposeRule BOTH_PROJECT_INCLUDE_OUTER =
-      CoreRules.JOIN_PROJECT_BOTH_TRANSPOSE_INCLUDE_OUTER;
-
-  /** @deprecated Use
-   * {@link CoreRules#JOIN_PROJECT_LEFT_TRANSPOSE_INCLUDE_OUTER}. */
-  @Deprecated // to be removed before 1.25
-  public static final JoinProjectTransposeRule LEFT_PROJECT_INCLUDE_OUTER =
-      CoreRules.JOIN_PROJECT_LEFT_TRANSPOSE_INCLUDE_OUTER;
-
-  /** @deprecated Use
-   * {@link CoreRules#JOIN_PROJECT_RIGHT_TRANSPOSE_INCLUDE_OUTER}. */
-  @Deprecated // to be removed before 1.25
-  public static final JoinProjectTransposeRule RIGHT_PROJECT_INCLUDE_OUTER =
-      CoreRules.JOIN_PROJECT_RIGHT_TRANSPOSE_INCLUDE_OUTER;
-
-  private final boolean includeOuter;
-
-  //~ Constructors -----------------------------------------------------------
+public class JoinProjectTransposeRule
+    extends RelRule<JoinProjectTransposeRule.Config>
+    implements TransformationRule {
 
   /** Creates a JoinProjectTransposeRule. */
+  protected JoinProjectTransposeRule(Config config) {
+    super(config);
+  }
+
+  @Deprecated // to be removed before 2.0
   public JoinProjectTransposeRule(RelOptRuleOperand operand,
       String description, boolean includeOuter,
       RelBuilderFactory relBuilderFactory) {
-    super(operand, relBuilderFactory, description);
-    this.includeOuter = includeOuter;
+    this(Config.DEFAULT.withDescription(description)
+        .withRelBuilderFactory(relBuilderFactory)
+        .withOperandSupplier(b -> b.exactly(operand))
+        .as(Config.class)
+        .withIncludeOuter(includeOuter));
   }
 
-  /** Creates a JoinProjectTransposeRule with default factory. */
+  @Deprecated // to be removed before 2.0
   public JoinProjectTransposeRule(
       RelOptRuleOperand operand,
       String description) {
-    this(operand, description, false, RelFactories.LOGICAL_BUILDER);
+    this(Config.DEFAULT.withDescription(description)
+        .withOperandSupplier(b -> b.exactly(operand))
+        .as(Config.class));
   }
 
   @Deprecated // to be removed before 2.0
   public JoinProjectTransposeRule(RelOptRuleOperand operand,
       String description, ProjectFactory projectFactory) {
-    this(operand, description, false,
-        RelBuilder.proto(Contexts.of(projectFactory)));
+    this(Config.DEFAULT.withDescription(description)
+        .withRelBuilderFactory(RelBuilder.proto(Contexts.of(projectFactory)))
+        .withOperandSupplier(b -> b.exactly(operand))
+        .as(Config.class));
   }
 
   @Deprecated // to be removed before 2.0
   public JoinProjectTransposeRule(RelOptRuleOperand operand,
       String description, boolean includeOuter,
       ProjectFactory projectFactory) {
-    this(operand, description, includeOuter,
-        RelBuilder.proto(Contexts.of(projectFactory)));
+    this(Config.DEFAULT.withDescription(description)
+        .withRelBuilderFactory(RelBuilder.proto(Contexts.of(projectFactory)))
+        .withOperandSupplier(b -> b.exactly(operand))
+        .as(Config.class)
+        .withIncludeOuter(includeOuter));
   }
 
   //~ Methods ----------------------------------------------------------------
 
-  // implement RelOptRule
-  public void onMatch(RelOptRuleCall call) {
-    Join joinRel = call.rel(0);
-    JoinRelType joinType = joinRel.getJoinType();
+  @Override public void onMatch(RelOptRuleCall call) {
+    final Join join = call.rel(0);
+    final JoinRelType joinType = join.getJoinType();
 
-    Project leftProj;
-    Project rightProj;
+    Project leftProject;
+    Project rightProject;
     RelNode leftJoinChild;
     RelNode rightJoinChild;
 
     // If 1) the rule works on outer joins, or
     //    2) input's projection doesn't generate nulls
+    final boolean includeOuter = config.isIncludeOuter();
     if (hasLeftChild(call)
-            && (includeOuter || !joinType.generatesNullsOnLeft())) {
-      leftProj = call.rel(1);
-      leftJoinChild = getProjectChild(call, leftProj, true);
+        && (includeOuter || !joinType.generatesNullsOnLeft())) {
+      leftProject = call.rel(1);
+      leftJoinChild = getProjectChild(call, leftProject, true);
     } else {
-      leftProj = null;
+      leftProject = null;
       leftJoinChild = call.rel(1);
     }
     if (hasRightChild(call)
-            && (includeOuter || !joinType.generatesNullsOnRight())) {
-      rightProj = getRightChild(call);
-      rightJoinChild = getProjectChild(call, rightProj, false);
+        && (includeOuter || !joinType.generatesNullsOnRight())) {
+      rightProject = getRightChild(call);
+      rightJoinChild = getProjectChild(call, rightProject, false);
     } else {
-      rightProj = null;
-      rightJoinChild = joinRel.getRight();
+      rightProject = null;
+      rightJoinChild = join.getRight();
     }
 
     // Skip projects containing over clause
-    if (leftProj != null && RexOver.containsOver(leftProj.getProjects(), null)) {
-      leftProj = null;
-      leftJoinChild = joinRel.getLeft();
+    if (leftProject != null && leftProject.containsOver()) {
+      leftProject = null;
+      leftJoinChild = join.getLeft();
     }
-    if (rightProj != null && RexOver.containsOver(rightProj.getProjects(), null)) {
-      rightProj = null;
-      rightJoinChild = joinRel.getRight();
+    if (rightProject != null && rightProject.containsOver()) {
+      rightProject = null;
+      rightJoinChild = join.getRight();
     }
 
-    if ((leftProj == null) && (rightProj == null)) {
+    if ((leftProject == null) && (rightProject == null)) {
       return;
     }
 
     if (includeOuter) {
-      if (leftProj != null && joinType.generatesNullsOnLeft()
-          && !Strong.allStrong(leftProj.getProjects())) {
+      if (leftProject != null && joinType.generatesNullsOnLeft()
+          && !Strong.allStrong(leftProject.getProjects())) {
         return;
       }
 
-      if (rightProj != null && joinType.generatesNullsOnRight()
-          && !Strong.allStrong(rightProj.getProjects())) {
+      if (rightProject != null && joinType.generatesNullsOnRight()
+          && !Strong.allStrong(rightProject.getProjects())) {
         return;
       }
     }
@@ -191,12 +171,12 @@ public class JoinProjectTransposeRule extends RelOptRule implements Transformati
     // underneath the projects that feed into the join.  This is the input
     // into the bottom RexProgram.  Note that the join type is an inner
     // join because the inputs haven't actually been joined yet.
-    RelDataType joinChildrenRowType =
+    final RelDataType joinChildrenRowType =
         SqlValidatorUtil.deriveJoinRowType(
             leftJoinChild.getRowType(),
             rightJoinChild.getRowType(),
             JoinRelType.INNER,
-            joinRel.getCluster().getTypeFactory(),
+            join.getCluster().getTypeFactory(),
             null,
             Collections.emptyList());
 
@@ -205,23 +185,23 @@ public class JoinProjectTransposeRule extends RelOptRule implements Transformati
     // expressions, shift them to the right by the number of fields on
     // the LHS.  If the join input was not a projection, simply create
     // references to the inputs.
-    int nProjExprs = joinRel.getRowType().getFieldCount();
+    final int nProjExprs = join.getRowType().getFieldCount();
     final List<Pair<RexNode, String>> projects = new ArrayList<>();
-    final RexBuilder rexBuilder = joinRel.getCluster().getRexBuilder();
+    final RexBuilder rexBuilder = join.getCluster().getRexBuilder();
 
     createProjectExprs(
-        leftProj,
+        leftProject,
         leftJoinChild,
         0,
         rexBuilder,
         joinChildrenRowType.getFieldList(),
         projects);
 
-    List<RelDataTypeField> leftFields =
+    final List<RelDataTypeField> leftFields =
         leftJoinChild.getRowType().getFieldList();
-    int nFieldsLeft = leftFields.size();
+    final int nFieldsLeft = leftFields.size();
     createProjectExprs(
-        rightProj,
+        rightProject,
         rightJoinChild,
         nFieldsLeft,
         rexBuilder,
@@ -238,21 +218,21 @@ public class JoinProjectTransposeRule extends RelOptRule implements Transformati
             Pair.right(projects));
 
     // create the RexPrograms and merge them
-    RexProgram bottomProgram =
+    final RexProgram bottomProgram =
         RexProgram.create(
             joinChildrenRowType,
             Pair.left(projects),
             null,
             projRowType,
             rexBuilder);
-    RexProgramBuilder topProgramBuilder =
+    final RexProgramBuilder topProgramBuilder =
         new RexProgramBuilder(
             projRowType,
             rexBuilder);
     topProgramBuilder.addIdentity();
-    topProgramBuilder.addCondition(joinRel.getCondition());
-    RexProgram topProgram = topProgramBuilder.getProgram();
-    RexProgram mergedProgram =
+    topProgramBuilder.addCondition(join.getCondition());
+    final RexProgram topProgram = topProgramBuilder.getProgram();
+    final RexProgram mergedProgram =
         RexProgramBuilder.mergePrograms(
             topProgram,
             bottomProgram,
... 15637 lines suppressed ...


[calcite] 08/09: [CALCITE-2569] UDFs that are table functions must implement SqlTableFunction and have CURSOR as their return type

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

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

commit 19edf52c76c6a1507721f5bd37f2a33497aa0c4c
Author: Julian Hyde <jh...@apache.org>
AuthorDate: Mon Jun 18 14:10:50 2018 -0700

    [CALCITE-2569] UDFs that are table functions must implement SqlTableFunction and have CURSOR as their return type
    
    Validate that table functions are not used in contexts that require
    scalar functions, such as the SELECT clause.
    
    Change operand type ANY to mean 'scalar expression of any type (but
    not a cursor)', and add operand type IGNORE to mean skip validation -
    for an operand that is not an expression. TABLE is one of the few
    operators that accepts a CURSOR operand.
    
    Add SqlKind.INTERVAL_QUALIFIER.
---
 .../calcite/adapter/enumerable/EnumUtils.java      |  20 ++-
 .../org/apache/calcite/model/ModelHandler.java     |   5 +-
 .../org/apache/calcite/rel/core/RelFactories.java  |  15 ++-
 .../apache/calcite/runtime/CalciteResource.java    |   3 +
 .../schema/impl/ReflectiveFunctionBase.java        |   5 +
 .../calcite/schema/impl/ScalarFunctionImpl.java    |  31 +++++
 .../java/org/apache/calcite/sql/SqlAsOperator.java |   2 +-
 .../org/apache/calcite/sql/SqlCallBinding.java     |  86 ++++++++++++-
 .../apache/calcite/sql/SqlIntervalQualifier.java   |   5 +
 .../main/java/org/apache/calcite/sql/SqlKind.java  |   3 +
 .../java/org/apache/calcite/sql/SqlLiteral.java    |  40 +++---
 .../org/apache/calcite/sql/SqlOperatorBinding.java |  12 ++
 .../org/apache/calcite/sql/SqlOverOperator.java    |   2 +-
 .../org/apache/calcite/sql/SqlTableFunction.java   |  33 +++++
 .../main/java/org/apache/calcite/sql/SqlUtil.java  |  24 ++--
 .../apache/calcite/sql/SqlWindowTableFunction.java |  91 +++++++-------
 .../apache/calcite/sql/SqlWithinGroupOperator.java |   4 +-
 .../sql/fun/SqlArgumentAssignmentOperator.java     |   2 +-
 .../sql/fun/SqlCollectionTableOperator.java        |   2 +-
 .../calcite/sql/type/FamilyOperandTypeChecker.java |  18 ++-
 .../org/apache/calcite/sql/type/OperandTypes.java  |   7 ++
 .../org/apache/calcite/sql/type/SqlTypeFamily.java |   5 +-
 .../apache/calcite/sql/validate/AggVisitor.java    |  22 ++--
 .../calcite/sql/validate/ProcedureNamespace.java   |  27 ++---
 .../sql/validate/SqlUserDefinedTableFunction.java  |  40 +++---
 .../sql/validate/SqlUserDefinedTableMacro.java     | 135 +++++----------------
 .../calcite/sql/validate/SqlValidatorImpl.java     |  21 ++++
 .../apache/calcite/sql2rel/SqlToRelConverter.java  |   5 +-
 .../calcite/runtime/CalciteResource.properties     |   1 +
 .../apache/calcite/test/MockSqlOperatorTable.java  | 111 ++++++++++-------
 .../calcite/test/SqlOperatorBindingTest.java       |  89 +++++++++-----
 .../org/apache/calcite/test/SqlValidatorTest.java  |  39 +++++-
 .../org/apache/calcite/test/TableFunctionTest.java |  32 +++--
 .../apache/calcite/test/ExampleFunctionTest.java   |   2 +-
 34 files changed, 609 insertions(+), 330 deletions(-)

diff --git a/core/src/main/java/org/apache/calcite/adapter/enumerable/EnumUtils.java b/core/src/main/java/org/apache/calcite/adapter/enumerable/EnumUtils.java
index 846e523..87fd196 100644
--- a/core/src/main/java/org/apache/calcite/adapter/enumerable/EnumUtils.java
+++ b/core/src/main/java/org/apache/calcite/adapter/enumerable/EnumUtils.java
@@ -32,6 +32,7 @@ import org.apache.calcite.linq4j.tree.ConstantUntypedNull;
 import org.apache.calcite.linq4j.tree.Expression;
 import org.apache.calcite.linq4j.tree.ExpressionType;
 import org.apache.calcite.linq4j.tree.Expressions;
+import org.apache.calcite.linq4j.tree.FunctionExpression;
 import org.apache.calcite.linq4j.tree.MethodCallExpression;
 import org.apache.calcite.linq4j.tree.MethodDeclaration;
 import org.apache.calcite.linq4j.tree.ParameterExpression;
@@ -321,7 +322,7 @@ public class EnumUtils {
    */
   public static Expression convert(Expression operand, Type toType) {
     final Type fromType = operand.getType();
-    return EnumUtils.convert(operand, fromType, toType);
+    return convert(operand, fromType, toType);
   }
 
   /**
@@ -541,6 +542,23 @@ public class EnumUtils {
     return Expressions.convert_(operand, toType);
   }
 
+  /** Converts a value to a given class. */
+  public static <T> T evaluate(Object o, Class<T> clazz) {
+    // We need optimization here for constant folding.
+    // Not all the expressions can be interpreted (e.g. ternary), so
+    // we rely on optimization capabilities to fold non-interpretable
+    // expressions.
+    //noinspection unchecked
+    clazz = Primitive.box(clazz);
+    BlockBuilder bb = new BlockBuilder();
+    final Expression expr =
+        convert(Expressions.constant(o), clazz);
+    bb.add(Expressions.return_(null, expr));
+    final FunctionExpression convert =
+        Expressions.lambda(bb.toBlock(), ImmutableList.of());
+    return clazz.cast(convert.compile().dynamicInvoke());
+  }
+
   private static boolean isA(Type fromType, Primitive primitive) {
     return Primitive.of(fromType) == primitive
         || Primitive.ofBox(fromType) == primitive;
diff --git a/core/src/main/java/org/apache/calcite/model/ModelHandler.java b/core/src/main/java/org/apache/calcite/model/ModelHandler.java
index edc367c..a674520 100644
--- a/core/src/main/java/org/apache/calcite/model/ModelHandler.java
+++ b/core/src/main/java/org/apache/calcite/model/ModelHandler.java
@@ -24,6 +24,7 @@ import org.apache.calcite.materialize.Lattice;
 import org.apache.calcite.rel.type.RelDataType;
 import org.apache.calcite.rel.type.RelDataTypeFactory;
 import org.apache.calcite.schema.AggregateFunction;
+import org.apache.calcite.schema.Function;
 import org.apache.calcite.schema.ScalarFunction;
 import org.apache.calcite.schema.Schema;
 import org.apache.calcite.schema.SchemaFactory;
@@ -145,8 +146,8 @@ public class ModelHandler {
       return;
     }
     if (methodName != null && methodName.equals("*")) {
-      for (Map.Entry<String, ScalarFunction> entry
-          : ScalarFunctionImpl.createAll(clazz).entries()) {
+      for (Map.Entry<String, Function> entry
+          : ScalarFunctionImpl.functions(clazz).entries()) {
         String name = entry.getKey();
         if (upCase) {
           name = name.toUpperCase(Locale.ROOT);
diff --git a/core/src/main/java/org/apache/calcite/rel/core/RelFactories.java b/core/src/main/java/org/apache/calcite/rel/core/RelFactories.java
index 7b49647..2649dce 100644
--- a/core/src/main/java/org/apache/calcite/rel/core/RelFactories.java
+++ b/core/src/main/java/org/apache/calcite/rel/core/RelFactories.java
@@ -46,9 +46,14 @@ import org.apache.calcite.rel.logical.LogicalUnion;
 import org.apache.calcite.rel.logical.LogicalValues;
 import org.apache.calcite.rel.metadata.RelColumnMapping;
 import org.apache.calcite.rel.type.RelDataType;
+import org.apache.calcite.rex.RexCall;
+import org.apache.calcite.rex.RexCallBinding;
 import org.apache.calcite.rex.RexLiteral;
 import org.apache.calcite.rex.RexNode;
 import org.apache.calcite.sql.SqlKind;
+import org.apache.calcite.sql.SqlOperatorBinding;
+import org.apache.calcite.sql.SqlTableFunction;
+import org.apache.calcite.sql.type.SqlReturnTypeInference;
 import org.apache.calcite.tools.RelBuilder;
 import org.apache.calcite.tools.RelBuilderFactory;
 import org.apache.calcite.util.ImmutableBitSet;
@@ -512,8 +517,16 @@ public class RelFactories {
     @Override public RelNode createTableFunctionScan(RelOptCluster cluster,
         List<RelNode> inputs, RexNode rexCall, Type elementType,
         Set<RelColumnMapping> columnMappings) {
+      final RexCall call = (RexCall) rexCall;
+      final SqlOperatorBinding callBinding =
+          new RexCallBinding(cluster.getTypeFactory(), call.getOperator(),
+              call.operands, ImmutableList.of());
+      final SqlTableFunction operator = (SqlTableFunction) call.getOperator();
+      final SqlReturnTypeInference rowTypeInference =
+          operator.getRowTypeInference();
+      final RelDataType rowType = rowTypeInference.inferReturnType(callBinding);
       return LogicalTableFunctionScan.create(cluster, inputs, rexCall,
-          elementType, rexCall.getType(), columnMappings);
+          elementType, rowType, columnMappings);
     }
   }
 
diff --git a/core/src/main/java/org/apache/calcite/runtime/CalciteResource.java b/core/src/main/java/org/apache/calcite/runtime/CalciteResource.java
index 53fe755..7987546 100644
--- a/core/src/main/java/org/apache/calcite/runtime/CalciteResource.java
+++ b/core/src/main/java/org/apache/calcite/runtime/CalciteResource.java
@@ -554,6 +554,9 @@ public interface CalciteResource {
   ExInst<CalciteException> illegalArgumentForTableFunctionCall(String a0,
       String a1, String a2);
 
+  @BaseMessage("Cannot call table function here: ''{0}''")
+  ExInst<CalciteException> cannotCallTableFunctionHere(String a0);
+
   @BaseMessage("''{0}'' is not a valid datetime format")
   ExInst<CalciteException> invalidDatetimeFormat(String a0);
 
diff --git a/core/src/main/java/org/apache/calcite/schema/impl/ReflectiveFunctionBase.java b/core/src/main/java/org/apache/calcite/schema/impl/ReflectiveFunctionBase.java
index 2ce8160..d01daf7 100644
--- a/core/src/main/java/org/apache/calcite/schema/impl/ReflectiveFunctionBase.java
+++ b/core/src/main/java/org/apache/calcite/schema/impl/ReflectiveFunctionBase.java
@@ -113,6 +113,11 @@ public abstract class ReflectiveFunctionBase implements Function {
       final int ordinal = builder.size();
       builder.add(
           new FunctionParameter() {
+            @Override public String toString() {
+              return ordinal + ": " + name + " " + type.getSimpleName()
+                  + (optional ? "?" : "");
+            }
+
             public int getOrdinal() {
               return ordinal;
             }
diff --git a/core/src/main/java/org/apache/calcite/schema/impl/ScalarFunctionImpl.java b/core/src/main/java/org/apache/calcite/schema/impl/ScalarFunctionImpl.java
index 1322607..4c2dc01 100644
--- a/core/src/main/java/org/apache/calcite/schema/impl/ScalarFunctionImpl.java
+++ b/core/src/main/java/org/apache/calcite/schema/impl/ScalarFunctionImpl.java
@@ -24,8 +24,10 @@ import org.apache.calcite.linq4j.function.SemiStrict;
 import org.apache.calcite.linq4j.function.Strict;
 import org.apache.calcite.rel.type.RelDataType;
 import org.apache.calcite.rel.type.RelDataTypeFactory;
+import org.apache.calcite.schema.Function;
 import org.apache.calcite.schema.ImplementableFunction;
 import org.apache.calcite.schema.ScalarFunction;
+import org.apache.calcite.schema.TableFunction;
 import org.apache.calcite.sql.SqlOperatorBinding;
 
 import com.google.common.collect.ImmutableMultimap;
@@ -52,6 +54,7 @@ public class ScalarFunctionImpl extends ReflectiveFunctionBase
    * Creates {@link org.apache.calcite.schema.ScalarFunction} for each method in
    * a given class.
    */
+  @Deprecated // to be removed before 2.0
   public static ImmutableMultimap<String, ScalarFunction> createAll(
       Class<?> clazz) {
     final ImmutableMultimap.Builder<String, ScalarFunction> builder =
@@ -71,6 +74,34 @@ public class ScalarFunctionImpl extends ReflectiveFunctionBase
   }
 
   /**
+   * Returns a map of all functions based on the methods in a given class.
+   * It is keyed by method names and maps to both
+   * {@link org.apache.calcite.schema.ScalarFunction}
+   * and {@link org.apache.calcite.schema.TableFunction}.
+   */
+  public static ImmutableMultimap<String, Function> functions(Class<?> clazz) {
+    final ImmutableMultimap.Builder<String, Function> builder =
+        ImmutableMultimap.builder();
+    for (Method method : clazz.getMethods()) {
+      if (method.getDeclaringClass() == Object.class) {
+        continue;
+      }
+      if (!Modifier.isStatic(method.getModifiers())
+          && !classHasPublicZeroArgsConstructor(clazz)) {
+        continue;
+      }
+      final TableFunction tableFunction = TableFunctionImpl.create(method);
+      if (tableFunction != null) {
+        builder.put(method.getName(), tableFunction);
+      } else {
+        final ScalarFunction function = create(method);
+        builder.put(method.getName(), function);
+      }
+    }
+    return builder.build();
+  }
+
+  /**
    * Creates {@link org.apache.calcite.schema.ScalarFunction} from given class.
    *
    * <p>If a method of the given name is not found or it does not suit,
diff --git a/core/src/main/java/org/apache/calcite/sql/SqlAsOperator.java b/core/src/main/java/org/apache/calcite/sql/SqlAsOperator.java
index e90340d..514c29a 100644
--- a/core/src/main/java/org/apache/calcite/sql/SqlAsOperator.java
+++ b/core/src/main/java/org/apache/calcite/sql/SqlAsOperator.java
@@ -51,7 +51,7 @@ public class SqlAsOperator extends SqlSpecialOperator {
         true,
         ReturnTypes.ARG0,
         InferTypes.RETURN_TYPE,
-        OperandTypes.ANY_ANY);
+        OperandTypes.ANY_IGNORE);
   }
 
   protected SqlAsOperator(String name, SqlKind kind, int prec,
diff --git a/core/src/main/java/org/apache/calcite/sql/SqlCallBinding.java b/core/src/main/java/org/apache/calcite/sql/SqlCallBinding.java
index bd70dd4..d944812 100644
--- a/core/src/main/java/org/apache/calcite/sql/SqlCallBinding.java
+++ b/core/src/main/java/org/apache/calcite/sql/SqlCallBinding.java
@@ -16,11 +16,15 @@
  */
 package org.apache.calcite.sql;
 
+import org.apache.calcite.adapter.enumerable.EnumUtils;
 import org.apache.calcite.rel.type.RelDataType;
+import org.apache.calcite.rel.type.RelDataTypeFactoryImpl;
 import org.apache.calcite.runtime.CalciteException;
 import org.apache.calcite.runtime.Resources;
+import org.apache.calcite.sql.fun.SqlLiteralChainOperator;
 import org.apache.calcite.sql.fun.SqlStdOperatorTable;
 import org.apache.calcite.sql.parser.SqlParserPos;
+import org.apache.calcite.sql.type.SqlTypeName;
 import org.apache.calcite.sql.validate.SelectScope;
 import org.apache.calcite.sql.validate.SqlMonotonicity;
 import org.apache.calcite.sql.validate.SqlValidator;
@@ -28,12 +32,16 @@ import org.apache.calcite.sql.validate.SqlValidatorException;
 import org.apache.calcite.sql.validate.SqlValidatorNamespace;
 import org.apache.calcite.sql.validate.SqlValidatorScope;
 import org.apache.calcite.sql.validate.SqlValidatorUtil;
+import org.apache.calcite.util.ImmutableNullableList;
 import org.apache.calcite.util.NlsString;
 
+import com.google.common.collect.ImmutableMap;
 import com.google.common.collect.Lists;
 
 import java.math.BigDecimal;
+import java.util.ArrayList;
 import java.util.List;
+import java.util.Objects;
 
 import static org.apache.calcite.util.Static.RESOURCE;
 
@@ -212,20 +220,86 @@ public class SqlCallBinding extends SqlOperatorBinding {
   }
 
   @Override public <T> T getOperandLiteralValue(int ordinal, Class<T> clazz) {
-    try {
-      final SqlNode node = call.operand(ordinal);
-      return SqlLiteral.unchain(node).getValueAs(clazz);
-    } catch (IllegalArgumentException e) {
+    final SqlNode node = operand(ordinal);
+    return valueAs(node, clazz);
+  }
+
+  @Override public Object getOperandLiteralValue(int ordinal, RelDataType type) {
+    if (!(type instanceof RelDataTypeFactoryImpl.JavaType)) {
       return null;
     }
+    final Class<?> clazz = ((RelDataTypeFactoryImpl.JavaType) type).getJavaClass();
+    final Object o = getOperandLiteralValue(ordinal, Object.class);
+    if (o == null) {
+      return null;
+    }
+    if (clazz.isInstance(o)) {
+      return clazz.cast(o);
+    }
+    final Object o2 = o instanceof NlsString ? ((NlsString) o).getValue() : o;
+    return EnumUtils.evaluate(o2, clazz);
+  }
+
+  private <T> T valueAs(SqlNode node, Class<T> clazz) {
+    final SqlLiteral literal;
+    switch (node.getKind()) {
+    case ARRAY_VALUE_CONSTRUCTOR:
+      final List<Object> list = new ArrayList<>();
+      for (SqlNode o : ((SqlCall) node).getOperandList()) {
+        list.add(valueAs(o, Object.class));
+      }
+      return clazz.cast(ImmutableNullableList.copyOf(list));
+
+    case MAP_VALUE_CONSTRUCTOR:
+      final ImmutableMap.Builder<Object, Object> builder2 =
+          ImmutableMap.builder();
+      final List<SqlNode> operands = ((SqlCall) node).getOperandList();
+      for (int i = 0; i < operands.size(); i += 2) {
+        final SqlNode key = operands.get(i);
+        final SqlNode value = operands.get(i + 1);
+        builder2.put(Objects.requireNonNull(valueAs(key, Object.class)),
+            Objects.requireNonNull(valueAs(value, Object.class)));
+      }
+      return clazz.cast(builder2.build());
+
+    case CAST:
+      return valueAs(((SqlCall) node).operand(0), clazz);
+
+    case LITERAL:
+      literal = (SqlLiteral) node;
+      if (literal.getTypeName() == SqlTypeName.NULL) {
+        return null;
+      }
+      return literal.getValueAs(clazz);
+
+    case LITERAL_CHAIN:
+      literal = SqlLiteralChainOperator.concatenateOperands((SqlCall) node);
+      return literal.getValueAs(clazz);
+
+    case INTERVAL_QUALIFIER:
+      final SqlIntervalQualifier q = (SqlIntervalQualifier) node;
+      final SqlIntervalLiteral.IntervalValue intervalValue =
+          new SqlIntervalLiteral.IntervalValue(q, 1, q.toString());
+      literal = new SqlLiteral(intervalValue, q.typeName(), q.pos);
+      return literal.getValueAs(clazz);
+
+    case DEFAULT:
+      return null; // currently NULL is the only default value
+
+    default:
+      if (SqlUtil.isNullLiteral(node, true)) {
+        return null; // NULL literal
+      }
+      return null; // not a literal
+    }
   }
 
   @Override public boolean isOperandNull(int ordinal, boolean allowCast) {
-    return SqlUtil.isNullLiteral(call.operand(ordinal), allowCast);
+    return SqlUtil.isNullLiteral(operand(ordinal), allowCast);
   }
 
   @Override public boolean isOperandLiteral(int ordinal, boolean allowCast) {
-    return SqlUtil.isLiteral(call.operand(ordinal), allowCast);
+    return SqlUtil.isLiteral(operand(ordinal), allowCast);
   }
 
   @Override public int getOperandCount() {
diff --git a/core/src/main/java/org/apache/calcite/sql/SqlIntervalQualifier.java b/core/src/main/java/org/apache/calcite/sql/SqlIntervalQualifier.java
index 032b5db..e0d0cd9 100644
--- a/core/src/main/java/org/apache/calcite/sql/SqlIntervalQualifier.java
+++ b/core/src/main/java/org/apache/calcite/sql/SqlIntervalQualifier.java
@@ -33,6 +33,7 @@ import java.math.BigDecimal;
 import java.util.Objects;
 import java.util.regex.Matcher;
 import java.util.regex.Pattern;
+import javax.annotation.Nonnull;
 
 import static org.apache.calcite.util.Static.RESOURCE;
 
@@ -127,6 +128,10 @@ public class SqlIntervalQualifier extends SqlNode {
 
   //~ Methods ----------------------------------------------------------------
 
+  @Nonnull @Override public SqlKind getKind() {
+    return SqlKind.INTERVAL_QUALIFIER;
+  }
+
   public SqlTypeName typeName() {
     switch (timeUnitRange) {
     case YEAR:
diff --git a/core/src/main/java/org/apache/calcite/sql/SqlKind.java b/core/src/main/java/org/apache/calcite/sql/SqlKind.java
index 1fee9bf..caaa6ba 100644
--- a/core/src/main/java/org/apache/calcite/sql/SqlKind.java
+++ b/core/src/main/java/org/apache/calcite/sql/SqlKind.java
@@ -120,6 +120,9 @@ public enum SqlKind {
   /** A literal. */
   LITERAL,
 
+  /** Interval qualifier. */
+  INTERVAL_QUALIFIER,
+
   /**
    * Function that is not a special function.
    *
diff --git a/core/src/main/java/org/apache/calcite/sql/SqlLiteral.java b/core/src/main/java/org/apache/calcite/sql/SqlLiteral.java
index 7ce5fa1..c3b7981 100644
--- a/core/src/main/java/org/apache/calcite/sql/SqlLiteral.java
+++ b/core/src/main/java/org/apache/calcite/sql/SqlLiteral.java
@@ -17,6 +17,7 @@
 package org.apache.calcite.sql;
 
 import org.apache.calcite.avatica.util.TimeUnitRange;
+import org.apache.calcite.rel.metadata.NullSentinel;
 import org.apache.calcite.rel.type.RelDataType;
 import org.apache.calcite.rel.type.RelDataTypeFactory;
 import org.apache.calcite.sql.fun.SqlLiteralChainOperator;
@@ -43,6 +44,7 @@ import java.nio.charset.Charset;
 import java.nio.charset.UnsupportedCharsetException;
 import java.util.Calendar;
 import java.util.Objects;
+import javax.annotation.Nonnull;
 
 import static org.apache.calcite.util.Static.RESOURCE;
 
@@ -234,7 +236,7 @@ public class SqlLiteral extends SqlNode {
     return new SqlLiteral(value, typeName, pos);
   }
 
-  public SqlKind getKind() {
+  public @Nonnull SqlKind getKind() {
     return SqlKind.LITERAL;
   }
 
@@ -253,24 +255,33 @@ public class SqlLiteral extends SqlNode {
   }
 
   /**
-   * Returns the value of this literal as a particular type.
+   * Returns the value of this literal as a given Java type.
    *
-   * <p>The type might be the internal type, or other convenient types.
-   * For example, numeric literals' values are stored internally as
+   * <p>Which type you may ask for depends on {@link #typeName}.
+   * You may always ask for the type where we store the value internally
+   * (as defined by {@link #valueMatchesType(Object, SqlTypeName)}), but may
+   * ask for other convenient types.
+   *
+   * <p>For example, numeric literals' values are stored internally as
    * {@link BigDecimal}, but other numeric types such as {@link Long} and
    * {@link Double} are also allowed.
    *
+   * <p>The result is never null. For the NULL literal, returns
+   * a {@link NullSentinel#INSTANCE}.
+   *
    * @param clazz Desired value type
    * @param <T> Value type
-   * @return Value of the literal
+   * @return Value of the literal in desired type, never null
    *
    * @throws AssertionError if the value type is not supported
    */
-  public <T> T getValueAs(Class<T> clazz) {
+  @Nonnull public <T> T getValueAs(Class<T> clazz) {
     if (clazz.isInstance(value)) {
       return clazz.cast(value);
     }
     switch (typeName) {
+    case NULL:
+      return clazz.cast(NullSentinel.INSTANCE);
     case CHAR:
       if (clazz == String.class) {
         return clazz.cast(((NlsString) value).getValue());
@@ -445,11 +456,10 @@ public class SqlLiteral extends SqlNode {
       assert SqlTypeUtil.inCharFamily(literal.getTypeName());
       return (NlsString) literal.value;
     }
-    if (node instanceof SqlIntervalQualifier) {
-      SqlIntervalQualifier qualifier = (SqlIntervalQualifier) node;
-      return qualifier.timeUnitRange;
-    }
     switch (node.getKind()) {
+    case INTERVAL_QUALIFIER:
+      //noinspection ConstantConditions
+      return ((SqlIntervalQualifier) node).timeUnitRange;
     case CAST:
       assert node instanceof SqlCall;
       return value(((SqlCall) node).operand(0));
@@ -485,7 +495,6 @@ public class SqlLiteral extends SqlNode {
       return literal.value.toString();
     } else if (node instanceof SqlCall
         && ((SqlCall) node).getOperator() == SqlStdOperatorTable.CAST) {
-      //noinspection deprecation
       return stringValue(((SqlCall) node).operand(0));
     } else {
       throw new AssertionError("invalid string literal: " + node);
@@ -499,16 +508,17 @@ public class SqlLiteral extends SqlNode {
    * and cannot be unchained.
    */
   public static SqlLiteral unchain(SqlNode node) {
-    if (node instanceof SqlLiteral) {
+    switch (node.getKind()) {
+    case LITERAL:
       return (SqlLiteral) node;
-    } else if (SqlUtil.isLiteralChain(node)) {
+    case LITERAL_CHAIN:
       return SqlLiteralChainOperator.concatenateOperands((SqlCall) node);
-    } else if (node instanceof SqlIntervalQualifier) {
+    case INTERVAL_QUALIFIER:
       final SqlIntervalQualifier q = (SqlIntervalQualifier) node;
       return new SqlLiteral(
           new SqlIntervalLiteral.IntervalValue(q, 1, q.toString()),
           q.typeName(), q.pos);
-    } else {
+    default:
       throw new IllegalArgumentException("invalid literal: " + node);
     }
   }
diff --git a/core/src/main/java/org/apache/calcite/sql/SqlOperatorBinding.java b/core/src/main/java/org/apache/calcite/sql/SqlOperatorBinding.java
index 21d1fd9..aabf09e 100644
--- a/core/src/main/java/org/apache/calcite/sql/SqlOperatorBinding.java
+++ b/core/src/main/java/org/apache/calcite/sql/SqlOperatorBinding.java
@@ -135,6 +135,18 @@ public abstract class SqlOperatorBinding {
     throw new UnsupportedOperationException();
   }
 
+  /**
+   * Gets the value of a literal operand as a Calcite type.
+   *
+   * @param ordinal zero-based ordinal of operand of interest
+   * @param type Desired valued type
+   *
+   * @return value of operand
+   */
+  public Object getOperandLiteralValue(int ordinal, RelDataType type) {
+    throw new UnsupportedOperationException();
+  }
+
   @Deprecated // to be removed before 2.0
   public Comparable getOperandLiteralValue(int ordinal) {
     return getOperandLiteralValue(ordinal, Comparable.class);
diff --git a/core/src/main/java/org/apache/calcite/sql/SqlOverOperator.java b/core/src/main/java/org/apache/calcite/sql/SqlOverOperator.java
index 5f6cb3f..99b7a63 100644
--- a/core/src/main/java/org/apache/calcite/sql/SqlOverOperator.java
+++ b/core/src/main/java/org/apache/calcite/sql/SqlOverOperator.java
@@ -52,7 +52,7 @@ public class SqlOverOperator extends SqlBinaryOperator {
         true,
         ReturnTypes.ARG0_FORCE_NULLABLE,
         null,
-        OperandTypes.ANY_ANY);
+        OperandTypes.ANY_IGNORE);
   }
 
   //~ Methods ----------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/sql/SqlTableFunction.java b/core/src/main/java/org/apache/calcite/sql/SqlTableFunction.java
new file mode 100644
index 0000000..af79dd5
--- /dev/null
+++ b/core/src/main/java/org/apache/calcite/sql/SqlTableFunction.java
@@ -0,0 +1,33 @@
+/*
+ * 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.sql;
+
+import org.apache.calcite.sql.type.SqlReturnTypeInference;
+
+/**
+ * A function that returns a table.
+ */
+public interface SqlTableFunction {
+  /**
+   * Returns the record type of the table yielded by this function when
+   * applied to given arguments. Only literal arguments are passed,
+   * non-literal are replaced with default values (null, 0, false, etc).
+   *
+   * @return strategy to infer the row type of a call to this function
+   */
+  SqlReturnTypeInference getRowTypeInference();
+}
diff --git a/core/src/main/java/org/apache/calcite/sql/SqlUtil.java b/core/src/main/java/org/apache/calcite/sql/SqlUtil.java
index 18dc6bb..9bfdd92 100644
--- a/core/src/main/java/org/apache/calcite/sql/SqlUtil.java
+++ b/core/src/main/java/org/apache/calcite/sql/SqlUtil.java
@@ -216,16 +216,22 @@ public abstract class SqlUtil {
     if (node instanceof SqlLiteral) {
       return true;
     }
-    if (allowCast) {
-      if (node.getKind() == SqlKind.CAST) {
-        SqlCall call = (SqlCall) node;
-        if (isLiteral(call.operand(0), false)) {
-          // node is "CAST(literal as type)"
-          return true;
-        }
-      }
+    if (!allowCast) {
+      return false;
+    }
+    switch (node.getKind()) {
+    case CAST:
+      // "CAST(e AS type)" is literal if "e" is literal
+      return isLiteral(((SqlCall) node).operand(0), true);
+    case MAP_VALUE_CONSTRUCTOR:
+    case ARRAY_VALUE_CONSTRUCTOR:
+      return ((SqlCall) node).getOperandList().stream()
+          .allMatch(o -> isLiteral(o, true));
+    case DEFAULT:
+      return true; // DEFAULT is always NULL
+    default:
+      return false;
     }
-    return false;
   }
 
   /**
diff --git a/core/src/main/java/org/apache/calcite/sql/SqlWindowTableFunction.java b/core/src/main/java/org/apache/calcite/sql/SqlWindowTableFunction.java
index 309efd5..f17a383 100644
--- a/core/src/main/java/org/apache/calcite/sql/SqlWindowTableFunction.java
+++ b/core/src/main/java/org/apache/calcite/sql/SqlWindowTableFunction.java
@@ -17,31 +17,46 @@
 package org.apache.calcite.sql;
 
 import org.apache.calcite.rel.type.RelDataType;
-import org.apache.calcite.rel.type.RelDataTypeField;
-import org.apache.calcite.rel.type.RelDataTypeFieldImpl;
-import org.apache.calcite.rel.type.RelRecordType;
+import org.apache.calcite.rel.type.RelDataTypeFactory;
+import org.apache.calcite.sql.type.ReturnTypes;
 import org.apache.calcite.sql.type.SqlReturnTypeInference;
 import org.apache.calcite.sql.type.SqlTypeName;
+import org.apache.calcite.sql.validate.SqlNameMatcher;
 import org.apache.calcite.sql.validate.SqlValidator;
 
-import java.util.ArrayList;
 import java.util.List;
 
 import static org.apache.calcite.util.Static.RESOURCE;
 
 /**
- * Base class for table-valued function windowing operator (TUMBLE, HOP and SESSION).
+ * Base class for a table-valued function that computes windows. Examples
+ * include {@code TUMBLE}, {@code HOP} and {@code SESSION}.
  */
-public class SqlWindowTableFunction extends SqlFunction {
+public class SqlWindowTableFunction extends SqlFunction
+    implements SqlTableFunction {
+  /**
+   * Type-inference strategy whereby the row type of a table function call is a
+   * ROW, which is combined from the row type of operand #0 (which is a TABLE)
+   * and two additional fields. The fields are as follows:
+   *
+   * <ol>
+   *  <li>{@code window_start}: TIMESTAMP type to indicate a window's start
+   *  <li>{@code window_end}: TIMESTAMP type to indicate a window's end
+   * </ol>
+   */
+  public static final SqlReturnTypeInference ARG0_TABLE_FUNCTION_WINDOWING =
+      SqlWindowTableFunction::inferRowType;
+
+  /** Creates a window table function with a given name. */
   public SqlWindowTableFunction(String name) {
-    super(name,
-        SqlKind.OTHER_FUNCTION,
-        ARG0_TABLE_FUNCTION_WINDOWING,
-        null,
-        null,
+    super(name, SqlKind.OTHER_FUNCTION, ReturnTypes.CURSOR, null, null,
         SqlFunctionCategory.SYSTEM);
   }
 
+  @Override public SqlReturnTypeInference getRowTypeInference() {
+    return ARG0_TABLE_FUNCTION_WINDOWING;
+  }
+
   protected boolean throwValidationSignatureErrorOrReturnFalse(SqlCallBinding callBinding,
       boolean throwOnFailure) {
     if (throwOnFailure) {
@@ -53,16 +68,10 @@ public class SqlWindowTableFunction extends SqlFunction {
 
   protected void validateColumnNames(SqlValidator validator,
       List<String> fieldNames, List<SqlNode> unvalidatedColumnNames) {
-    for (SqlNode descOperand: unvalidatedColumnNames) {
+    final SqlNameMatcher matcher = validator.getCatalogReader().nameMatcher();
+    for (SqlNode descOperand : unvalidatedColumnNames) {
       final String colName = ((SqlIdentifier) descOperand).getSimple();
-      boolean matches = false;
-      for (String field : fieldNames) {
-        if (validator.getCatalogReader().nameMatcher().matches(field, colName)) {
-          matches = true;
-          break;
-        }
-      }
-      if (!matches) {
+      if (matcher.frequency(fieldNames, colName) == 0) {
         throw SqlUtil.newContextException(descOperand.getParserPosition(),
             RESOURCE.unknownIdentifier(colName));
       }
@@ -70,7 +79,9 @@ public class SqlWindowTableFunction extends SqlFunction {
   }
 
   /**
-   * Overrides SqlOperator.argumentMustBeScalar because the first parameter of
+   * {@inheritDoc}
+   *
+   * <p>Overrides because the first parameter of
    * table-value function windowing is an explicit TABLE parameter,
    * which is not scalar.
    */
@@ -78,29 +89,17 @@ public class SqlWindowTableFunction extends SqlFunction {
     return ordinal != 0;
   }
 
-  /**
-   * Type-inference strategy whereby the result type of a table function call is
-   * a ROW, which is combined from the operand #0(TABLE parameter)'s schema and
-   * two additional fields. The fields are as follows:
-   *
-   * <ol>
-   *  <li>window_start: TIMESTAMP type to indicate a window's start.</li>
-   *  <li>window_end: TIMESTAMP type to indicate a window's end.</li>
-   * </ol>
-   */
-  public static final SqlReturnTypeInference ARG0_TABLE_FUNCTION_WINDOWING =
-      opBinding -> {
-        RelDataType inputRowType = opBinding.getOperandType(0);
-        List<RelDataTypeField> newFields = new ArrayList<>(inputRowType.getFieldList());
-        RelDataType timestampType = opBinding.getTypeFactory().createSqlType(SqlTypeName.TIMESTAMP);
-
-        RelDataTypeField windowStartField =
-            new RelDataTypeFieldImpl("window_start", newFields.size(), timestampType);
-        newFields.add(windowStartField);
-        RelDataTypeField windowEndField =
-            new RelDataTypeFieldImpl("window_end", newFields.size(), timestampType);
-        newFields.add(windowEndField);
-
-        return new RelRecordType(inputRowType.getStructKind(), newFields);
-      };
+  /** Helper for {@link #ARG0_TABLE_FUNCTION_WINDOWING}. */
+  private static RelDataType inferRowType(SqlOperatorBinding opBinding) {
+    final RelDataType inputRowType = opBinding.getOperandType(0);
+    final RelDataTypeFactory typeFactory = opBinding.getTypeFactory();
+    final RelDataType timestampType =
+        typeFactory.createSqlType(SqlTypeName.TIMESTAMP);
+    return typeFactory.builder()
+        .kind(inputRowType.getStructKind())
+        .addAll(inputRowType.getFieldList())
+        .add("window_start", timestampType)
+        .add("window_end", timestampType)
+        .build();
+  }
 }
diff --git a/core/src/main/java/org/apache/calcite/sql/SqlWithinGroupOperator.java b/core/src/main/java/org/apache/calcite/sql/SqlWithinGroupOperator.java
index b20fee9..d5402a6 100644
--- a/core/src/main/java/org/apache/calcite/sql/SqlWithinGroupOperator.java
+++ b/core/src/main/java/org/apache/calcite/sql/SqlWithinGroupOperator.java
@@ -38,7 +38,7 @@ public class SqlWithinGroupOperator extends SqlBinaryOperator {
 
   public SqlWithinGroupOperator() {
     super("WITHIN GROUP", SqlKind.WITHIN_GROUP, 100, true, ReturnTypes.ARG0,
-        null, OperandTypes.ANY_ANY);
+        null, OperandTypes.ANY_IGNORE);
   }
 
   @Override public void unparse(SqlWriter writer, SqlCall call, int leftPrec, int rightPrec) {
@@ -48,7 +48,7 @@ public class SqlWithinGroupOperator extends SqlBinaryOperator {
     final SqlWriter.Frame orderFrame =
         writer.startList(SqlWriter.FrameTypeEnum.ORDER_BY_LIST, "(", ")");
     writer.keyword("ORDER BY");
-    ((SqlNodeList) call.operand(1)).unparse(writer, 0, 0);
+    call.operand(1).unparse(writer, 0, 0);
     writer.endList(orderFrame);
   }
 
diff --git a/core/src/main/java/org/apache/calcite/sql/fun/SqlArgumentAssignmentOperator.java b/core/src/main/java/org/apache/calcite/sql/fun/SqlArgumentAssignmentOperator.java
index b5298d9..f303eda 100644
--- a/core/src/main/java/org/apache/calcite/sql/fun/SqlArgumentAssignmentOperator.java
+++ b/core/src/main/java/org/apache/calcite/sql/fun/SqlArgumentAssignmentOperator.java
@@ -37,7 +37,7 @@ import org.apache.calcite.sql.type.ReturnTypes;
 class SqlArgumentAssignmentOperator extends SqlAsOperator {
   SqlArgumentAssignmentOperator() {
     super("=>", SqlKind.ARGUMENT_ASSIGNMENT, 20, true, ReturnTypes.ARG0,
-        InferTypes.RETURN_TYPE, OperandTypes.ANY_ANY);
+        InferTypes.RETURN_TYPE, OperandTypes.ANY_IGNORE);
   }
 
   @Override public void unparse(SqlWriter writer, SqlCall call, int leftPrec,
diff --git a/core/src/main/java/org/apache/calcite/sql/fun/SqlCollectionTableOperator.java b/core/src/main/java/org/apache/calcite/sql/fun/SqlCollectionTableOperator.java
index 95ac447..83dddc7 100644
--- a/core/src/main/java/org/apache/calcite/sql/fun/SqlCollectionTableOperator.java
+++ b/core/src/main/java/org/apache/calcite/sql/fun/SqlCollectionTableOperator.java
@@ -37,7 +37,7 @@ public class SqlCollectionTableOperator extends SqlFunctionalOperator {
 
   public SqlCollectionTableOperator(String name, SqlModality modality) {
     super(name, SqlKind.COLLECTION_TABLE, 200, true, ReturnTypes.ARG0, null,
-        OperandTypes.ANY);
+        OperandTypes.CURSOR);
     this.modality = modality;
   }
 
diff --git a/core/src/main/java/org/apache/calcite/sql/type/FamilyOperandTypeChecker.java b/core/src/main/java/org/apache/calcite/sql/type/FamilyOperandTypeChecker.java
index a37848d..de9e186 100644
--- a/core/src/main/java/org/apache/calcite/sql/type/FamilyOperandTypeChecker.java
+++ b/core/src/main/java/org/apache/calcite/sql/type/FamilyOperandTypeChecker.java
@@ -65,8 +65,22 @@ public class FamilyOperandTypeChecker implements SqlSingleOperandTypeChecker,
       SqlNode node,
       int iFormalOperand,
       boolean throwOnFailure) {
-    SqlTypeFamily family = families.get(iFormalOperand);
-    if (family == SqlTypeFamily.ANY) {
+    final SqlTypeFamily family = families.get(iFormalOperand);
+    switch (family) {
+    case ANY:
+      final RelDataType type = callBinding.getValidator()
+          .deriveType(callBinding.getScope(), node);
+      SqlTypeName typeName = type.getSqlTypeName();
+
+      if (typeName == SqlTypeName.CURSOR) {
+        // We do not allow CURSOR operands, even for ANY
+        if (throwOnFailure) {
+          throw callBinding.newValidationSignatureError();
+        }
+        return false;
+      }
+      // fall through
+    case IGNORE:
       // no need to check
       return true;
     }
diff --git a/core/src/main/java/org/apache/calcite/sql/type/OperandTypes.java b/core/src/main/java/org/apache/calcite/sql/type/OperandTypes.java
index f23cd2a..37d0250 100644
--- a/core/src/main/java/org/apache/calcite/sql/type/OperandTypes.java
+++ b/core/src/main/java/org/apache/calcite/sql/type/OperandTypes.java
@@ -441,9 +441,16 @@ public abstract class OperandTypes {
 
   public static final SqlSingleOperandTypeChecker ANY_ANY =
       family(SqlTypeFamily.ANY, SqlTypeFamily.ANY);
+  public static final SqlSingleOperandTypeChecker ANY_IGNORE =
+      family(SqlTypeFamily.ANY, SqlTypeFamily.IGNORE);
+  public static final SqlSingleOperandTypeChecker IGNORE_ANY =
+      family(SqlTypeFamily.IGNORE, SqlTypeFamily.ANY);
   public static final SqlSingleOperandTypeChecker ANY_NUMERIC =
       family(SqlTypeFamily.ANY, SqlTypeFamily.NUMERIC);
 
+  public static final SqlSingleOperandTypeChecker CURSOR =
+      family(SqlTypeFamily.CURSOR);
+
   /**
    * Parameter type-checking strategy where type must a nullable time interval,
    * nullable time interval.
diff --git a/core/src/main/java/org/apache/calcite/sql/type/SqlTypeFamily.java b/core/src/main/java/org/apache/calcite/sql/type/SqlTypeFamily.java
index 3a689c5..1666e83 100644
--- a/core/src/main/java/org/apache/calcite/sql/type/SqlTypeFamily.java
+++ b/core/src/main/java/org/apache/calcite/sql/type/SqlTypeFamily.java
@@ -72,7 +72,10 @@ public enum SqlTypeFamily implements RelDataTypeFamily {
   ANY,
   CURSOR,
   COLUMN_LIST,
-  GEO;
+  GEO,
+  /** Like ANY, but do not even validate the operand. It may not be an
+   * expression. */
+  IGNORE;
 
   private static final Map<Integer, SqlTypeFamily> JDBC_TYPE_TO_FAMILY =
       ImmutableMap.<Integer, SqlTypeFamily>builder()
diff --git a/core/src/main/java/org/apache/calcite/sql/validate/AggVisitor.java b/core/src/main/java/org/apache/calcite/sql/validate/AggVisitor.java
index 10c384a..3f63e34 100644
--- a/core/src/main/java/org/apache/calcite/sql/validate/AggVisitor.java
+++ b/core/src/main/java/org/apache/calcite/sql/validate/AggVisitor.java
@@ -18,6 +18,7 @@ package org.apache.calcite.sql.validate;
 
 import org.apache.calcite.sql.SqlCall;
 import org.apache.calcite.sql.SqlFunction;
+import org.apache.calcite.sql.SqlIdentifier;
 import org.apache.calcite.sql.SqlKind;
 import org.apache.calcite.sql.SqlOperator;
 import org.apache.calcite.sql.SqlOperatorTable;
@@ -87,15 +88,18 @@ abstract class AggVisitor extends SqlBasicVisitor<Void> {
       final SqlFunction sqlFunction = (SqlFunction) operator;
       if (sqlFunction.getFunctionType().isUserDefinedNotSpecificFunction()) {
         final List<SqlOperator> list = new ArrayList<>();
-        opTab.lookupOperatorOverloads(sqlFunction.getSqlIdentifier(),
-            sqlFunction.getFunctionType(), SqlSyntax.FUNCTION, list,
-            nameMatcher);
-        for (SqlOperator operator2 : list) {
-          if (operator2.isAggregator() && !operator2.requiresOver()) {
-            // If nested aggregates disallowed or found aggregate at invalid
-            // level
-            if (aggregate) {
-              found(call);
+        final SqlIdentifier identifier = sqlFunction.getSqlIdentifier();
+        if (identifier != null) {
+          opTab.lookupOperatorOverloads(identifier,
+              sqlFunction.getFunctionType(), SqlSyntax.FUNCTION, list,
+              nameMatcher);
+          for (SqlOperator operator2 : list) {
+            if (operator2.isAggregator() && !operator2.requiresOver()) {
+              // If nested aggregates disallowed or found aggregate at invalid
+              // level
+              if (aggregate) {
+                found(call);
+              }
             }
           }
         }
diff --git a/core/src/main/java/org/apache/calcite/sql/validate/ProcedureNamespace.java b/core/src/main/java/org/apache/calcite/sql/validate/ProcedureNamespace.java
index 46f4534..c4ffb6d 100644
--- a/core/src/main/java/org/apache/calcite/sql/validate/ProcedureNamespace.java
+++ b/core/src/main/java/org/apache/calcite/sql/validate/ProcedureNamespace.java
@@ -21,6 +21,8 @@ import org.apache.calcite.sql.SqlCall;
 import org.apache.calcite.sql.SqlCallBinding;
 import org.apache.calcite.sql.SqlNode;
 import org.apache.calcite.sql.SqlOperator;
+import org.apache.calcite.sql.SqlTableFunction;
+import org.apache.calcite.sql.type.SqlReturnTypeInference;
 import org.apache.calcite.sql.type.SqlTypeName;
 
 /**
@@ -54,21 +56,18 @@ public class ProcedureNamespace extends AbstractNamespace {
     final SqlOperator operator = call.getOperator();
     final SqlCallBinding callBinding =
         new SqlCallBinding(validator, scope, call);
-    if (operator instanceof SqlUserDefinedTableFunction) {
-      assert type.getSqlTypeName() == SqlTypeName.CURSOR
-          : "User-defined table function should have CURSOR type, not " + type;
-      final SqlUserDefinedTableFunction udf =
-          (SqlUserDefinedTableFunction) operator;
-      return udf.getRowType(validator.typeFactory, callBinding.operands());
-    } else if (operator instanceof SqlUserDefinedTableMacro) {
-      assert type.getSqlTypeName() == SqlTypeName.CURSOR
-          : "User-defined table macro should have CURSOR type, not " + type;
-      final SqlUserDefinedTableMacro udf =
-          (SqlUserDefinedTableMacro) operator;
-      return udf.getTable(validator.typeFactory, callBinding.operands())
-          .getRowType(validator.typeFactory);
+    if (!(operator instanceof SqlTableFunction)) {
+      throw new IllegalArgumentException("Argument must be a table function: "
+          + operator.getNameAsId());
     }
-    return type;
+    final SqlTableFunction tableFunction = (SqlTableFunction) operator;
+    if (type.getSqlTypeName() != SqlTypeName.CURSOR) {
+      throw new IllegalArgumentException("Table function should have CURSOR "
+          + "type, not " + type);
+    }
+    final SqlReturnTypeInference rowTypeInference =
+        tableFunction.getRowTypeInference();
+    return rowTypeInference.inferReturnType(callBinding);
   }
 
   public SqlNode getNode() {
diff --git a/core/src/main/java/org/apache/calcite/sql/validate/SqlUserDefinedTableFunction.java b/core/src/main/java/org/apache/calcite/sql/validate/SqlUserDefinedTableFunction.java
index 622b20b..735d299 100644
--- a/core/src/main/java/org/apache/calcite/sql/validate/SqlUserDefinedTableFunction.java
+++ b/core/src/main/java/org/apache/calcite/sql/validate/SqlUserDefinedTableFunction.java
@@ -17,11 +17,11 @@
 package org.apache.calcite.sql.validate;
 
 import org.apache.calcite.rel.type.RelDataType;
-import org.apache.calcite.rel.type.RelDataTypeFactory;
 import org.apache.calcite.schema.TableFunction;
 import org.apache.calcite.sql.SqlFunctionCategory;
 import org.apache.calcite.sql.SqlIdentifier;
-import org.apache.calcite.sql.SqlNode;
+import org.apache.calcite.sql.SqlOperatorBinding;
+import org.apache.calcite.sql.SqlTableFunction;
 import org.apache.calcite.sql.type.SqlOperandTypeChecker;
 import org.apache.calcite.sql.type.SqlOperandTypeInference;
 import org.apache.calcite.sql.type.SqlReturnTypeInference;
@@ -35,7 +35,8 @@ import java.util.List;
  * <p>Created by the validator, after resolving a function call to a function
  * defined in a Calcite schema.
 */
-public class SqlUserDefinedTableFunction extends SqlUserDefinedFunction {
+public class SqlUserDefinedTableFunction extends SqlUserDefinedFunction
+    implements SqlTableFunction {
   public SqlUserDefinedTableFunction(SqlIdentifier opName,
       SqlReturnTypeInference returnTypeInference,
       SqlOperandTypeInference operandTypeInference,
@@ -54,22 +55,15 @@ public class SqlUserDefinedTableFunction extends SqlUserDefinedFunction {
     return (TableFunction) super.getFunction();
   }
 
-  /**
-   * Returns the record type of the table yielded by this function when
-   * applied to given arguments. Only literal arguments are passed,
-   * non-literal are replaced with default values (null, 0, false, etc).
-   *
-   * @param typeFactory Type factory
-   * @param operandList arguments of a function call (only literal arguments
-   *                    are passed, nulls for non-literal ones)
-   * @return row type of the table
-   */
-  public RelDataType getRowType(RelDataTypeFactory typeFactory,
-      List<SqlNode> operandList) {
+  @Override public SqlReturnTypeInference getRowTypeInference() {
+    return this::inferRowType;
+  }
+
+  private RelDataType inferRowType(SqlOperatorBinding callBinding) {
     List<Object> arguments =
-        SqlUserDefinedTableMacro.convertArguments(typeFactory, operandList,
-          function, getNameAsId(), false);
-    return getFunction().getRowType(typeFactory, arguments);
+        SqlUserDefinedTableMacro.convertArguments(callBinding, function,
+            getNameAsId(), false);
+    return getFunction().getRowType(callBinding.getTypeFactory(), arguments);
   }
 
   /**
@@ -77,15 +71,13 @@ public class SqlUserDefinedTableFunction extends SqlUserDefinedFunction {
    * applied to given arguments. Only literal arguments are passed,
    * non-literal are replaced with default values (null, 0, false, etc).
    *
-   * @param operandList arguments of a function call (only literal arguments
-   *                    are passed, nulls for non-literal ones)
+   * @param callBinding Operand bound to arguments
    * @return element type of the table (e.g. {@code Object[].class})
    */
-  public Type getElementType(RelDataTypeFactory typeFactory,
-      List<SqlNode> operandList) {
+  public Type getElementType(SqlOperatorBinding callBinding) {
     List<Object> arguments =
-        SqlUserDefinedTableMacro.convertArguments(typeFactory, operandList,
-            function, getNameAsId(), false);
+        SqlUserDefinedTableMacro.convertArguments(callBinding, function,
+            getNameAsId(), false);
     return getFunction().getElementType(arguments);
   }
 }
diff --git a/core/src/main/java/org/apache/calcite/sql/validate/SqlUserDefinedTableMacro.java b/core/src/main/java/org/apache/calcite/sql/validate/SqlUserDefinedTableMacro.java
index 3d4dbd7..b9066dd 100644
--- a/core/src/main/java/org/apache/calcite/sql/validate/SqlUserDefinedTableMacro.java
+++ b/core/src/main/java/org/apache/calcite/sql/validate/SqlUserDefinedTableMacro.java
@@ -16,39 +16,27 @@
  */
 package org.apache.calcite.sql.validate;
 
-import org.apache.calcite.adapter.enumerable.EnumUtils;
-import org.apache.calcite.linq4j.tree.BlockBuilder;
-import org.apache.calcite.linq4j.tree.Expression;
-import org.apache.calcite.linq4j.tree.Expressions;
-import org.apache.calcite.linq4j.tree.FunctionExpression;
+import org.apache.calcite.linq4j.Ord;
 import org.apache.calcite.rel.type.RelDataType;
 import org.apache.calcite.rel.type.RelDataTypeFactory;
-import org.apache.calcite.rel.type.RelDataTypeFactoryImpl;
 import org.apache.calcite.schema.Function;
 import org.apache.calcite.schema.FunctionParameter;
 import org.apache.calcite.schema.TableMacro;
 import org.apache.calcite.schema.TranslatableTable;
-import org.apache.calcite.sql.SqlCall;
 import org.apache.calcite.sql.SqlFunction;
 import org.apache.calcite.sql.SqlFunctionCategory;
 import org.apache.calcite.sql.SqlIdentifier;
 import org.apache.calcite.sql.SqlKind;
-import org.apache.calcite.sql.SqlLiteral;
-import org.apache.calcite.sql.SqlNode;
-import org.apache.calcite.sql.SqlUtil;
+import org.apache.calcite.sql.SqlOperatorBinding;
+import org.apache.calcite.sql.SqlTableFunction;
 import org.apache.calcite.sql.type.SqlOperandTypeChecker;
 import org.apache.calcite.sql.type.SqlOperandTypeInference;
 import org.apache.calcite.sql.type.SqlReturnTypeInference;
-import org.apache.calcite.util.ImmutableNullableList;
-import org.apache.calcite.util.NlsString;
-import org.apache.calcite.util.Pair;
 import org.apache.calcite.util.Util;
 
-import com.google.common.collect.ImmutableMap;
 import com.google.common.collect.Lists;
 
 import java.util.ArrayList;
-import java.util.Collections;
 import java.util.List;
 import java.util.Objects;
 
@@ -58,7 +46,8 @@ import java.util.Objects;
  * <p>Created by the validator, after resolving a function call to a function
  * defined in a Calcite schema.
 */
-public class SqlUserDefinedTableMacro extends SqlFunction {
+public class SqlUserDefinedTableMacro extends SqlFunction
+    implements SqlTableFunction {
   private final TableMacro tableMacro;
 
   public SqlUserDefinedTableMacro(SqlIdentifier opName,
@@ -79,10 +68,9 @@ public class SqlUserDefinedTableMacro extends SqlFunction {
   }
 
   /** Returns the table in this UDF, or null if there is no table. */
-  public TranslatableTable getTable(RelDataTypeFactory typeFactory,
-      List<SqlNode> operandList) {
-    List<Object> arguments = convertArguments(typeFactory, operandList,
-        tableMacro, getNameAsId(), true);
+  public TranslatableTable getTable(SqlOperatorBinding callBinding) {
+    List<Object> arguments =
+        convertArguments(callBinding, tableMacro, getNameAsId(), true);
     return tableMacro.apply(arguments);
   }
 
@@ -90,111 +78,46 @@ public class SqlUserDefinedTableMacro extends SqlFunction {
    * Converts arguments from {@link org.apache.calcite.sql.SqlNode} to
    * java object format.
    *
-   * @param typeFactory type factory used to convert the arguments
-   * @param operandList input arguments
+   * @param callBinding Operator bound to arguments
    * @param function target function to get parameter types from
    * @param opName name of the operator to use in error message
    * @param failOnNonLiteral true when conversion should fail on non-literal
    * @return converted list of arguments
    */
-  public static List<Object> convertArguments(RelDataTypeFactory typeFactory,
-      List<SqlNode> operandList, Function function,
-      SqlIdentifier opName,
-      boolean failOnNonLiteral) {
-    List<Object> arguments = new ArrayList<>(operandList.size());
-    // Construct a list of arguments, if they are all constants.
-    for (Pair<FunctionParameter, SqlNode> pair
-        : Pair.zip(function.getParameters(), operandList)) {
-      try {
-        final Object o = getValue(pair.right);
-        final Object o2 = coerce(o, pair.left.getType(typeFactory));
-        arguments.add(o2);
-      } catch (NonLiteralException e) {
+  static List<Object> convertArguments(SqlOperatorBinding callBinding,
+      Function function, SqlIdentifier opName, boolean failOnNonLiteral) {
+    RelDataTypeFactory typeFactory = callBinding.getTypeFactory();
+    List<Object> arguments = new ArrayList<>(callBinding.getOperandCount());
+    Ord.forEach(function.getParameters(), (parameter, i) -> {
+      final RelDataType type = parameter.getType(typeFactory);
+      final Object value;
+      if (callBinding.isOperandLiteral(i, true)) {
+        value = callBinding.getOperandLiteralValue(i, type);
+      } else {
         if (failOnNonLiteral) {
           throw new IllegalArgumentException("All arguments of call to macro "
               + opName + " should be literal. Actual argument #"
-              + pair.left.getOrdinal() + " (" + pair.left.getName()
-              + ") is not literal: " + pair.right);
+              + parameter.getOrdinal() + " (" + parameter.getName()
+              + ") is not literal");
         }
-        final RelDataType type = pair.left.getType(typeFactory);
-        final Object value;
         if (type.isNullable()) {
           value = null;
         } else {
           value = 0L;
         }
-        arguments.add(value);
       }
-    }
+      arguments.add(value);
+    });
     return arguments;
   }
 
-  private static Object getValue(SqlNode right) throws NonLiteralException {
-    switch (right.getKind()) {
-    case ARRAY_VALUE_CONSTRUCTOR:
-      final List<Object> list = new ArrayList<>();
-      for (SqlNode o : ((SqlCall) right).getOperandList()) {
-        list.add(getValue(o));
-      }
-      return ImmutableNullableList.copyOf(list);
-    case MAP_VALUE_CONSTRUCTOR:
-      final ImmutableMap.Builder<Object, Object> builder2 =
-          ImmutableMap.builder();
-      final List<SqlNode> operands = ((SqlCall) right).getOperandList();
-      for (int i = 0; i < operands.size(); i += 2) {
-        final SqlNode key = operands.get(i);
-        final SqlNode value = operands.get(i + 1);
-        builder2.put(getValue(key), getValue(value));
-      }
-      return builder2.build();
-    case CAST:
-      return getValue(((SqlCall) right).operand(0));
-    default:
-      if (SqlUtil.isNullLiteral(right, true)) {
-        return null;
-      }
-      if (SqlUtil.isLiteral(right)) {
-        return ((SqlLiteral) right).getValue();
-      }
-      if (right.getKind() == SqlKind.DEFAULT) {
-        return null; // currently NULL is the only default value
-      }
-      throw new NonLiteralException();
-    }
-  }
-
-  private static Object coerce(Object o, RelDataType type) {
-    if (o == null) {
-      return null;
-    }
-    if (!(type instanceof RelDataTypeFactoryImpl.JavaType)) {
-      return null;
-    }
-    final RelDataTypeFactoryImpl.JavaType javaType =
-        (RelDataTypeFactoryImpl.JavaType) type;
-    final Class clazz = javaType.getJavaClass();
-    //noinspection unchecked
-    if (clazz.isAssignableFrom(o.getClass())) {
-      return o;
-    }
-    if (o instanceof NlsString) {
-      return coerce(((NlsString) o).getValue(), type);
-    }
-    // We need optimization here for constant folding.
-    // Not all the expressions can be interpreted (e.g. ternary), so
-    // we rely on optimization capabilities to fold non-interpretable
-    // expressions.
-    BlockBuilder bb = new BlockBuilder();
-    final Expression expr =
-        EnumUtils.convert(Expressions.constant(o), clazz);
-    bb.add(Expressions.return_(null, expr));
-    final FunctionExpression convert =
-        Expressions.lambda(bb.toBlock(), Collections.emptyList());
-    return convert.compile().dynamicInvoke();
+  @Override public SqlReturnTypeInference getRowTypeInference() {
+    return this::inferRowType;
   }
 
-  /** Thrown when a non-literal occurs in an argument to a user-defined
-   * table macro. */
-  private static class NonLiteralException extends Exception {
+  private RelDataType inferRowType(SqlOperatorBinding callBinding) {
+    final RelDataTypeFactory typeFactory = callBinding.getTypeFactory();
+    final TranslatableTable table = getTable(callBinding);
+    return table.getRowType(typeFactory);
   }
 }
diff --git a/core/src/main/java/org/apache/calcite/sql/validate/SqlValidatorImpl.java b/core/src/main/java/org/apache/calcite/sql/validate/SqlValidatorImpl.java
index a81f359..0919ee2 100644
--- a/core/src/main/java/org/apache/calcite/sql/validate/SqlValidatorImpl.java
+++ b/core/src/main/java/org/apache/calcite/sql/validate/SqlValidatorImpl.java
@@ -71,6 +71,7 @@ import org.apache.calcite.sql.SqlSelect;
 import org.apache.calcite.sql.SqlSelectKeyword;
 import org.apache.calcite.sql.SqlSnapshot;
 import org.apache.calcite.sql.SqlSyntax;
+import org.apache.calcite.sql.SqlTableFunction;
 import org.apache.calcite.sql.SqlUnresolvedFunction;
 import org.apache.calcite.sql.SqlUpdate;
 import org.apache.calcite.sql.SqlUtil;
@@ -3958,9 +3959,26 @@ public class SqlValidatorImpl implements SqlValidatorWithHints {
    */
   private void validateGroupByItem(SqlSelect select, SqlNode groupByItem) {
     final SqlValidatorScope groupByScope = getGroupScope(select);
+    validateGroupByExpr(groupByItem, groupByScope);
     groupByScope.validateExpr(groupByItem);
   }
 
+  private void validateGroupByExpr(SqlNode groupByItem,
+      SqlValidatorScope groupByScope) {
+    switch (groupByItem.getKind()) {
+    case GROUPING_SETS:
+    case ROLLUP:
+    case CUBE:
+      final SqlCall call = (SqlCall) groupByItem;
+      for (SqlNode operand : call.getOperandList()) {
+        validateExpr(operand, groupByScope);
+      }
+      break;
+    default:
+      validateExpr(groupByItem, groupByScope);
+    }
+  }
+
   /**
    * Validates an item in the ORDER BY clause of a SELECT statement.
    *
@@ -4218,6 +4236,9 @@ public class SqlValidatorImpl implements SqlValidatorWithHints {
         throw newValidationError(expr,
             RESOURCE.absentOverClause());
       }
+      if (op instanceof SqlTableFunction) {
+        throw RESOURCE.cannotCallTableFunctionHere(op.getName()).ex();
+      }
     }
 
     // Call on the expression to validate itself.
diff --git a/core/src/main/java/org/apache/calcite/sql2rel/SqlToRelConverter.java b/core/src/main/java/org/apache/calcite/sql2rel/SqlToRelConverter.java
index 8b696b5..6483786 100644
--- a/core/src/main/java/org/apache/calcite/sql2rel/SqlToRelConverter.java
+++ b/core/src/main/java/org/apache/calcite/sql2rel/SqlToRelConverter.java
@@ -2480,8 +2480,7 @@ public class SqlToRelConverter {
     if (operator instanceof SqlUserDefinedTableMacro) {
       final SqlUserDefinedTableMacro udf =
           (SqlUserDefinedTableMacro) operator;
-      final TranslatableTable table =
-          udf.getTable(typeFactory, callBinding.operands());
+      final TranslatableTable table = udf.getTable(callBinding);
       final RelDataType rowType = table.getRowType(typeFactory);
       RelOptTable relOptTable = RelOptTableImpl.create(null, rowType, table,
           udf.getNameAsId().names);
@@ -2493,7 +2492,7 @@ public class SqlToRelConverter {
     Type elementType;
     if (operator instanceof SqlUserDefinedTableFunction) {
       SqlUserDefinedTableFunction udtf = (SqlUserDefinedTableFunction) operator;
-      elementType = udtf.getElementType(typeFactory, callBinding.operands());
+      elementType = udtf.getElementType(callBinding);
     } else {
       elementType = null;
     }
diff --git a/core/src/main/resources/org/apache/calcite/runtime/CalciteResource.properties b/core/src/main/resources/org/apache/calcite/runtime/CalciteResource.properties
index 9f4b453..3a42da3 100644
--- a/core/src/main/resources/org/apache/calcite/runtime/CalciteResource.properties
+++ b/core/src/main/resources/org/apache/calcite/runtime/CalciteResource.properties
@@ -185,6 +185,7 @@ DynamicParamIllegal=Illegal use of dynamic parameter
 InvalidBoolean=''{0}'' is not a valid boolean value
 ArgumentMustBeValidPrecision=Argument to function ''{0}'' must be a valid precision between ''{1,number,#}'' and ''{2,number,#}''
 IllegalArgumentForTableFunctionCall=Wrong arguments for table function ''{0}'' call. Expected ''{1}'', actual ''{2}''
+CannotCallTableFunctionHere=Cannot call table function here: ''{0}''
 InvalidDatetimeFormat=''{0}'' is not a valid datetime format
 InsertIntoAlwaysGenerated=Cannot INSERT into generated column ''{0}''
 ArgumentMustHaveScaleZero=Argument to function ''{0}'' must have a scale of 0
diff --git a/core/src/test/java/org/apache/calcite/test/MockSqlOperatorTable.java b/core/src/test/java/org/apache/calcite/test/MockSqlOperatorTable.java
index 0da69b8..3cced47 100644
--- a/core/src/test/java/org/apache/calcite/test/MockSqlOperatorTable.java
+++ b/core/src/test/java/org/apache/calcite/test/MockSqlOperatorTable.java
@@ -18,9 +18,6 @@ package org.apache.calcite.test;
 
 import org.apache.calcite.rel.type.RelDataType;
 import org.apache.calcite.rel.type.RelDataTypeFactory;
-import org.apache.calcite.rel.type.RelDataTypeFieldImpl;
-import org.apache.calcite.rel.type.RelRecordType;
-import org.apache.calcite.rel.type.StructKind;
 import org.apache.calcite.sql.SqlAggFunction;
 import org.apache.calcite.sql.SqlFunction;
 import org.apache.calcite.sql.SqlFunctionCategory;
@@ -29,9 +26,11 @@ import org.apache.calcite.sql.SqlKind;
 import org.apache.calcite.sql.SqlOperator;
 import org.apache.calcite.sql.SqlOperatorBinding;
 import org.apache.calcite.sql.SqlOperatorTable;
+import org.apache.calcite.sql.SqlTableFunction;
 import org.apache.calcite.sql.parser.SqlParserPos;
 import org.apache.calcite.sql.type.OperandTypes;
 import org.apache.calcite.sql.type.ReturnTypes;
+import org.apache.calcite.sql.type.SqlReturnTypeInference;
 import org.apache.calcite.sql.type.SqlTypeFamily;
 import org.apache.calcite.sql.type.SqlTypeName;
 import org.apache.calcite.sql.util.ChainedSqlOperatorTable;
@@ -40,7 +39,6 @@ import org.apache.calcite.util.Optionality;
 
 import com.google.common.collect.ImmutableList;
 
-import java.util.Arrays;
 
 /**
  * Mock operator table for testing purposes. Contains the standard SQL operator
@@ -69,43 +67,81 @@ public class MockSqlOperatorTable extends ChainedSqlOperatorTable {
     opTab.addOperator(new MyFunction());
     opTab.addOperator(new MyAvgAggFunction());
     opTab.addOperator(new RowFunction());
+    opTab.addOperator(new NotATableFunction());
+    opTab.addOperator(new BadTableFunction());
   }
 
   /** "RAMP" user-defined function. */
-  public static class RampFunction extends SqlFunction {
+  public static class RampFunction extends SqlFunction
+      implements SqlTableFunction {
     public RampFunction() {
       super("RAMP",
           SqlKind.OTHER_FUNCTION,
+          ReturnTypes.CURSOR,
           null,
+          OperandTypes.NUMERIC,
+          SqlFunctionCategory.USER_DEFINED_TABLE_FUNCTION);
+    }
+
+    @Override public SqlReturnTypeInference getRowTypeInference() {
+      return opBinding -> opBinding.getTypeFactory().builder()
+          .add("I", SqlTypeName.INTEGER)
+          .build();
+    }
+  }
+
+  /** Not valid as a table function, even though it returns CURSOR, because
+   * it does not implement {@link SqlTableFunction}. */
+  public static class NotATableFunction extends SqlFunction {
+    public NotATableFunction() {
+      super("BAD_RAMP",
+          SqlKind.OTHER_FUNCTION,
+          ReturnTypes.CURSOR,
           null,
           OperandTypes.NUMERIC,
           SqlFunctionCategory.USER_DEFINED_FUNCTION);
     }
+  }
+
+  /** Another bad table function: declares itself as a table function but does
+   * not return CURSOR. */
+  public static class BadTableFunction extends SqlFunction
+      implements SqlTableFunction {
+    public BadTableFunction() {
+      super("BAD_TABLE_FUNCTION",
+          SqlKind.OTHER_FUNCTION,
+          null,
+          null,
+          OperandTypes.NUMERIC,
+          SqlFunctionCategory.USER_DEFINED_TABLE_FUNCTION);
+    }
 
     public RelDataType inferReturnType(SqlOperatorBinding opBinding) {
-      final RelDataTypeFactory typeFactory =
-          opBinding.getTypeFactory();
-      return typeFactory.builder()
+      // This is wrong. A table function should return CURSOR.
+      return opBinding.getTypeFactory().builder()
           .add("I", SqlTypeName.INTEGER)
           .build();
     }
+
+    @Override public SqlReturnTypeInference getRowTypeInference() {
+      return this::inferReturnType;
+    }
   }
 
   /** "DEDUP" user-defined function. */
-  public static class DedupFunction extends SqlFunction {
+  public static class DedupFunction extends SqlFunction
+      implements SqlTableFunction {
     public DedupFunction() {
       super("DEDUP",
           SqlKind.OTHER_FUNCTION,
-          null,
+          ReturnTypes.CURSOR,
           null,
           OperandTypes.VARIADIC,
-          SqlFunctionCategory.USER_DEFINED_FUNCTION);
+          SqlFunctionCategory.USER_DEFINED_TABLE_FUNCTION);
     }
 
-    public RelDataType inferReturnType(SqlOperatorBinding opBinding) {
-      final RelDataTypeFactory typeFactory =
-          opBinding.getTypeFactory();
-      return typeFactory.builder()
+    @Override public SqlReturnTypeInference getRowTypeInference() {
+      return opBinding -> opBinding.getTypeFactory().builder()
           .add("NAME", SqlTypeName.VARCHAR, 1024)
           .build();
     }
@@ -177,36 +213,25 @@ public class MockSqlOperatorTable extends ChainedSqlOperatorTable {
 
   /** "ROW_FUNC" user-defined function whose return type is
    * nullable row type with non-nullable fields. */
-  public static class RowFunction extends SqlFunction {
-    public RowFunction() {
-      super("ROW_FUNC",
-          SqlKind.OTHER_FUNCTION,
-          null,
-          null,
-          OperandTypes.NILADIC,
-          SqlFunctionCategory.USER_DEFINED_FUNCTION);
+  public static class RowFunction extends SqlFunction
+      implements SqlTableFunction {
+    RowFunction() {
+      super("ROW_FUNC", SqlKind.OTHER_FUNCTION, ReturnTypes.CURSOR, null,
+          OperandTypes.NILADIC, SqlFunctionCategory.USER_DEFINED_FUNCTION);
     }
 
-    public RelDataType inferReturnType(SqlOperatorBinding opBinding) {
-      final RelDataTypeFactory typeFactory =
-          opBinding.getTypeFactory();
-      final RelDataType bigIntNotNull = typeFactory.createSqlType(SqlTypeName.BIGINT);
-      final RelDataType bigIntNullable =
-          typeFactory.createTypeWithNullability(bigIntNotNull, true);
-      return new RelRecordType(
-          StructKind.FULLY_QUALIFIED,
-          Arrays.asList(
-              new RelDataTypeFieldImpl(
-                  "NOT_NULL_FIELD",
-                  0,
-                  bigIntNotNull),
-              new RelDataTypeFieldImpl(
-                  "NULLABLE_FIELD",
-                  0,
-                  bigIntNullable)
-          ),
-          true
-      );
+    private static RelDataType inferRowType(SqlOperatorBinding opBinding) {
+      final RelDataTypeFactory typeFactory = opBinding.getTypeFactory();
+      final RelDataType bigintType =
+          typeFactory.createSqlType(SqlTypeName.BIGINT);
+      return typeFactory.builder()
+          .add("NOT_NULL_FIELD", bigintType)
+          .add("NULLABLE_FIELD", bigintType).nullable(true)
+          .build();
+    }
+
+    @Override public SqlReturnTypeInference getRowTypeInference() {
+      return RowFunction::inferRowType;
     }
   }
 }
diff --git a/core/src/test/java/org/apache/calcite/test/SqlOperatorBindingTest.java b/core/src/test/java/org/apache/calcite/test/SqlOperatorBindingTest.java
index cac4e3b..4f8ea4e 100644
--- a/core/src/test/java/org/apache/calcite/test/SqlOperatorBindingTest.java
+++ b/core/src/test/java/org/apache/calcite/test/SqlOperatorBindingTest.java
@@ -21,13 +21,15 @@ import org.apache.calcite.jdbc.JavaTypeFactoryImpl;
 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.RexCallBinding;
 import org.apache.calcite.rex.RexNode;
-import org.apache.calcite.rex.RexProgram;
-import org.apache.calcite.rex.RexProgramBuilder;
 import org.apache.calcite.rex.RexUtil;
+import org.apache.calcite.sql.SqlCallBinding;
+import org.apache.calcite.sql.SqlCharStringLiteral;
 import org.apache.calcite.sql.SqlDataTypeSpec;
 import org.apache.calcite.sql.SqlLiteral;
 import org.apache.calcite.sql.SqlNode;
+import org.apache.calcite.sql.SqlOperatorBinding;
 import org.apache.calcite.sql.SqlUtil;
 import org.apache.calcite.sql.fun.SqlStdOperatorTable;
 import org.apache.calcite.sql.parser.SqlParserPos;
@@ -39,11 +41,12 @@ import com.google.common.collect.Lists;
 import org.junit.jupiter.api.BeforeEach;
 import org.junit.jupiter.api.Test;
 
-import static org.junit.jupiter.api.Assertions.assertSame;
+import static org.hamcrest.MatcherAssert.assertThat;
+import static org.hamcrest.core.Is.is;
 
 /**
- * Unit tests for {@link RexProgram} and
- * {@link RexProgramBuilder}.
+ * Unit tests for {@link SqlOperatorBinding} and its sub-classes
+ * {@link SqlCallBinding} and {@link RexCallBinding}.
  */
 class SqlOperatorBindingTest {
   private RexBuilder rexBuilder;
@@ -51,7 +54,7 @@ class SqlOperatorBindingTest {
   private SqlDataTypeSpec integerType;
 
   @BeforeEach
-  public void setUp() {
+  void setUp() {
     JavaTypeFactory typeFactory = new JavaTypeFactoryImpl(RelDataTypeSystem.DEFAULT);
     integerDataType = typeFactory.createSqlType(SqlTypeName.INTEGER);
     integerType = SqlTypeUtil.convertTypeToSpec(integerDataType);
@@ -65,24 +68,52 @@ class SqlOperatorBindingTest {
    * literal</a>.
    */
   @Test void testSqlNodeLiteral() {
-    final SqlNode literal = SqlLiteral.createExactNumeric(
-        "0",
-        SqlParserPos.ZERO);
-    final SqlNode castLiteral = SqlStdOperatorTable.CAST.createCall(
-        SqlParserPos.ZERO,
-        literal,
-        integerType);
-    final SqlNode castCastLiteral = SqlStdOperatorTable.CAST.createCall(
-        SqlParserPos.ZERO,
-        castLiteral,
-        integerType);
+    final SqlParserPos pos = SqlParserPos.ZERO;
+    final SqlNode zeroLiteral = SqlLiteral.createExactNumeric("0", pos);
+    final SqlNode oneLiteral = SqlLiteral.createExactNumeric("1", pos);
+    final SqlNode nullLiteral = SqlLiteral.createNull(pos);
+    final SqlCharStringLiteral aLiteral = SqlLiteral.createCharString("a", pos);
 
-    // SqlLiteral is considered as a Literal
-    assertSame(true, SqlUtil.isLiteral(literal, true));
-    // CAST(SqlLiteral as type) is considered as a Literal
-    assertSame(true, SqlUtil.isLiteral(castLiteral, true));
-    // CAST(CAST(SqlLiteral as type) as type) is NOT considered as a Literal
-    assertSame(false, SqlUtil.isLiteral(castCastLiteral, true));
+    final SqlNode castLiteral =
+        SqlStdOperatorTable.CAST.createCall(pos, zeroLiteral, integerType);
+    final SqlNode castCastLiteral =
+        SqlStdOperatorTable.CAST.createCall(pos, castLiteral, integerType);
+    final SqlNode mapLiteral =
+        SqlStdOperatorTable.MAP_VALUE_CONSTRUCTOR.createCall(pos,
+            aLiteral, oneLiteral);
+    final SqlNode map2Literal =
+        SqlStdOperatorTable.MAP_VALUE_CONSTRUCTOR.createCall(pos,
+            aLiteral, castLiteral);
+    final SqlNode arrayLiteral =
+        SqlStdOperatorTable.ARRAY_VALUE_CONSTRUCTOR.createCall(pos,
+            zeroLiteral, oneLiteral);
+    final SqlNode defaultCall = SqlStdOperatorTable.DEFAULT.createCall(pos);
+
+    // SqlLiteral is considered a literal
+    assertThat(SqlUtil.isLiteral(zeroLiteral, false), is(true));
+    assertThat(SqlUtil.isLiteral(zeroLiteral, true), is(true));
+    // NULL literal is considered a literal
+    assertThat(SqlUtil.isLiteral(nullLiteral, false), is(true));
+    assertThat(SqlUtil.isLiteral(nullLiteral, true), is(true));
+    // CAST(SqlLiteral as type) is considered a literal, iff allowCast
+    assertThat(SqlUtil.isLiteral(castLiteral, false), is(false));
+    assertThat(SqlUtil.isLiteral(castLiteral, true), is(true));
+    // CAST(CAST(SqlLiteral as type) as type) is considered a literal,
+    // iff allowCast
+    assertThat(SqlUtil.isLiteral(castCastLiteral, false), is(false));
+    assertThat(SqlUtil.isLiteral(castCastLiteral, true), is(true));
+    // MAP['a', 1] and MAP['a', CAST(0 AS INTEGER)] are considered literals,
+    // iff allowCast
+    assertThat(SqlUtil.isLiteral(mapLiteral, false), is(false));
+    assertThat(SqlUtil.isLiteral(mapLiteral, true), is(true));
+    assertThat(SqlUtil.isLiteral(map2Literal, false), is(false));
+    assertThat(SqlUtil.isLiteral(map2Literal, true), is(true));
+    // ARRAY[0, 1] is considered a literal, iff allowCast
+    assertThat(SqlUtil.isLiteral(arrayLiteral, false), is(false));
+    assertThat(SqlUtil.isLiteral(arrayLiteral, true), is(true));
+    // DEFAULT is considered a literal, iff allowCast
+    assertThat(SqlUtil.isLiteral(defaultCall, false), is(false));
+    assertThat(SqlUtil.isLiteral(defaultCall, true), is(true));
   }
 
   /** Tests {@link org.apache.calcite.rex.RexUtil#isLiteral(RexNode, boolean)},
@@ -105,11 +136,11 @@ class SqlOperatorBindingTest {
         SqlStdOperatorTable.CAST,
         Lists.newArrayList(castLiteral));
 
-    // RexLiteral is considered as a Literal
-    assertSame(true, RexUtil.isLiteral(literal, true));
-    // CAST(RexLiteral as type) is considered as a Literal
-    assertSame(true, RexUtil.isLiteral(castLiteral, true));
-    // CAST(CAST(RexLiteral as type) as type) is NOT considered as a Literal
-    assertSame(false, RexUtil.isLiteral(castCastLiteral, true));
+    // RexLiteral is considered a literal
+    assertThat(RexUtil.isLiteral(literal, true), is(true));
+    // CAST(RexLiteral as type) is considered a literal
+    assertThat(RexUtil.isLiteral(castLiteral, true), is(true));
+    // CAST(CAST(RexLiteral as type) as type) is NOT considered a literal
+    assertThat(RexUtil.isLiteral(castCastLiteral, true), is(false));
   }
 }
diff --git a/core/src/test/java/org/apache/calcite/test/SqlValidatorTest.java b/core/src/test/java/org/apache/calcite/test/SqlValidatorTest.java
index 12cdca6..32cc678 100644
--- a/core/src/test/java/org/apache/calcite/test/SqlValidatorTest.java
+++ b/core/src/test/java/org/apache/calcite/test/SqlValidatorTest.java
@@ -8078,8 +8078,45 @@ public class SqlValidatorTest extends SqlValidatorTestCase {
     sql("select * from table(ramp('3'))")
         .type("RecordType(INTEGER NOT NULL I) NOT NULL");
 
+    sql("select * from table(abs(-1))")
+        .fails("(?s)Encountered \"abs\" at .*");
+
+    sql("select * from table(1 + 2)")
+        .fails("(?s)Encountered \"1\" at .*");
+
     sql("select * from table(^nonExistentRamp('3')^)")
         .fails("No match found for function signature NONEXISTENTRAMP\\(<CHARACTER>\\)");
+
+    sql("select * from table(^bad_ramp(3)^)")
+        .fails("Argument must be a table function: BAD_RAMP");
+
+    sql("select * from table(^bad_table_function(3)^)")
+        .fails("Table function should have CURSOR type, not"
+            + " RecordType\\(INTEGER I\\)");
+  }
+
+  /** Tests that Calcite gives an error if a table function is used anywhere
+   * that a scalar expression is expected. */
+  @Test void testTableFunctionAsExpression() {
+    sql("select ^ramp(3)^ from (values (1))")
+        .fails("Cannot call table function here: 'RAMP'");
+    sql("select * from (values (1)) where ^ramp(3)^")
+        .fails("WHERE clause must be a condition");
+    sql("select * from (values (1)) where ^ramp(3) and 1 = 1^")
+        .fails("Cannot apply 'AND' to arguments of type '<CURSOR> AND "
+            + "<BOOLEAN>'\\. Supported form\\(s\\): '<BOOLEAN> AND <BOOLEAN>'");
+    sql("select * from (values (1)) where ^ramp(3) is not null^")
+        .fails("Cannot apply 'IS NOT NULL' to arguments of type '<CURSOR> IS"
+            + " NOT NULL'\\. Supported form\\(s\\): '<ANY> IS NOT NULL'");
+    sql("select ^sum(ramp(3))^ from (values (1))")
+        .fails("Cannot apply 'SUM' to arguments of type 'SUM\\(<CURSOR>\\)'\\. "
+            + "Supported form\\(s\\): 'SUM\\(<NUMERIC>\\)'");
+    sql("select * from (values (1)) group by ^ramp(3)^")
+        .fails("Cannot call table function here: 'RAMP'");
+    sql("select count(*) from (values (1)) having ^ramp(3)^")
+        .fails("HAVING clause must be a condition");
+    sql("select * from (values (1)) order by ^ramp(3)^ asc, 1 desc")
+        .fails("Cannot call table function here: 'RAMP'");
   }
 
   /** Test case for
@@ -11550,6 +11587,6 @@ public class SqlValidatorTest extends SqlValidatorTestCase {
     MockSqlOperatorTable.addRamp(operatorTable);
     sql("select * FROM TABLE(ROW_FUNC()) AS T(a, b)")
         .withOperatorTable(operatorTable)
-        .type("RecordType(BIGINT A, BIGINT B) NOT NULL");
+        .type("RecordType(BIGINT NOT NULL A, BIGINT B) NOT NULL");
   }
 }
diff --git a/core/src/test/java/org/apache/calcite/test/TableFunctionTest.java b/core/src/test/java/org/apache/calcite/test/TableFunctionTest.java
index 3508dd6..f0f0148 100644
--- a/core/src/test/java/org/apache/calcite/test/TableFunctionTest.java
+++ b/core/src/test/java/org/apache/calcite/test/TableFunctionTest.java
@@ -145,8 +145,7 @@ class TableFunctionTest {
    * Tests a table function that implements {@link ScannableTable} and returns
    * a single column.
    */
-  @Test void testScannableTableFunction()
-      throws SQLException, ClassNotFoundException {
+  @Test void testScannableTableFunction() throws SQLException {
     Connection connection = DriverManager.getConnection("jdbc:calcite:");
     CalciteConnection calciteConnection =
         connection.unwrap(CalciteConnection.class);
@@ -165,7 +164,7 @@ class TableFunctionTest {
 
   /** As {@link #testScannableTableFunction()} but with named parameters. */
   @Test void testScannableTableFunctionWithNamedParameters()
-      throws SQLException, ClassNotFoundException {
+      throws SQLException {
     Connection connection = DriverManager.getConnection("jdbc:calcite:");
     CalciteConnection calciteConnection =
         connection.unwrap(CalciteConnection.class);
@@ -198,7 +197,7 @@ class TableFunctionTest {
 
   /** As {@link #testScannableTableFunction()} but with named parameters. */
   @Test void testMultipleScannableTableFunctionWithNamedParameters()
-      throws SQLException, ClassNotFoundException {
+      throws SQLException {
     try (Connection connection = DriverManager.getConnection("jdbc:calcite:");
          Statement statement = connection.createStatement()) {
       CalciteConnection calciteConnection =
@@ -243,8 +242,7 @@ class TableFunctionTest {
    * Tests a table function that returns different row type based on
    * actual call arguments.
    */
-  @Test void testTableFunctionDynamicStructure()
-      throws SQLException, ClassNotFoundException {
+  @Test void testTableFunctionDynamicStructure() throws SQLException {
     Connection connection = getConnectionWithMultiplyFunction();
     final PreparedStatement ps = connection.prepareStatement("select *\n"
         + "from table(\"s\".\"multiplication\"(4, 3, ?))\n");
@@ -262,7 +260,7 @@ class TableFunctionTest {
    */
   @Disabled("SQLException does not include message from nested exception")
   @Test void testTableFunctionNonNullableMustBeLiterals()
-      throws SQLException, ClassNotFoundException {
+      throws SQLException {
     Connection connection = getConnectionWithMultiplyFunction();
     try {
       final PreparedStatement ps = connection.prepareStatement("select *\n"
@@ -299,8 +297,7 @@ class TableFunctionTest {
    */
   @Disabled("CannotPlanException: Node [rel#18:Subset#4.ENUMERABLE.[]] "
       + "could not be implemented")
-  @Test void testTableFunctionCursorInputs()
-      throws SQLException, ClassNotFoundException {
+  @Test void testTableFunctionCursorInputs() throws SQLException {
     try (Connection connection =
              DriverManager.getConnection("jdbc:calcite:")) {
       CalciteConnection calciteConnection =
@@ -334,8 +331,7 @@ class TableFunctionTest {
    */
   @Disabled("CannotPlanException: Node [rel#24:Subset#6.ENUMERABLE.[]] "
       + "could not be implemented")
-  @Test void testTableFunctionCursorsInputs()
-      throws SQLException, ClassNotFoundException {
+  @Test void testTableFunctionCursorsInputs() throws SQLException {
     try (Connection connection = getConnectionWithMultiplyFunction()) {
       CalciteConnection calciteConnection =
           connection.unwrap(CalciteConnection.class);
@@ -369,6 +365,9 @@ class TableFunctionTest {
     }
   }
 
+  /** Tests a query with a table function in the FROM clause.
+   *
+   * @see Smalls#multiplicationTable */
   @Test void testUserDefinedTableFunction() {
     final String q = "select *\n"
         + "from table(\"s\".\"multiplication\"(2, 3, 100))\n";
@@ -379,6 +378,11 @@ class TableFunctionTest {
             "row_name=row 2; c1=103; c2=106");
   }
 
+  /** Tests a query with a table function in the FROM clause,
+   * attempting to reference a column from the table function in the WHERE
+   * clause but getting the case wrong.
+   *
+   * @see Smalls#multiplicationTable */
   @Test void testUserDefinedTableFunction2() {
     final String q = "select c1\n"
         + "from table(\"s\".\"multiplication\"(2, 3, 100))\n"
@@ -387,6 +391,10 @@ class TableFunctionTest {
         .throws_("Column 'C1' not found in any table; did you mean 'c1'?");
   }
 
+  /** Tests a query with a table function in the FROM clause,
+   * referencing columns in the WHERE clause.
+   *
+   * @see Smalls#multiplicationTable */
   @Test void testUserDefinedTableFunction3() {
     final String q = "select \"c1\"\n"
         + "from table(\"s\".\"multiplication\"(2, 3, 100))\n"
@@ -394,6 +402,8 @@ class TableFunctionTest {
     with().query(q).returnsUnordered("c1=103");
   }
 
+  /** As {@link #testUserDefinedTableFunction3()}, but provides a character
+   * literal argument for an integer parameter. */
   @Test void testUserDefinedTableFunction4() {
     final String q = "select \"c1\"\n"
         + "from table(\"s\".\"multiplication\"('2', 3, 100))\n"
diff --git a/example/function/src/test/java/org/apache/calcite/test/ExampleFunctionTest.java b/example/function/src/test/java/org/apache/calcite/test/ExampleFunctionTest.java
index ac88123..f4cc0ea 100644
--- a/example/function/src/test/java/org/apache/calcite/test/ExampleFunctionTest.java
+++ b/example/function/src/test/java/org/apache/calcite/test/ExampleFunctionTest.java
@@ -75,7 +75,7 @@ class ExampleFunctionTest {
   }
 
   public void checkMazeTableFunction(Boolean solution, String maze)
-      throws SQLException, ClassNotFoundException {
+      throws SQLException {
     Connection connection = DriverManager.getConnection("jdbc:calcite:");
     CalciteConnection calciteConnection =
         connection.unwrap(CalciteConnection.class);


[calcite] 09/09: [CALCITE-2160] Spatial: Add functions ST_MakeGrid and ST_MakeGridPoints

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

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

commit 885a3da76cfd59171624dc569e83e93a5bdaffe3
Author: Julian Hyde <jh...@apache.org>
AuthorDate: Tue Jul 21 14:18:37 2020 -0700

    [CALCITE-2160] Spatial: Add functions ST_MakeGrid and ST_MakeGridPoints
    
    These will be the foundations of a spatial grid index, to accelerate
    polygon-to-polygon spatial joins.
    
    Add "states" and "parks" data sets.
    
    Fix lateral references to fields.
---
 .../calcite/prepare/CalciteCatalogReader.java      |  11 +-
 .../org/apache/calcite/runtime/GeoFunctions.java   |  96 ++++++++++++
 .../apache/calcite/sql/fun/SqlGeoFunctions.java    | 134 ++++++++++++++++
 .../sql/fun/SqlLibraryOperatorTableFactory.java    |   4 +-
 .../org/apache/calcite/test/CalciteAssert.java     |  35 ++++-
 .../apache/calcite/test/StatesTableFunction.java   | 121 +++++++++++++++
 core/src/test/resources/sql/spatial.iq             |  96 +++++++++++-
 file/src/test/resources/geo/states.json            | 168 +++++++++++++++++++++
 site/_docs/reference.md                            |   8 +-
 9 files changed, 656 insertions(+), 17 deletions(-)

diff --git a/core/src/main/java/org/apache/calcite/prepare/CalciteCatalogReader.java b/core/src/main/java/org/apache/calcite/prepare/CalciteCatalogReader.java
index 74fe767..f77679c 100644
--- a/core/src/main/java/org/apache/calcite/prepare/CalciteCatalogReader.java
+++ b/core/src/main/java/org/apache/calcite/prepare/CalciteCatalogReader.java
@@ -272,15 +272,18 @@ public class CalciteCatalogReader implements Prepare.CatalogReader {
         .forEachOrdered(operatorList::add);
   }
 
-  /** Creates an operator table that contains functions in the given class.
+  /** Creates an operator table that contains functions in the given class
+   * or classes.
    *
    * @see ModelHandler#addFunctions */
-  public static SqlOperatorTable operatorTable(String className) {
+  public static SqlOperatorTable operatorTable(String... classNames) {
     // Dummy schema to collect the functions
     final CalciteSchema schema =
         CalciteSchema.createRootSchema(false, false);
-    ModelHandler.addFunctions(schema.plus(), null, ImmutableList.of(),
-        className, "*", true);
+    for (String className : classNames) {
+      ModelHandler.addFunctions(schema.plus(), null, ImmutableList.of(),
+          className, "*", true);
+    }
 
     // The following is technical debt; see [CALCITE-2082] Remove
     // RelDataTypeFactory argument from SqlUserDefinedAggFunction constructor
diff --git a/core/src/main/java/org/apache/calcite/runtime/GeoFunctions.java b/core/src/main/java/org/apache/calcite/runtime/GeoFunctions.java
index 1918436..9732a0c 100644
--- a/core/src/main/java/org/apache/calcite/runtime/GeoFunctions.java
+++ b/core/src/main/java/org/apache/calcite/runtime/GeoFunctions.java
@@ -16,6 +16,8 @@
  */
 package org.apache.calcite.runtime;
 
+import org.apache.calcite.linq4j.AbstractEnumerable;
+import org.apache.calcite.linq4j.Enumerator;
 import org.apache.calcite.linq4j.function.Deterministic;
 import org.apache.calcite.linq4j.function.Experimental;
 import org.apache.calcite.linq4j.function.SemiStrict;
@@ -179,6 +181,22 @@ public class GeoFunctions {
 
   // Geometry creation functions ==============================================
 
+  /** Calculates a regular grid of polygons based on {@code geom}. */
+  private static void ST_MakeGrid(final Geom geom,
+      final BigDecimal deltaX, final BigDecimal deltaY) {
+    // This is a dummy function. We cannot include table functions in this
+    // package, because they have too many dependencies. See the real definition
+    // in SqlGeoFunctions.
+  }
+
+  /** Calculates a regular grid of points based on {@code geom}. */
+  private static void ST_MakeGridPoints(final Geom geom,
+      final BigDecimal deltaX, final BigDecimal deltaY) {
+    // This is a dummy function. We cannot include table functions in this
+    // package, because they have too many dependencies. See the real definition
+    // in SqlGeoFunctions.
+  }
+
   /**  Creates a line-string from the given POINTs (or MULTIPOINTs). */
   public static Geom ST_MakeLine(Geom geom1, Geom geom2) {
     return makeLine(geom1, geom2);
@@ -655,4 +673,82 @@ public class GeoFunctions {
       this.code = code;
     }
   }
+
+  /** Used at run time by the {@link #ST_MakeGrid} and
+   * {@link #ST_MakeGridPoints} functions. */
+  public static class GridEnumerable extends AbstractEnumerable<Object[]> {
+    private final Envelope envelope;
+    private final boolean point;
+    private final double deltaX;
+    private final double deltaY;
+    private final double minX;
+    private final double minY;
+    private final int baseX;
+    private final int baseY;
+    private final int spanX;
+    private final int spanY;
+    private final int area;
+
+    public GridEnumerable(Envelope envelope, BigDecimal deltaX,
+        BigDecimal deltaY, boolean point) {
+      this.envelope = envelope;
+      this.deltaX = deltaX.doubleValue();
+      this.deltaY = deltaY.doubleValue();
+      this.point = point;
+      this.spanX = (int) Math.floor((envelope.getXMax() - envelope.getXMin())
+          / this.deltaX) + 1;
+      this.baseX = (int) Math.floor(envelope.getXMin() / this.deltaX);
+      this.minX = this.deltaX * baseX;
+      this.spanY = (int) Math.floor((envelope.getYMax() - envelope.getYMin())
+          / this.deltaY) + 1;
+      this.baseY = (int) Math.floor(envelope.getYMin() / this.deltaY);
+      this.minY = this.deltaY * baseY;
+      this.area = this.spanX * this.spanY;
+    }
+
+    @Override public Enumerator<Object[]> enumerator() {
+      return new Enumerator<Object[]>() {
+        int id = -1;
+
+        @Override public Object[] current() {
+          final Geom geom;
+          final int x = id % spanX;
+          final int y = id / spanX;
+          if (point) {
+            final double xCurrent = minX + (x + 0.5D) * deltaX;
+            final double yCurrent = minY + (y + 0.5D) * deltaY;
+            geom = ST_MakePoint(BigDecimal.valueOf(xCurrent),
+                BigDecimal.valueOf(yCurrent));
+          } else {
+            final Polygon polygon = new Polygon();
+            final double left = minX + x * deltaX;
+            final double right = left + deltaX;
+            final double bottom = minY + y * deltaY;
+            final double top = bottom + deltaY;
+
+            final Polyline polyline = new Polyline();
+            polyline.addSegment(new Line(left, bottom, right, bottom), true);
+            polyline.addSegment(new Line(right, bottom, right, top), false);
+            polyline.addSegment(new Line(right, top, left, top), false);
+            polyline.addSegment(new Line(left, top, left, bottom), false);
+            polygon.add(polyline, false);
+            geom = new SimpleGeom(polygon);
+          }
+          return new Object[] {geom, id, x + 1, y + 1, baseX + x, baseY + y};
+        }
+
+        public boolean moveNext() {
+          return ++id < area;
+        }
+
+        public void reset() {
+          id = -1;
+        }
+
+        public void close() {
+        }
+      };
+    }
+  }
+
 }
diff --git a/core/src/main/java/org/apache/calcite/sql/fun/SqlGeoFunctions.java b/core/src/main/java/org/apache/calcite/sql/fun/SqlGeoFunctions.java
new file mode 100644
index 0000000..dd4a411
--- /dev/null
+++ b/core/src/main/java/org/apache/calcite/sql/fun/SqlGeoFunctions.java
@@ -0,0 +1,134 @@
+/*
+ * 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.sql.fun;
+
+import org.apache.calcite.DataContext;
+import org.apache.calcite.config.CalciteConnectionConfig;
+import org.apache.calcite.linq4j.Enumerable;
+import org.apache.calcite.linq4j.Linq4j;
+import org.apache.calcite.rel.type.RelDataType;
+import org.apache.calcite.rel.type.RelDataTypeFactory;
+import org.apache.calcite.runtime.GeoFunctions;
+import org.apache.calcite.schema.ScannableTable;
+import org.apache.calcite.schema.Schema;
+import org.apache.calcite.schema.Statistic;
+import org.apache.calcite.schema.Statistics;
+import org.apache.calcite.sql.SqlCall;
+import org.apache.calcite.sql.SqlNode;
+import org.apache.calcite.sql.type.SqlTypeName;
+import org.apache.calcite.util.ImmutableBitSet;
+
+import com.esri.core.geometry.Envelope;
+import com.esri.core.geometry.Geometry;
+import com.google.common.collect.ImmutableList;
+
+import java.math.BigDecimal;
+
+/**
+ * Utilities for Geo/Spatial functions.
+ *
+ * <p>Includes some table functions, and may in future include other functions
+ * that have dependencies beyond the {@code org.apache.calcite.runtime} package.
+ */
+public class SqlGeoFunctions {
+  private SqlGeoFunctions() {}
+
+  // Geometry table functions =================================================
+
+  /** Calculates a regular grid of polygons based on {@code geom}.
+   *
+   * @see GeoFunctions#ST_MakeGrid */
+  @SuppressWarnings({"WeakerAccess", "unused"})
+  public static ScannableTable ST_MakeGrid(final GeoFunctions.Geom geom,
+      final BigDecimal deltaX, final BigDecimal deltaY) {
+    return new GridTable(geom, deltaX, deltaY, false);
+  }
+
+  /** Calculates a regular grid of points based on {@code geom}.
+   *
+   * @see GeoFunctions#ST_MakeGridPoints */
+  @SuppressWarnings({"WeakerAccess", "unused"})
+  public static ScannableTable ST_MakeGridPoints(final GeoFunctions.Geom geom,
+      final BigDecimal deltaX, final BigDecimal deltaY) {
+    return new GridTable(geom, deltaX, deltaY, true);
+  }
+
+  /** Returns the points or rectangles in a grid that covers a given
+   * geometry. */
+  public static class GridTable implements ScannableTable {
+    private final GeoFunctions.Geom geom;
+    private final BigDecimal deltaX;
+    private final BigDecimal deltaY;
+    private boolean point;
+
+    GridTable(GeoFunctions.Geom geom, BigDecimal deltaX, BigDecimal deltaY,
+        boolean point) {
+      this.geom = geom;
+      this.deltaX = deltaX;
+      this.deltaY = deltaY;
+      this.point = point;
+    }
+
+    @Override public RelDataType getRowType(RelDataTypeFactory typeFactory) {
+      return typeFactory.builder()
+          // a point (for ST_MakeGridPoints) or a rectangle (for ST_MakeGrid)
+          .add("THE_GEOM", SqlTypeName.GEOMETRY)
+          // in [0, width * height)
+          .add("ID", SqlTypeName.INTEGER)
+          // in [1, width]
+          .add("ID_COL", SqlTypeName.INTEGER)
+          // in [1, height]
+          .add("ID_ROW", SqlTypeName.INTEGER)
+          // absolute column, with 0 somewhere near the origin; not standard
+          .add("ABS_COL", SqlTypeName.INTEGER)
+          // absolute row, with 0 somewhere near the origin; not standard
+          .add("ABS_ROW", SqlTypeName.INTEGER)
+          .build();
+    }
+
+    public Enumerable<Object[]> scan(DataContext root) {
+      if (geom != null && deltaX != null && deltaY != null) {
+        final Geometry geometry = geom.g();
+        final Envelope envelope = new Envelope();
+        geometry.queryEnvelope(envelope);
+        if (deltaX.compareTo(BigDecimal.ZERO) > 0
+            && deltaY.compareTo(BigDecimal.ZERO) > 0) {
+          return new GeoFunctions.GridEnumerable(envelope, deltaX, deltaY,
+              point);
+        }
+      }
+      return Linq4j.emptyEnumerable();
+    }
+
+    public Statistic getStatistic() {
+      return Statistics.of(100d, ImmutableList.of(ImmutableBitSet.of(0, 1)));
+    }
+
+    public Schema.TableType getJdbcTableType() {
+      return Schema.TableType.OTHER;
+    }
+
+    public boolean isRolledUp(String column) {
+      return false;
+    }
+
+    public boolean rolledUpColumnValidInsideAgg(String column, SqlCall call,
+        SqlNode parent, CalciteConnectionConfig config) {
+      return false;
+    }
+  }
+}
diff --git a/core/src/main/java/org/apache/calcite/sql/fun/SqlLibraryOperatorTableFactory.java b/core/src/main/java/org/apache/calcite/sql/fun/SqlLibraryOperatorTableFactory.java
index 85dec50..0444d16 100644
--- a/core/src/main/java/org/apache/calcite/sql/fun/SqlLibraryOperatorTableFactory.java
+++ b/core/src/main/java/org/apache/calcite/sql/fun/SqlLibraryOperatorTableFactory.java
@@ -86,8 +86,8 @@ public class SqlLibraryOperatorTableFactory {
         break;
       case SPATIAL:
         list.addAll(
-            CalciteCatalogReader.operatorTable(GeoFunctions.class.getName())
-                .getOperatorList());
+            CalciteCatalogReader.operatorTable(GeoFunctions.class.getName(),
+                SqlGeoFunctions.class.getName()).getOperatorList());
         break;
       default:
         custom = true;
diff --git a/core/src/test/java/org/apache/calcite/test/CalciteAssert.java b/core/src/test/java/org/apache/calcite/test/CalciteAssert.java
index 26e96f6..a801c05 100644
--- a/core/src/test/java/org/apache/calcite/test/CalciteAssert.java
+++ b/core/src/test/java/org/apache/calcite/test/CalciteAssert.java
@@ -53,6 +53,7 @@ import org.apache.calcite.schema.impl.ViewTable;
 import org.apache.calcite.schema.impl.ViewTableMacro;
 import org.apache.calcite.sql.SqlDialect;
 import org.apache.calcite.sql.SqlExplainLevel;
+import org.apache.calcite.sql.fun.SqlGeoFunctions;
 import org.apache.calcite.sql.type.SqlTypeName;
 import org.apache.calcite.sql.validate.SqlConformanceEnum;
 import org.apache.calcite.sql.validate.SqlValidatorException;
@@ -750,6 +751,7 @@ public class CalciteAssert {
     final SchemaPlus scott;
     final ConnectionSpec cs;
     final DataSource dataSource;
+    final ImmutableList<String> emptyPath = ImmutableList.of();
     switch (schema) {
     case REFLECTIVE_FOODMART:
       return rootSchema.add(schema.schemaName,
@@ -793,16 +795,37 @@ public class CalciteAssert {
       foodmart = addSchemaIfNotExists(rootSchema, SchemaSpec.JDBC_FOODMART);
       return rootSchema.add("foodmart2", new CloneSchema(foodmart));
     case GEO:
-      ModelHandler.addFunctions(rootSchema, null, ImmutableList.of(),
+      ModelHandler.addFunctions(rootSchema, null, emptyPath,
           GeoFunctions.class.getName(), "*", true);
+      ModelHandler.addFunctions(rootSchema, null, emptyPath,
+          SqlGeoFunctions.class.getName(), "*", true);
       final SchemaPlus s =
           rootSchema.add(schema.schemaName, new AbstractSchema());
-      ModelHandler.addFunctions(s, "countries", ImmutableList.of(),
+      ModelHandler.addFunctions(s, "countries", emptyPath,
           CountriesTableFunction.class.getName(), null, false);
       final String sql = "select * from table(\"countries\"(true))";
       final ViewTableMacro viewMacro = ViewTable.viewMacro(rootSchema, sql,
-          ImmutableList.of("GEO"), ImmutableList.of(), false);
+          ImmutableList.of("GEO"), emptyPath, false);
       s.add("countries", viewMacro);
+
+      ModelHandler.addFunctions(s, "states", emptyPath,
+          StatesTableFunction.class.getName(), "states", false);
+      final String sql2 = "select \"name\",\n"
+          + " ST_PolyFromText(\"geom\") as \"geom\"\n"
+          + "from table(\"states\"(true))";
+      final ViewTableMacro viewMacro2 = ViewTable.viewMacro(rootSchema, sql2,
+          ImmutableList.of("GEO"), emptyPath, false);
+      s.add("states", viewMacro2);
+
+      ModelHandler.addFunctions(s, "parks", emptyPath,
+          StatesTableFunction.class.getName(), "parks", false);
+      final String sql3 = "select \"name\",\n"
+          + " ST_PolyFromText(\"geom\") as \"geom\"\n"
+          + "from table(\"parks\"(true))";
+      final ViewTableMacro viewMacro3 = ViewTable.viewMacro(rootSchema, sql3,
+          ImmutableList.of("GEO"), emptyPath, false);
+      s.add("parks", viewMacro3);
+
       return s;
     case HR:
       return rootSchema.add(schema.schemaName,
@@ -835,7 +858,7 @@ public class CalciteAssert {
                   + "    ('Grace', 60, 'F'),\n"
                   + "    ('Wilma', cast(null as integer), 'F'))\n"
                   + "  as t(ename, deptno, gender)",
-              ImmutableList.of(), ImmutableList.of("POST", "EMP"),
+              emptyPath, ImmutableList.of("POST", "EMP"),
               null));
       post.add("DEPT",
           ViewTable.viewMacro(post,
@@ -844,7 +867,7 @@ public class CalciteAssert {
                   + "    (20, 'Marketing'),\n"
                   + "    (30, 'Engineering'),\n"
                   + "    (40, 'Empty')) as t(deptno, dname)",
-              ImmutableList.of(), ImmutableList.of("POST", "DEPT"),
+              emptyPath, ImmutableList.of("POST", "DEPT"),
               null));
       post.add("DEPT30",
           ViewTable.viewMacro(post,
@@ -860,7 +883,7 @@ public class CalciteAssert {
                   + "    (120, 'Wilma', 20, 'F',                   CAST(NULL AS VARCHAR(20)), 1,                 5, UNKNOWN, TRUE,  DATE '2005-09-07'),\n"
                   + "    (130, 'Alice', 40, 'F',                   'Vancouver',               2, CAST(NULL AS INT), FALSE,   TRUE,  DATE '2007-01-01'))\n"
                   + " as t(empno, name, deptno, gender, city, empid, age, slacker, manager, joinedat)",
-              ImmutableList.of(), ImmutableList.of("POST", "EMPS"),
+              emptyPath, ImmutableList.of("POST", "EMPS"),
               null));
       post.add("TICKER",
           ViewTable.viewMacro(post,
diff --git a/core/src/test/java/org/apache/calcite/test/StatesTableFunction.java b/core/src/test/java/org/apache/calcite/test/StatesTableFunction.java
new file mode 100644
index 0000000..2d72fd0
--- /dev/null
+++ b/core/src/test/java/org/apache/calcite/test/StatesTableFunction.java
@@ -0,0 +1,121 @@
+/*
+ * 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.test;
+
+import org.apache.calcite.DataContext;
+import org.apache.calcite.config.CalciteConnectionConfig;
+import org.apache.calcite.linq4j.Enumerable;
+import org.apache.calcite.linq4j.Linq4j;
+import org.apache.calcite.rel.type.RelDataType;
+import org.apache.calcite.rel.type.RelDataTypeFactory;
+import org.apache.calcite.schema.ScannableTable;
+import org.apache.calcite.schema.Schema;
+import org.apache.calcite.schema.Statistic;
+import org.apache.calcite.schema.Statistics;
+import org.apache.calcite.sql.SqlCall;
+import org.apache.calcite.sql.SqlNode;
+import org.apache.calcite.sql.type.SqlTypeName;
+import org.apache.calcite.util.ImmutableBitSet;
+
+import com.google.common.collect.ImmutableList;
+
+/** A table function that returns states and their boundaries; also national
+ * parks.
+ *
+ * <p>Has same content as
+ * <code>file/src/test/resources/geo/states.json</code>. */
+public class StatesTableFunction {
+  private StatesTableFunction() {}
+
+  private static final Object[][] STATE_ROWS = {
+      {"NV", "Polygon((-120 42, -114 42, -114 37, -114.75 35.1, -120 39,"
+          + " -120 42))"},
+      {"UT", "Polygon((-114 42, -111.05 42, -111.05 41, -109.05 41, -109.05 37,"
+          + " -114 37, -114 42))"},
+      {"CA", "Polygon((-124.25 42, -120 42, -120 39, -114.75 35.1,"
+          + " -114.75 32.5, -117.15 32.5, -118.30 33.75, -120.5 34.5,"
+          + " -122.4 37.2, -124.25 42))"},
+      {"AZ", "Polygon((-114 37, -109.05 37, -109.05 31.33, -111.07 31.33,"
+          + " -114.75 32.5, -114.75 35.1, -114 37))"},
+      {"CO", "Polygon((-109.05 41, -102 41, -102 37, -109.05 37, -109.05 41))"},
+      {"OR", "Polygon((-123.9 46.2, -122.7 45.7, -119 46, -117 46, -116.5 45.5,"
+          + " -117.03 44.2, -117.03 42, -124.25 42, -124.6 42.8,"
+          + " -123.9 46.2))"},
+      {"WA", "Polygon((-124.80 48.4, -123.2 48.2, -123.2 49, -117 49, -117 46,"
+          + " -119 46, -122.7 45.7, -123.9 46.2, -124.80 48.4))"},
+      {"ID", "Polygon((-117 49, -116.05 49, -116.05 48, -114.4 46.6,"
+          + " -112.9 44.45, -111.05 44.45, -111.05 42, -117.03 42,"
+          + " -117.03 44.2, -116.5 45.5, -117 46, -117 49))"},
+      {"MT", "Polygon((-116.05 49, -104.05 49, -104.05 45, -111.05 45,"
+          + " -111.05 44.45, -112.9 44.45, -114.4 46.6, -116.05 48,"
+          + " -116.05 49))"},
+      {"WY", "Polygon((-111.05 45, -104.05 45, -104.05 41, -111.05 41,"
+          + " -111.05 45))"},
+      {"NM", "Polygon((-109.05 37, -103 37, -103 32, -106.65 32, -106.5 31.8,"
+          + " -108.2 31.8, -108.2 31.33, -109.05 31.33, -109.05 37))"}
+  };
+
+  private static final Object[][] PARK_ROWS = {
+      {"Yellowstone NP", "Polygon((-111.2 45.1, -109.30 45.1, -109.30 44.1,"
+          + " -109 43.8, -110 43, -111.2 43.4, -111.2 45.1))"},
+      {"Yosemite NP", "Polygon((-120.2 38, -119.30 38.2, -119 37.7,"
+          + " -119.9 37.6, -120.2 38))"},
+      {"Death Valley NP", "Polygon((-118.2 37.3, -117 37, -116.3 35.7,"
+          + " -117 35.7, -117.2 36.2, -117.8 36.4, -118.2 37.3))"},
+  };
+
+  public static ScannableTable states(boolean b) {
+    return eval(STATE_ROWS);
+  };
+
+  public static ScannableTable parks(boolean b) {
+    return eval(PARK_ROWS);
+  };
+
+  private static ScannableTable eval(final Object[][] rows) {
+    return new ScannableTable() {
+      public Enumerable<Object[]> scan(DataContext root) {
+        return Linq4j.asEnumerable(rows);
+      }
+
+      public RelDataType getRowType(RelDataTypeFactory typeFactory) {
+        return typeFactory.builder()
+            .add("name", SqlTypeName.VARCHAR)
+            .add("geom", SqlTypeName.VARCHAR)
+            .build();
+      }
+
+      public Statistic getStatistic() {
+        return Statistics.of(rows.length,
+            ImmutableList.of(ImmutableBitSet.of(0)));
+      }
+
+      public Schema.TableType getJdbcTableType() {
+        return Schema.TableType.TABLE;
+      }
+
+      public boolean isRolledUp(String column) {
+        return false;
+      }
+
+      public boolean rolledUpColumnValidInsideAgg(String column, SqlCall call,
+          SqlNode parent, CalciteConnectionConfig config) {
+        return false;
+      }
+    };
+  }
+}
diff --git a/core/src/test/resources/sql/spatial.iq b/core/src/test/resources/sql/spatial.iq
index 67e1d46..a9aac81 100644
--- a/core/src/test/resources/sql/spatial.iq
+++ b/core/src/test/resources/sql/spatial.iq
@@ -24,6 +24,16 @@ C
 245
 !ok
 
+select count(*) as c from GEO."states";
+C
+11
+!ok
+
+select count(*) as c from GEO."parks";
+C
+3
+!ok
+
 #### Geometry conversion functions (2D)
 
 # ST_AsBinary(geom) Geometry to Well Known Binary
@@ -258,10 +268,28 @@ EXPR$0
 # Not implemented
 
 # ST_MakeGrid(geom, deltaX, deltaY) Calculates a regular grid of polygons based on *geom*
-# Not implemented
+SELECT * FROM TABLE(ST_MakeGrid(ST_Point(13.0,22.0), 10.0, 5.0));
+THE_GEOM, ID, ID_COL, ID_ROW, ABS_COL, ABS_ROW
+{"rings":[[[10,20],[20,20],[20,25],[10,25],[10,20]]]}, 0, 1, 1, 1, 4
+!ok
 
 # ST_MakeGridPoints(geom, deltaX, deltaY) Calculates a regular grid of points based on *geom*
-# Not implemented
+SELECT * FROM TABLE(ST_MakeGridPoints(ST_Point(13.0,22.0), 10.0, 5.0));
+THE_GEOM, ID, ID_COL, ID_ROW, ABS_COL, ABS_ROW
+{"x":15,"y":22.5}, 0, 1, 1, 1, 4
+!ok
+
+# Call ST_MakeGridPoints for each geometry in a set
+select "name", "latitude", "longitude", p.*
+from GEO."countries" AS c,
+  lateral table(
+    ST_MakeGridPoints(ST_MakePoint("latitude", "longitude"), 10.0, 10.0)) as p
+ORDER BY "latitude" DESC LIMIT 3;
+name, latitude, longitude, THE_GEOM, ID, ID_COL, ID_ROW, ABS_COL, ABS_ROW
+Svalbard and Jan Mayen, 77.553604, 23.670272, {"x":75,"y":25}, 0, 1, 1, 7, 2
+Greenland, 71.706936, -42.604303, {"x":75,"y":-45}, 0, 1, 1, 7, -5
+Iceland, 64.963051, -19.020835, {"x":65,"y":-15}, 0, 1, 1, 6, -2
+!ok
 
 # ST_MakeLine(point1 [, point ]*) Creates a line-string from the given points (or multi-points)
 
@@ -1143,4 +1171,68 @@ POINT Z (-2 3 1)
 POINT Z (5 5 5)
 !ok
 
+# Polygon-to-polygon joins
+select *
+from GEO."states" as s
+order by "name";
+name, geom
+AZ, {"rings":[[[-114,37],[-109.05,37],[-109.05,31.33],[-111.07,31.33],[-114.75,32.5],[-114.75,35.1],[-114,37]]]}
+CA, {"rings":[[[-124.25,42],[-120,42],[-120,39],[-114.75,35.1],[-114.75,32.5],[-117.15,32.5],[-118.3,33.75],[-120.5,34.5],[-122.4,37.2],[-124.25,42]]]}
+CO, {"rings":[[[-109.05,41],[-102,41],[-102,37],[-109.05,37],[-109.05,41]]]}
+ID, {"rings":[[[-117,49],[-116.05,49],[-116.05,48],[-114.4,46.6],[-112.9,44.45],[-111.05,44.45],[-111.05,42],[-117.03,42],[-117.03,44.2],[-116.5,45.5],[-117,46],[-117,49]]]}
+MT, {"rings":[[[-116.05,49],[-104.05,49],[-104.05,45],[-111.05,45],[-111.05,44.45],[-112.9,44.45],[-114.4,46.6],[-116.05,48],[-116.05,49]]]}
+NM, {"rings":[[[-109.05,37],[-103,37],[-103,32],[-106.65,32],[-106.5,31.8],[-108.2,31.8],[-108.2,31.33],[-109.05,31.33],[-109.05,37]]]}
+NV, {"rings":[[[-120,42],[-114,42],[-114,37],[-114.75,35.1],[-120,39],[-120,42]]]}
+OR, {"rings":[[[-123.9,46.2],[-122.7,45.7],[-119,46],[-117,46],[-116.5,45.5],[-117.03,44.2],[-117.03,42],[-124.25,42],[-124.6,42.8],[-123.9,46.2]]]}
+UT, {"rings":[[[-114,42],[-111.05,42],[-111.05,41],[-109.05,41],[-109.05,37],[-114,37],[-114,42]]]}
+WA, {"rings":[[[-124.8,48.4],[-123.2,48.2],[-123.2,49],[-117,49],[-117,46],[-119,46],[-122.7,45.7],[-123.9,46.2],[-124.8,48.4]]]}
+WY, {"rings":[[[-111.05,45],[-104.05,45],[-104.05,41],[-111.05,41],[-111.05,45]]]}
+!ok
+
+select *
+from GEO."parks" as p
+order by "name";
+name, geom
+Death Valley NP, {"rings":[[[-118.2,37.3],[-117,37],[-116.3,35.7],[-117,35.7],[-117.2,36.2],[-117.8,36.4],[-118.2,37.3]]]}
+Yellowstone NP, {"rings":[[[-111.2,45.1],[-109.3,45.1],[-109.3,44.1],[-109,43.8],[-110,43],[-111.2,43.4],[-111.2,45.1]]]}
+Yosemite NP, {"rings":[[[-120.2,38],[-119.3,38.2],[-119,37.7],[-119.9,37.6],[-120.2,38]]]}
+!ok
+
+# Parks that may intersect states
+select s."name", p."name"
+from GEO."states" as s
+cross apply table(ST_MakeGrid(s."geom", 5.0, 5.0)) as sg,
+    GEO."parks" as p
+cross apply table(ST_MakeGrid(p."geom", 5.0, 5.0)) as pg
+where (sg.abs_col, sg.abs_row) = (pg.abs_col, pg.abs_row)
+order by 2, 1;
+name, name
+CA, Death Valley NP
+NV, Death Valley NP
+ID, Yellowstone NP
+MT, Yellowstone NP
+NV, Yellowstone NP
+UT, Yellowstone NP
+WY, Yellowstone NP
+CA, Yosemite NP
+!ok
+
+# Parks that intersect states
+select s."name", p."name"
+from GEO."states" as s
+cross apply table(ST_MakeGrid(s."geom", 5.0, 5.0)) as sg,
+    GEO."parks" as p
+cross apply table(ST_MakeGrid(p."geom", 5.0, 5.0)) as pg
+where (sg.abs_col, sg.abs_row) = (pg.abs_col, pg.abs_row)
+and ST_Intersects(s."geom", p."geom")
+order by 2, 1;
+name, name
+CA, Death Valley NP
+NV, Death Valley NP
+ID, Yellowstone NP
+MT, Yellowstone NP
+WY, Yellowstone NP
+CA, Yosemite NP
+!ok
+
 # End spatial.iq
diff --git a/file/src/test/resources/geo/states.json b/file/src/test/resources/geo/states.json
new file mode 100644
index 0000000..0a32aa0
--- /dev/null
+++ b/file/src/test/resources/geo/states.json
@@ -0,0 +1,168 @@
+/*
+ * 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.
+ *
+ * GeoJSON document containing 11 western states and 3 national parks;
+ * some of the parks cross state boundaries.
+ * Created using Simple GeoJSON editor, representing most regions as polygons
+ * with fewer than 10 points, then rounded coordinates to 1/10 or 1/100 degree,
+ * and aligned the borders of neighboring states.
+ */
+{
+  "type": "FeatureCollection",
+  "features": [
+    {
+      "type": "Feature",
+      "geometry": {
+        "type": "Polygon",
+        "coordinates": [[[-120, 42], [-114, 42], [-114, 37], [-114.75, 35.1], [-120, 39], [-120, 42]]]
+      },
+      "properties": {
+        "name": "NV"
+      }
+    },
+    {
+      "type": "Feature",
+      "geometry": {
+        "type": "Polygon",
+        "coordinates": [[[-114, 42], [-111.05, 42], [-111.05, 41], [-109.05, 41], [-109.05, 37], [-114, 37], [-114, 42]]]
+      },
+      "properties": {
+        "name": "UT"
+      }
+    },
+    {
+      "type": "Feature",
+      "geometry": {
+        "type": "Polygon",
+        "coordinates": [[[-124.25, 42], [-120, 42], [-120, 39], [-114.75, 35.1], [-114.75, 32.5], [-117.15, 32.5], [-118.30, 33.75], [-120.5, 34.5], [-122.4, 37.2], [-124.25, 42]]]
+      },
+      "properties": {
+        "name": "CA"
+      }
+    },
+    {
+      "type": "Feature",
+      "geometry": {
+        "type": "Polygon",
+        "coordinates": [[[-114, 37], [-109.05, 37], [-109.05, 31.33], [-111.07, 31.33], [-114.75, 32.5], [-114.75, 35.1], [-114, 37]]]
+      },
+      "properties": {
+        "name": "AZ"
+      }
+    },
+    {
+      "type": "Feature",
+      "geometry": {
+        "type": "Polygon",
+        "coordinates": [[[-109.05, 41], [-102, 41], [-102, 37], [-109.05, 37], [-109.05, 41]]]
+      },
+      "properties": {
+        "name": "CO"
+      }
+    },
+    {
+      "type": "Feature",
+      "geometry": {
+        "type": "Polygon",
+        "coordinates": [[[-123.9, 46.2], [-122.7, 45.7], [-119, 46], [-117, 46], [-116.5, 45.5], [-117.03, 44.2], [-117.03, 42], [-124.25, 42], [-124.6, 42.8], [-123.9, 46.2]]]
+      },
+      "properties": {
+        "name": "OR"
+      }
+    },
+    {
+      "type": "Feature",
+      "geometry": {
+        "type": "Polygon",
+        "coordinates": [[[-124.80, 48.4], [-123.2, 48.2], [-123.2, 49], [-117, 49], [-117, 46], [-119, 46], [-122.7, 45.7], [-123.9, 46.2], [-124.80, 48.4]]]
+      },
+      "properties": {
+        "name": "WA"
+      }
+    },
+    {
+      "type": "Feature",
+      "geometry": {
+        "type": "Polygon",
+        "coordinates": [[[-117, 49], [-116.05, 49], [-116.05, 48], [-114.4, 46.6], [-112.9, 44.45], [-111.05, 44.45], [-111.05, 42], [-117.03, 42], [-117.03, 44.2], [-116.5, 45.5], [-117, 46], [-117, 49]]]
+      },
+      "properties": {
+        "name": "ID"
+      }
+    },
+    {
+      "type": "Feature",
+      "geometry": {
+        "type": "Polygon",
+        "coordinates": [[[-116.05, 49], [-104.05, 49], [-104.05, 45], [-111.05, 45], [-111.05, 44.45], [-112.9, 44.45], [-114.4, 46.6], [-116.05, 48], [-116.05, 49]]
+        ]
+      },
+      "properties": {
+        "name": "MT"
+      }
+    },
+    {
+      "type": "Feature",
+      "geometry": {
+        "type": "Polygon",
+        "coordinates": [[[-111.05, 45], [-104.05, 45], [-104.05, 41], [-111.05, 41], [-111.05, 45]]]
+      },
+      "properties": {
+        "name": "WY"
+      }
+    },
+    {
+      "type": "Feature",
+      "geometry": {
+        "type": "Polygon",
+        "coordinates": [[[-109.05, 37], [-103, 37], [-103, 32], [-106.65, 32], [-106.5, 31.8], [-108.2, 31.8], [-108.2, 31.33], [-109.05, 31.33], [-109.05, 37]]]
+      },
+      "properties": {
+        "name": "NM"
+      }
+    },
+    {
+      "type": "Feature",
+      "geometry": {
+        "type": "Polygon",
+        "coordinates": [[[-111.2, 45.1], [-109.30, 45.1], [-109.30, 44.1], [-109, 43.8], [-110, 43], [-111.2, 43.4], [-111.2, 45.1]]]
+      },
+      "properties": {
+        "name": "Yellowstone NP"
+      }
+    },
+    {
+      "type": "Feature",
+      "geometry": {
+        "type": "Polygon",
+        "coordinates": [[[-120.2, 38], [-119.30, 38.2], [-119, 37.7], [-119.9, 37.6], [-120.2, 38]]]
+      },
+      "properties": {
+        "name": "Yosemite NP"
+      }
+    },
+    {
+      "type": "Feature",
+      "geometry": {
+        "type": "Polygon",
+        "coordinates": [[[-118.2, 37.3], [-117, 37], [-116.3, 35.7], [-117, 35.7], [-117.2, 36.2], [-117.8, 36.4], [-118.2, 37.3]]]
+      },
+      "properties": {
+        "name": "Death Valley NP"
+      }
+    }
+  ]
+}
diff --git a/site/_docs/reference.md b/site/_docs/reference.md
index becd53e..3bf384f 100644
--- a/site/_docs/reference.md
+++ b/site/_docs/reference.md
@@ -1962,7 +1962,9 @@ In the "C" (for "compatibility") column, "o" indicates that the function
 implements the OpenGIS Simple Features Implementation Specification for SQL,
 [version 1.2.1](https://www.opengeospatial.org/standards/sfs);
 "p" indicates that the function is a
-[PostGIS](https://www.postgis.net/docs/reference.html) extension to OpenGIS.
+[PostGIS](https://www.postgis.net/docs/reference.html) extension to OpenGIS;
+"h" indicates that the function is an
+[H2GIS](http://www.h2gis.org/docs/dev/functions/) extension.
 
 #### Geometry conversion functions (2D)
 
@@ -2004,6 +2006,8 @@ Not implemented:
 
 | C | Operator syntax      | Description
 |:- |:-------------------- |:-----------
+| h | ST_MakeGrid(geom, deltaX, deltaY) | Calculates a regular grid of POLYGONs based on *geom*
+| h | ST_MakeGridPoints(geom, deltaX, deltaY) | Calculates a regular grid of points based on *geom*
 | o | ST_MakeLine(point1 [, point ]*) | Creates a line-string from the given POINTs (or MULTIPOINTs)
 | p | ST_MakePoint(x, y [, z ]) | Alias for `ST_Point`
 | o | ST_Point(x, y [, z ]) | Constructs a point from two or three coordinates
@@ -2015,8 +2019,6 @@ Not implemented:
 * ST_Expand(geom, deltaX, deltaY) Expands *geom*'s envelope
 * ST_MakeEllipse(point, width, height) Constructs an ellipse
 * ST_MakeEnvelope(xMin, yMin, xMax, yMax  [, srid ]) Creates a rectangular POLYGON
-* ST_MakeGrid(geom, deltaX, deltaY) Calculates a regular grid of POLYGONs based on *geom*
-* ST_MakeGridPoints(geom, deltaX, deltaY) Calculates a regular grid of points based on *geom*
 * ST_MakePolygon(lineString [, hole ]*) Creates a POLYGON from *lineString* with the given holes (which are required to be closed LINESTRINGs)
 * ST_MinimumDiameter(geom) Returns the minimum diameter of *geom*
 * ST_MinimumRectangle(geom) Returns the minimum rectangle enclosing *geom*