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:15 UTC

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

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 =